mirror of
https://github.com/apache/sqoop.git
synced 2025-05-03 01:50:00 +08:00
SQOOP-1287: Add high performance Oracle connector into Sqoop
(David Robson via Venkat Ranganathan)
This commit is contained in:
parent
d03faf3544
commit
6bfaa9d653
@ -85,9 +85,12 @@ jdbc:mysql://localhost/
|
|||||||
|
|
||||||
=== Oracle
|
=== 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.
|
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
|
Use the system property sqoop.test.oracle.connectstring to specify the
|
||||||
connection string for Oracle host used for testing. Specify this property on the
|
connection string for Oracle host used for testing. Specify this property on the
|
||||||
command line or via the build.properties file. For example:
|
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:
|
If not specified, the default value used for this property is:
|
||||||
jdbc:oracle:thin:@//localhost/xe
|
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
|
=== PostgreSQL
|
||||||
|
|
||||||
Install PostgreSQL 8.3.9. Download the postgresql 8.4 jdbc driver. Instructions
|
Install PostgreSQL 8.3.9. Download the postgresql 8.4 jdbc driver. Instructions
|
||||||
|
@ -827,6 +827,11 @@
|
|||||||
<mkdir dir="${cobertura.class.dir}" />
|
<mkdir dir="${cobertura.class.dir}" />
|
||||||
<copy file="${test.dir}/fi-site.xml"
|
<copy file="${test.dir}/fi-site.xml"
|
||||||
todir="${test.build.extraconf}" />
|
todir="${test.build.extraconf}" />
|
||||||
|
<copy file="${basedir}/conf/oraoop-site-template.xml"
|
||||||
|
todir="${test.build.extraconf}" />
|
||||||
|
<copy todir="${test.build.extraconf}/oraoop">
|
||||||
|
<fileset dir="${test.dir}/oraoop"/>
|
||||||
|
</copy>
|
||||||
<junit
|
<junit
|
||||||
printsummary="yes" showoutput="${test.output}"
|
printsummary="yes" showoutput="${test.output}"
|
||||||
haltonfailure="no" fork="yes" maxmemory="512m"
|
haltonfailure="no" fork="yes" maxmemory="512m"
|
||||||
|
103
conf/oraoop-site-template.xml
Normal file
103
conf/oraoop-site-template.xml
Normal file
@ -0,0 +1,103 @@
|
|||||||
|
<?xml version="1.0"?>
|
||||||
|
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
|
||||||
|
<!--
|
||||||
|
Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
or more contributor license agreements. See the NOTICE file
|
||||||
|
distributed with this work for additional information
|
||||||
|
regarding copyright ownership. The ASF licenses this file
|
||||||
|
to you under the Apache License, Version 2.0 (the
|
||||||
|
"License"); you may not use this file except in compliance
|
||||||
|
with the License. You may obtain a copy of the License at
|
||||||
|
|
||||||
|
http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
|
||||||
|
Unless required by applicable law or agreed to in writing,
|
||||||
|
software distributed under the License is distributed on an
|
||||||
|
"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
KIND, either express or implied. See the License for the
|
||||||
|
specific language governing permissions and limitations
|
||||||
|
under the License.
|
||||||
|
-->
|
||||||
|
|
||||||
|
<!-- Put OraOop-specific properties in this file. -->
|
||||||
|
|
||||||
|
<configuration>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>oraoop.oracle.session.initialization.statements</name>
|
||||||
|
<value>alter session disable parallel query;
|
||||||
|
alter session set "_serial_direct_read"=true;
|
||||||
|
alter session set tracefile_identifier=oraoop;
|
||||||
|
--alter session set events '10046 trace name context forever, level 8';
|
||||||
|
</value>
|
||||||
|
<description>A semicolon-delimited list of Oracle statements that are executed, in order, to initialize each Oracle session.
|
||||||
|
Use {[property_name]|[default_value]} characters to refer to a Sqoop/Hadoop configuration property.
|
||||||
|
If the property does not exist, the specified default value will be used.
|
||||||
|
E.g. {oracle.sessionTimeZone|GMT} will equate to the value of the property named "oracle.sessionTimeZone" or
|
||||||
|
to "GMT" if this property has not been set.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>mapred.map.tasks.speculative.execution</name>
|
||||||
|
<value>false</value>
|
||||||
|
<description>Speculative execution is disabled to prevent redundant load on the Oracle database.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>oraoop.import.hint</name>
|
||||||
|
<value>NO_INDEX(t)</value>
|
||||||
|
<description>Hint to add to the SELECT statement for an IMPORT job.
|
||||||
|
The table will have an alias of t which can be used in the hint.
|
||||||
|
By default the NO_INDEX hint is applied to stop the use of an index.
|
||||||
|
To override this in oraoop-site.xml set the value to a blank string.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
|
||||||
|
<!--
|
||||||
|
<property>
|
||||||
|
<name>oraoop.block.allocation</name>
|
||||||
|
<value>ROUNDROBIN</value>
|
||||||
|
<description>Supported values are: ROUNDROBIN or SEQUENTIAL or RANDOM.
|
||||||
|
Refer to the OraOop documentation for more details.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
-->
|
||||||
|
|
||||||
|
<!--
|
||||||
|
<property>
|
||||||
|
<name>oraoop.import.omit.lobs.and.long</name>
|
||||||
|
<value>false</value>
|
||||||
|
<description>If true, OraOop will omit BLOB, CLOB, NCLOB and LONG columns during an Import.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
-->
|
||||||
|
|
||||||
|
<!--
|
||||||
|
<property>
|
||||||
|
<name>oraoop.table.import.where.clause.location</name>
|
||||||
|
<value>SUBSPLIT</value>
|
||||||
|
<description>Supported values are: SUBSPLIT or SPLIT.
|
||||||
|
Refer to the OraOop documentation for more details.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
-->
|
||||||
|
|
||||||
|
<!--
|
||||||
|
<property>
|
||||||
|
<name>oraoop.oracle.append.values.hint.usage</name>
|
||||||
|
<value>AUTO</value>
|
||||||
|
<description>Supported values are: AUTO or ON or OFF.
|
||||||
|
ON:
|
||||||
|
OraOop will use the APPEND_VALUES Oracle hint during a Sqoop export, when inserting
|
||||||
|
data into an Oracle table.
|
||||||
|
OFF:
|
||||||
|
OraOop will not use the APPEND_VALUES Oracle hint during a Sqoop export.
|
||||||
|
AUTO:
|
||||||
|
For OraOop 1.1, the AUTO setting will not use the APPEND_VALUES hint.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
-->
|
||||||
|
|
||||||
|
</configuration>
|
@ -43,6 +43,7 @@
|
|||||||
|
|
||||||
import com.cloudera.sqoop.util.ClassLoaderStack;
|
import com.cloudera.sqoop.util.ClassLoaderStack;
|
||||||
import org.apache.sqoop.manager.GenericJdbcManager;
|
import org.apache.sqoop.manager.GenericJdbcManager;
|
||||||
|
import org.apache.sqoop.manager.oracle.OraOopManagerFactory;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Factory class to create the ConnManager type required
|
* 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
|
// The default value for sqoop.connection.factories is the
|
||||||
// name of the DefaultManagerFactory.
|
// 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 =
|
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().
|
/** The list of ManagerFactory instances consulted by getManager().
|
||||||
*/
|
*/
|
||||||
@ -84,7 +89,8 @@ public ConnFactory(Configuration conf) {
|
|||||||
private void instantiateFactories(Configuration conf) {
|
private void instantiateFactories(Configuration conf) {
|
||||||
loadManagersFromConfDir(conf);
|
loadManagersFromConfDir(conf);
|
||||||
String [] classNameArray =
|
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) {
|
for (String className : classNameArray) {
|
||||||
try {
|
try {
|
||||||
|
630
src/java/org/apache/sqoop/manager/oracle/OraOopConnManager.java
Normal file
630
src/java/org/apache/sqoop/manager/oracle/OraOopConnManager.java
Normal file
@ -0,0 +1,630 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.Connection;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.sql.Timestamp;
|
||||||
|
import java.sql.Types;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Properties;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||||
|
|
||||||
|
import com.cloudera.sqoop.SqoopOptions;
|
||||||
|
import com.cloudera.sqoop.manager.ExportJobContext;
|
||||||
|
import com.cloudera.sqoop.manager.GenericJdbcManager;
|
||||||
|
import com.cloudera.sqoop.manager.ImportJobContext;
|
||||||
|
import com.cloudera.sqoop.mapreduce.JdbcExportJob;
|
||||||
|
import com.cloudera.sqoop.mapreduce.JdbcUpdateExportJob;
|
||||||
|
import com.cloudera.sqoop.util.ExportException;
|
||||||
|
import com.cloudera.sqoop.util.ImportException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* OraOop manager for high performance Oracle import / export.
|
||||||
|
* NOTES:
|
||||||
|
* Escaping Column Names:
|
||||||
|
* ----------------------
|
||||||
|
* There are 3 main queries that occur during a Sqoop import.
|
||||||
|
* (1) Selecting columns to obtain their data-type via getColTypesQuery();
|
||||||
|
* (2) selecting column names via getColNamesQuery(); and
|
||||||
|
* (3) getting the data during the import via
|
||||||
|
* OraOopDBRecordReader.getSelectQuery();
|
||||||
|
* In each of these queries, we'd ideally escape the column names so that
|
||||||
|
* Oracle columns that require this work okay.
|
||||||
|
* Unfortunately we can't do this, because if the user specifies column
|
||||||
|
* names via the "--columns" clause, these names will be used (verbatim)
|
||||||
|
* during OraOopDBRecordReader.getSelectQuery(). This means that we could
|
||||||
|
* only escape the column names during OraOopDBRecordReader.getSelectQuery()
|
||||||
|
* if the user entered them in the correct case.
|
||||||
|
* Therefore, escapeColName() in this class does not actually do anything so
|
||||||
|
* that OraOopDBRecordReader.getSelectQuery() generates a valid SQL statement
|
||||||
|
* when the user utilises the "--columns" clause.
|
||||||
|
* However, getColTypesQuery() and getColNamesQuery() do escape column names
|
||||||
|
* via the method escapeOracleColumnName(). We also get getColumnTypes() to
|
||||||
|
* unescape the column names so that Sqoop has the most accurate column
|
||||||
|
* name strings.
|
||||||
|
*/
|
||||||
|
public class OraOopConnManager extends GenericJdbcManager {
|
||||||
|
|
||||||
|
public static final OraOopLog LOG = OraOopLogFactory
|
||||||
|
.getLog(OraOopConnManager.class.getName());
|
||||||
|
private List<String> columnNamesInOracleTable = null;
|
||||||
|
private Map<String, Integer> columnTypesInOracleTable = null;
|
||||||
|
private final String timestampJavaType;
|
||||||
|
|
||||||
|
public OraOopConnManager(final SqoopOptions sqoopOptions) {
|
||||||
|
super(OraOopConstants.ORACLE_JDBC_DRIVER_CLASS, sqoopOptions);
|
||||||
|
if (this.options.getConf().getBoolean(
|
||||||
|
OraOopConstants.ORAOOP_MAP_TIMESTAMP_AS_STRING,
|
||||||
|
OraOopConstants.ORAOOP_MAP_TIMESTAMP_AS_STRING_DEFAULT)) {
|
||||||
|
timestampJavaType = "String";
|
||||||
|
} else {
|
||||||
|
timestampJavaType = super.toJavaType(Types.TIMESTAMP);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Connection makeConnection() throws SQLException {
|
||||||
|
|
||||||
|
String connectStr = this.options.getConnectString();
|
||||||
|
String username = this.options.getUsername();
|
||||||
|
String password = this.options.getPassword();
|
||||||
|
Properties additionalProps = this.options.getConnectionParams();
|
||||||
|
|
||||||
|
Connection connection =
|
||||||
|
OracleConnectionFactory.createOracleJdbcConnection(this
|
||||||
|
.getDriverClass(), connectStr, username, password, additionalProps);
|
||||||
|
|
||||||
|
return connection;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws SQLException {
|
||||||
|
|
||||||
|
super.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<String> getColumnNamesInOracleTable(String tableName) {
|
||||||
|
|
||||||
|
if (this.columnNamesInOracleTable == null) {
|
||||||
|
|
||||||
|
OracleTable tableContext = null;
|
||||||
|
|
||||||
|
try {
|
||||||
|
tableContext = getOracleTableContext();
|
||||||
|
|
||||||
|
Configuration conf = this.options.getConf();
|
||||||
|
|
||||||
|
this.columnNamesInOracleTable =
|
||||||
|
OraOopOracleQueries.getTableColumnNames(getConnection(),
|
||||||
|
tableContext, OraOopUtilities
|
||||||
|
.omitLobAndLongColumnsDuringImport(conf), OraOopUtilities
|
||||||
|
.recallSqoopJobType(conf), true, // <-
|
||||||
|
// onlyOraOopSupportedTypes
|
||||||
|
true // <- omitOraOopPseudoColumns
|
||||||
|
);
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
throw new RuntimeException(ex);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Return a copy of our list, so the original will not be inadvertently
|
||||||
|
// altered...
|
||||||
|
return OraOopUtilities.copyStringList(this.columnNamesInOracleTable);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected List<String> getSelectedColumnNamesInOracleTable(String tableName) {
|
||||||
|
|
||||||
|
List<String> colNamesInTable = getColumnNamesInOracleTable(tableName);
|
||||||
|
|
||||||
|
String[] selectedColumns = this.options.getColumns();
|
||||||
|
if (selectedColumns != null && selectedColumns.length > 0) {
|
||||||
|
|
||||||
|
for (int idx = 0; idx < selectedColumns.length; idx++) {
|
||||||
|
|
||||||
|
String selectedColumn = selectedColumns[idx];
|
||||||
|
// If the user did not escape this column name, then we should
|
||||||
|
// uppercase it...
|
||||||
|
if (!isEscaped(selectedColumn)) {
|
||||||
|
selectedColumns[idx] = selectedColumn.toUpperCase();
|
||||||
|
} else {
|
||||||
|
// If the user escaped this column name, then we should
|
||||||
|
// retain its case...
|
||||||
|
selectedColumns[idx] = unescapeOracleColumnName(selectedColumn);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Ensure there are no duplicated column names...
|
||||||
|
String[] duplicates =
|
||||||
|
OraOopUtilities
|
||||||
|
.getDuplicatedStringArrayValues(selectedColumns, false);
|
||||||
|
if (duplicates.length > 0) {
|
||||||
|
StringBuilder msg = new StringBuilder();
|
||||||
|
msg.append("The following column names have been duplicated in the ");
|
||||||
|
msg.append("\"--columns\" clause:\n");
|
||||||
|
|
||||||
|
for (String duplicate : duplicates) {
|
||||||
|
msg.append("\t" + duplicate + "\n");
|
||||||
|
}
|
||||||
|
|
||||||
|
throw new RuntimeException(msg.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
// Ensure the user selected column names that actually exist...
|
||||||
|
for (String selectedColumn : selectedColumns) {
|
||||||
|
if (!colNamesInTable.contains(selectedColumn)) {
|
||||||
|
OracleTable tableContext = getOracleTableContext();
|
||||||
|
throw new RuntimeException(String.format(
|
||||||
|
"The column named \"%s\" does not exist within the table"
|
||||||
|
+ "%s (or is of an unsupported data-type).", selectedColumn,
|
||||||
|
tableContext.toString()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Remove any columns (that exist in the table) that were not
|
||||||
|
// selected by the user...
|
||||||
|
for (int idx = colNamesInTable.size() - 1; idx >= 0; idx--) {
|
||||||
|
String colName = colNamesInTable.get(idx);
|
||||||
|
if (!OraOopUtilities.stringArrayContains(selectedColumns, colName,
|
||||||
|
false)) {
|
||||||
|
colNamesInTable.remove(idx);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// To assist development/testing of Oracle data-types, you can use this
|
||||||
|
// to limit the number of columns from the table...
|
||||||
|
int columnNameLimit =
|
||||||
|
this.options.getConf().getInt("oraoop.column.limit", 0);
|
||||||
|
if (columnNameLimit > 0) {
|
||||||
|
columnNameLimit = Math.min(columnNameLimit, colNamesInTable.size());
|
||||||
|
colNamesInTable = colNamesInTable.subList(0, columnNameLimit);
|
||||||
|
}
|
||||||
|
|
||||||
|
return colNamesInTable;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getColTypesQuery(String tableName) {
|
||||||
|
|
||||||
|
List<String> colNames = getSelectedColumnNamesInOracleTable(tableName);
|
||||||
|
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
sb.append("SELECT ");
|
||||||
|
for (int idx = 0; idx < colNames.size(); idx++) {
|
||||||
|
if (idx > 0) {
|
||||||
|
sb.append(",");
|
||||||
|
}
|
||||||
|
sb.append(escapeOracleColumnName(colNames.get(idx))); // <- See notes at
|
||||||
|
// top about escaped
|
||||||
|
// column names
|
||||||
|
}
|
||||||
|
sb.append(String.format(" FROM %s WHERE 0=1", tableName));
|
||||||
|
|
||||||
|
return sb.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getColNamesQuery(String tableName) {
|
||||||
|
|
||||||
|
// NOTE: This code is similar to getColTypesQuery() - except the
|
||||||
|
// escaping of column names and table name differs.
|
||||||
|
|
||||||
|
List<String> colNames = getSelectedColumnNamesInOracleTable(tableName);
|
||||||
|
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
sb.append("SELECT ");
|
||||||
|
for (int idx = 0; idx < colNames.size(); idx++) {
|
||||||
|
if (idx > 0) {
|
||||||
|
sb.append(",");
|
||||||
|
}
|
||||||
|
sb.append(escapeColName(colNames.get(idx))); // <- See notes at top about
|
||||||
|
// escaped column names
|
||||||
|
}
|
||||||
|
sb.append(String.format(" FROM %s WHERE 1=0", escapeTableName(tableName)));
|
||||||
|
|
||||||
|
return sb.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getSplitColumn(SqoopOptions opts, String tableName) {
|
||||||
|
|
||||||
|
// If we're importing an Oracle table and will be generating
|
||||||
|
// "splits" based on its Oracle data-files, we don't actually require
|
||||||
|
// a primary key to exist, or for the user to identify the split-column.
|
||||||
|
// As a consequence, return "NotRequired" to prevent sqoop code
|
||||||
|
// such as SqlManager.importTable() from throwing an exception.
|
||||||
|
//
|
||||||
|
// NB: The tableName parameter will be null if no table is involved,
|
||||||
|
// such as when importing data via an (arbitrary) SQL query.
|
||||||
|
if (tableName != null) {
|
||||||
|
return OraOopConstants.TABLE_SPLIT_COLUMN_NOT_REQUIRED;
|
||||||
|
} else {
|
||||||
|
return super.getSplitColumn(opts, tableName);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void importTable(ImportJobContext context) throws IOException,
|
||||||
|
ImportException {
|
||||||
|
|
||||||
|
logImportTableDetails(context);
|
||||||
|
|
||||||
|
context.setConnManager(this);
|
||||||
|
|
||||||
|
// Specify the Oracle-specific DBInputFormat for import.
|
||||||
|
context.setInputFormat(OraOopDataDrivenDBInputFormat.class);
|
||||||
|
|
||||||
|
super.importTable(context);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void exportTable(ExportJobContext context) throws IOException,
|
||||||
|
ExportException {
|
||||||
|
|
||||||
|
logExportTableDetails(context);
|
||||||
|
|
||||||
|
if (this.columnTypesInOracleTable == null) {
|
||||||
|
throw new ExportException("The column-types for the table are not"
|
||||||
|
+ "known.");
|
||||||
|
}
|
||||||
|
if (this.columnTypesInOracleTable.containsValue(OraOopOracleQueries
|
||||||
|
.getOracleType("BINARY_DOUBLE"))) {
|
||||||
|
context.getOptions().getConf().setBoolean(
|
||||||
|
OraOopConstants.TABLE_CONTAINS_BINARY_DOUBLE_COLUMN, true);
|
||||||
|
}
|
||||||
|
if (this.columnTypesInOracleTable.containsValue(OraOopOracleQueries
|
||||||
|
.getOracleType("BINARY_FLOAT"))) {
|
||||||
|
context.getOptions().getConf().setBoolean(
|
||||||
|
OraOopConstants.TABLE_CONTAINS_BINARY_FLOAT_COLUMN, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
context.setConnManager(this);
|
||||||
|
|
||||||
|
@SuppressWarnings("rawtypes")
|
||||||
|
Class<? extends OutputFormat> oraOopOutputFormatClass;
|
||||||
|
try {
|
||||||
|
oraOopOutputFormatClass = OraOopOutputFormatInsert.class;
|
||||||
|
} catch (NoClassDefFoundError ex) {
|
||||||
|
explainWhyExportClassCannotBeLoaded(ex, "OraOopOutputFormatInsert");
|
||||||
|
throw ex;
|
||||||
|
}
|
||||||
|
JdbcExportJob exportJob =
|
||||||
|
new JdbcExportJob(context, null, null, oraOopOutputFormatClass);
|
||||||
|
exportJob.runExport();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void updateTable(ExportJobContext context) throws IOException,
|
||||||
|
ExportException {
|
||||||
|
|
||||||
|
logExportTableDetails(context);
|
||||||
|
|
||||||
|
context.setConnManager(this);
|
||||||
|
|
||||||
|
@SuppressWarnings("rawtypes")
|
||||||
|
Class<? extends OutputFormat> oraOopOutputFormatClass;
|
||||||
|
try {
|
||||||
|
oraOopOutputFormatClass = OraOopOutputFormatUpdate.class;
|
||||||
|
} catch (NoClassDefFoundError ex) {
|
||||||
|
explainWhyExportClassCannotBeLoaded(ex, "OraOopOutputFormatUpdate");
|
||||||
|
throw ex;
|
||||||
|
}
|
||||||
|
|
||||||
|
JdbcUpdateExportJob exportJob =
|
||||||
|
new JdbcUpdateExportJob(context, null, null, oraOopOutputFormatClass);
|
||||||
|
exportJob.runExport();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void finalize() throws Throwable {
|
||||||
|
|
||||||
|
close();
|
||||||
|
super.finalize();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toHiveType(int sqlType) {
|
||||||
|
|
||||||
|
String hiveType = super.toHiveType(sqlType);
|
||||||
|
|
||||||
|
if (hiveType == null) {
|
||||||
|
|
||||||
|
// http://wiki.apache.org/hadoop/Hive/Tutorial#Primitive_Types
|
||||||
|
|
||||||
|
if (sqlType == OraOopOracleQueries.getOracleType("BFILE")
|
||||||
|
|| sqlType == OraOopOracleQueries.getOracleType("INTERVALYM")
|
||||||
|
|| sqlType == OraOopOracleQueries.getOracleType("INTERVALDS")
|
||||||
|
|| sqlType == OraOopOracleQueries.getOracleType("NCLOB")
|
||||||
|
|| sqlType == OraOopOracleQueries.getOracleType("NCHAR")
|
||||||
|
|| sqlType == OraOopOracleQueries.getOracleType("NVARCHAR")
|
||||||
|
|| sqlType == OraOopOracleQueries.getOracleType("OTHER")
|
||||||
|
|| sqlType == OraOopOracleQueries.getOracleType("ROWID")
|
||||||
|
|| sqlType == OraOopOracleQueries.getOracleType("TIMESTAMPTZ")
|
||||||
|
|| sqlType == OraOopOracleQueries.getOracleType("TIMESTAMPLTZ")
|
||||||
|
|| sqlType == OraOopOracleQueries.getOracleType("STRUCT")) {
|
||||||
|
hiveType = "STRING";
|
||||||
|
}
|
||||||
|
|
||||||
|
if (sqlType == OraOopOracleQueries.getOracleType("BINARY_FLOAT")) {
|
||||||
|
hiveType = "FLOAT";
|
||||||
|
}
|
||||||
|
|
||||||
|
if (sqlType == OraOopOracleQueries.getOracleType("BINARY_DOUBLE")) {
|
||||||
|
hiveType = "DOUBLE";
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (hiveType == null) {
|
||||||
|
LOG.warn(String.format("%s should be updated to cater for data-type: %d",
|
||||||
|
OraOopUtilities.getCurrentMethodName(), sqlType));
|
||||||
|
}
|
||||||
|
|
||||||
|
return hiveType;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toJavaType(int sqlType) {
|
||||||
|
|
||||||
|
String javaType = super.toJavaType(sqlType);
|
||||||
|
|
||||||
|
if (sqlType == OraOopOracleQueries.getOracleType("TIMESTAMP")) {
|
||||||
|
// Get the Oracle JDBC driver to convert this value to a string
|
||||||
|
// instead of the generic JDBC driver.
|
||||||
|
// If the generic JDBC driver is used, it will take into account the
|
||||||
|
// timezone of the client machine's locale. The problem with this is that
|
||||||
|
// timestamp data should not be associated with a timezone. In practice,
|
||||||
|
// this
|
||||||
|
// leads to problems, for example, the time '2010-10-03 02:01:00' being
|
||||||
|
// changed to '2010-10-03 03:01:00' if the client machine's locale is
|
||||||
|
// Melbourne.
|
||||||
|
// (This is in response to daylight saving starting in Melbourne on
|
||||||
|
// this date at 2am.)
|
||||||
|
javaType = timestampJavaType;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (sqlType == OraOopOracleQueries.getOracleType("TIMESTAMPTZ")) {
|
||||||
|
// Returning "String" produces: "2010-08-08 09:00:00.0 +10:00"
|
||||||
|
// Returning "java.sql.Timestamp" produces: "2010-08-08 09:00:00.0"
|
||||||
|
|
||||||
|
// If we use "java.sql.Timestamp", the field's value will not
|
||||||
|
// contain the timezone when converted to a string and written to the HDFS
|
||||||
|
// CSV file.
|
||||||
|
// I.e. Get the Oracle JDBC driver to convert this value to a string
|
||||||
|
// instead of the generic JDBC driver...
|
||||||
|
javaType = timestampJavaType;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (sqlType == OraOopOracleQueries.getOracleType("TIMESTAMPLTZ")) {
|
||||||
|
// Returning "String" produces:
|
||||||
|
// "2010-08-08 09:00:00.0 Australia/Melbourne"
|
||||||
|
// Returning "java.sql.Timestamp" produces: "2010-08-08 09:00:00.0"
|
||||||
|
javaType = timestampJavaType;
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* http://www.oracle.com/technology/sample_code/tech/java/sqlj_jdbc/files
|
||||||
|
* /oracle10g/ieee/Readme.html
|
||||||
|
*
|
||||||
|
* BINARY_DOUBLE is a 64-bit, double-precision floating-point number
|
||||||
|
* datatype. (IEEE 754) Each BINARY_DOUBLE value requires 9 bytes, including
|
||||||
|
* a length byte. A 64-bit double format number X is divided as sign s 1-bit
|
||||||
|
* exponent e 11-bits fraction f 52-bits
|
||||||
|
*
|
||||||
|
* BINARY_FLOAT is a 32-bit, single-precision floating-point number
|
||||||
|
* datatype. (IEEE 754) Each BINARY_FLOAT value requires 5 bytes, including
|
||||||
|
* a length byte. A 32-bit single format number X is divided as sign s 1-bit
|
||||||
|
* exponent e 8-bits fraction f 23-bits
|
||||||
|
*/
|
||||||
|
if (sqlType == OraOopOracleQueries.getOracleType("BINARY_FLOAT")) {
|
||||||
|
// http://people.uncw.edu/tompkinsj/133/numbers/Reals.htm
|
||||||
|
javaType = "Float";
|
||||||
|
}
|
||||||
|
|
||||||
|
if (sqlType == OraOopOracleQueries.getOracleType("BINARY_DOUBLE")) {
|
||||||
|
// http://people.uncw.edu/tompkinsj/133/numbers/Reals.htm
|
||||||
|
javaType = "Double";
|
||||||
|
}
|
||||||
|
|
||||||
|
if (sqlType == OraOopOracleQueries.getOracleType("STRUCT")) {
|
||||||
|
// E.g. URITYPE
|
||||||
|
javaType = "String";
|
||||||
|
}
|
||||||
|
|
||||||
|
if (javaType == null) {
|
||||||
|
|
||||||
|
// For constant values, refer to:
|
||||||
|
// http://oracleadvisor.com/documentation/oracle/database/11.2/
|
||||||
|
// appdev.112/e13995/constant-values.html#oracle_jdbc
|
||||||
|
|
||||||
|
if (sqlType == OraOopOracleQueries.getOracleType("BFILE")
|
||||||
|
|| sqlType == OraOopOracleQueries.getOracleType("NCLOB")
|
||||||
|
|| sqlType == OraOopOracleQueries.getOracleType("NCHAR")
|
||||||
|
|| sqlType == OraOopOracleQueries.getOracleType("NVARCHAR")
|
||||||
|
|| sqlType == OraOopOracleQueries.getOracleType("ROWID")
|
||||||
|
|| sqlType == OraOopOracleQueries.getOracleType("INTERVALYM")
|
||||||
|
|| sqlType == OraOopOracleQueries.getOracleType("INTERVALDS")
|
||||||
|
|| sqlType == OraOopOracleQueries.getOracleType("OTHER")) {
|
||||||
|
javaType = "String";
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
if (javaType == null) {
|
||||||
|
LOG.warn(String.format("%s should be updated to cater for data-type: %d",
|
||||||
|
OraOopUtilities.getCurrentMethodName(), sqlType));
|
||||||
|
}
|
||||||
|
|
||||||
|
return javaType;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String timestampToQueryString(Timestamp ts) {
|
||||||
|
|
||||||
|
return "TO_TIMESTAMP('" + ts + "', 'YYYY-MM-DD HH24:MI:SS.FF')";
|
||||||
|
}
|
||||||
|
|
||||||
|
public OracleTable getOracleTableContext() {
|
||||||
|
|
||||||
|
return OraOopUtilities.decodeOracleTableName(this.options.getUsername(),
|
||||||
|
this.options.getTableName(), this.options.getConf());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, Integer> getColumnTypes(String tableName) {
|
||||||
|
|
||||||
|
if (this.columnTypesInOracleTable == null) {
|
||||||
|
|
||||||
|
Map<String, Integer> columnTypes = super.getColumnTypes(tableName);
|
||||||
|
this.columnTypesInOracleTable = new HashMap<String, Integer>();
|
||||||
|
|
||||||
|
List<String> colNames = getColumnNamesInOracleTable(tableName);
|
||||||
|
|
||||||
|
for (int idx = 0; idx < colNames.size(); idx++) {
|
||||||
|
|
||||||
|
String columnNameInTable = colNames.get(idx);
|
||||||
|
if (columnTypes.containsKey(columnNameInTable)) {
|
||||||
|
|
||||||
|
// Unescape the column names being returned...
|
||||||
|
int colType = columnTypes.get(columnNameInTable);
|
||||||
|
String key = unescapeOracleColumnName(columnNameInTable); // <- See
|
||||||
|
// notes at
|
||||||
|
// top about
|
||||||
|
// escaped
|
||||||
|
// column
|
||||||
|
// names
|
||||||
|
this.columnTypesInOracleTable.put(key, colType);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return this.columnTypesInOracleTable;
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean isEscaped(String name) {
|
||||||
|
|
||||||
|
return name.startsWith("\"") && name.endsWith("\"");
|
||||||
|
}
|
||||||
|
|
||||||
|
private String escapeOracleColumnName(String columnName) {
|
||||||
|
// See notes at top about escaped column names
|
||||||
|
if (isEscaped(columnName)) {
|
||||||
|
return columnName;
|
||||||
|
} else {
|
||||||
|
return "\"" + columnName + "\"";
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String escapeColName(String colName) {
|
||||||
|
|
||||||
|
return super.escapeColName(colName); // <- See notes at top about escaped
|
||||||
|
// column names
|
||||||
|
}
|
||||||
|
|
||||||
|
private String unescapeOracleColumnName(String columnName) {
|
||||||
|
|
||||||
|
if (isEscaped(columnName)) {
|
||||||
|
return columnName.substring(1, columnName.length() - 1);
|
||||||
|
} else {
|
||||||
|
return columnName;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void logImportTableDetails(ImportJobContext context) {
|
||||||
|
|
||||||
|
Path outputDirectory = context.getDestination();
|
||||||
|
if (outputDirectory != null) {
|
||||||
|
LOG.debug("The output directory for the sqoop table import is : "
|
||||||
|
+ outputDirectory.getName());
|
||||||
|
}
|
||||||
|
|
||||||
|
// Indicate whether we can load the class named: OraOopOraStats
|
||||||
|
showUserWhetherOraOopOraStatsIsAvailable(context.getOptions().getConf());
|
||||||
|
}
|
||||||
|
|
||||||
|
private void logExportTableDetails(ExportJobContext context) {
|
||||||
|
|
||||||
|
// Indicate whether we can load the class named: OraOopOraStats
|
||||||
|
showUserWhetherOraOopOraStatsIsAvailable(context.getOptions().getConf());
|
||||||
|
|
||||||
|
// Indicate what the update/merge columns are...
|
||||||
|
String[] updateKeyColumns =
|
||||||
|
OraOopUtilities.getExportUpdateKeyColumnNames(context.getOptions());
|
||||||
|
if (updateKeyColumns.length > 0) {
|
||||||
|
LOG.info(String.format(
|
||||||
|
"The column%s used to match rows in the HDFS file with rows in "
|
||||||
|
+ "the Oracle table %s: %s", updateKeyColumns.length > 1 ? "s"
|
||||||
|
: "", updateKeyColumns.length > 1 ? "are" : "is", OraOopUtilities
|
||||||
|
.stringArrayToCSV(updateKeyColumns)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void showUserWhetherOraOopOraStatsIsAvailable(Configuration conf) {
|
||||||
|
|
||||||
|
if (OraOopUtilities.userWantsOracleSessionStatisticsReports(conf)) {
|
||||||
|
|
||||||
|
LOG.info(String.format("%s=true",
|
||||||
|
OraOopConstants.ORAOOP_REPORT_SESSION_STATISTICS));
|
||||||
|
|
||||||
|
// This will log a warning if it's unable to load the OraOopOraStats
|
||||||
|
// class...
|
||||||
|
OraOopUtilities.startSessionSnapshot(null);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getCurTimestampQuery() {
|
||||||
|
|
||||||
|
return "SELECT SYSTIMESTAMP FROM DUAL";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void checkTableImportOptions(ImportJobContext context)
|
||||||
|
throws IOException, ImportException {
|
||||||
|
|
||||||
|
// Update the unit-test code if you modify this method.
|
||||||
|
super.checkTableImportOptions(context);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void explainWhyExportClassCannotBeLoaded(NoClassDefFoundError ex,
|
||||||
|
String exportClassName) {
|
||||||
|
|
||||||
|
String msg =
|
||||||
|
String.format("Unable to load class %s.\n"
|
||||||
|
+ "This is most likely caused by the Cloudera Shim Jar "
|
||||||
|
+ "not being included in the Java Classpath.\n" + "Either:\n"
|
||||||
|
+ "\tUse \"-libjars\" on the Sqoop command-line to "
|
||||||
|
+ "include the Cloudera shim jar in the Java Classpath; or"
|
||||||
|
+ "\n\tCopy the Cloudera shim jar into the Sqoop/lib "
|
||||||
|
+ "directory so that it is automatically included in the "
|
||||||
|
+ "Java Classpath; or\n"
|
||||||
|
+ "\tObtain an updated version of Sqoop that addresses "
|
||||||
|
+ "the Sqoop Jira \"SQOOP-127\".\n" + "\n"
|
||||||
|
+ "The Java Classpath is:\n%s", exportClassName, OraOopUtilities
|
||||||
|
.getJavaClassPath());
|
||||||
|
LOG.fatal(msg, ex);
|
||||||
|
}
|
||||||
|
}
|
512
src/java/org/apache/sqoop/manager/oracle/OraOopConstants.java
Normal file
512
src/java/org/apache/sqoop/manager/oracle/OraOopConstants.java
Normal file
@ -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
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
195
src/java/org/apache/sqoop/manager/oracle/OraOopDBInputSplit.java
Normal file
195
src/java/org/apache/sqoop/manager/oracle/OraOopDBInputSplit.java
Normal file
@ -0,0 +1,195 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBInputFormat;
|
||||||
|
|
||||||
|
class OraOopDBInputSplit extends DBInputFormat.DBInputSplit {
|
||||||
|
|
||||||
|
private int splitId;
|
||||||
|
private double totalNumberOfBlocksInAllSplits;
|
||||||
|
private String splitLocation;
|
||||||
|
private List<OraOopOracleDataChunk> oracleDataChunks;
|
||||||
|
|
||||||
|
// NB: Update write(), readFields() and getDebugDetails() if you add fields
|
||||||
|
// here.
|
||||||
|
|
||||||
|
public OraOopDBInputSplit() {
|
||||||
|
|
||||||
|
this.splitId = -1;
|
||||||
|
this.splitLocation = "";
|
||||||
|
this.oracleDataChunks = new ArrayList<OraOopOracleDataChunk>();
|
||||||
|
}
|
||||||
|
|
||||||
|
public OraOopDBInputSplit(List<OraOopOracleDataChunk> dataChunks) {
|
||||||
|
|
||||||
|
setOracleDataChunks(dataChunks);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setOracleDataChunks(List<OraOopOracleDataChunk> dataChunks) {
|
||||||
|
|
||||||
|
this.oracleDataChunks = dataChunks;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<OraOopOracleDataChunk> getDataChunks() {
|
||||||
|
|
||||||
|
return this.oracleDataChunks;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getNumberOfDataChunks() {
|
||||||
|
|
||||||
|
if (this.getDataChunks() == null) {
|
||||||
|
return 0;
|
||||||
|
} else {
|
||||||
|
return this.getDataChunks().size();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String[] getLocations() throws IOException {
|
||||||
|
|
||||||
|
if (this.splitLocation.isEmpty()) {
|
||||||
|
return new String[] {};
|
||||||
|
} else {
|
||||||
|
return new String[] { this.splitLocation };
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The total number of blocks within the data-chunks of this split
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public long getLength() {
|
||||||
|
|
||||||
|
return this.getTotalNumberOfBlocksInThisSplit();
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getTotalNumberOfBlocksInThisSplit() {
|
||||||
|
|
||||||
|
if (this.getNumberOfDataChunks() == 0) {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
int result = 0;
|
||||||
|
for (OraOopOracleDataChunk dataChunk : this.getDataChunks()) {
|
||||||
|
result += dataChunk.getNumberOfBlocks();
|
||||||
|
}
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OraOopOracleDataChunk findDataChunkById(String id) {
|
||||||
|
|
||||||
|
for (OraOopOracleDataChunk dataChunk : this.getDataChunks()) {
|
||||||
|
if (dataChunk.getId().equals(id)) {
|
||||||
|
return dataChunk;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public void write(DataOutput output) throws IOException {
|
||||||
|
|
||||||
|
output.writeInt(splitId);
|
||||||
|
|
||||||
|
if (this.oracleDataChunks == null) {
|
||||||
|
output.writeInt(0);
|
||||||
|
} else {
|
||||||
|
output.writeInt(this.oracleDataChunks.size());
|
||||||
|
for (OraOopOracleDataChunk dataChunk : this.oracleDataChunks) {
|
||||||
|
Text.writeString(output, dataChunk.getClass().getName());
|
||||||
|
dataChunk.write(output);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public void readFields(DataInput input) throws IOException {
|
||||||
|
|
||||||
|
this.splitId = input.readInt();
|
||||||
|
|
||||||
|
int dataChunkCount = input.readInt();
|
||||||
|
if (dataChunkCount == 0) {
|
||||||
|
this.oracleDataChunks = null;
|
||||||
|
} else {
|
||||||
|
Class<? extends OraOopOracleDataChunk> dataChunkClass;
|
||||||
|
OraOopOracleDataChunk dataChunk;
|
||||||
|
this.oracleDataChunks =
|
||||||
|
new ArrayList<OraOopOracleDataChunk>(dataChunkCount);
|
||||||
|
for (int idx = 0; idx < dataChunkCount; idx++) {
|
||||||
|
try {
|
||||||
|
dataChunkClass =
|
||||||
|
(Class<? extends OraOopOracleDataChunk>) Class.forName(Text
|
||||||
|
.readString(input));
|
||||||
|
dataChunk = dataChunkClass.newInstance();
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
dataChunk.readFields(input);
|
||||||
|
this.oracleDataChunks.add(dataChunk);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getDebugDetails() {
|
||||||
|
|
||||||
|
StringBuilder result = new StringBuilder();
|
||||||
|
|
||||||
|
if (this.getNumberOfDataChunks() == 0) {
|
||||||
|
result.append(String.format(
|
||||||
|
"Split[%s] does not contain any Oracle data-chunks.", this.splitId));
|
||||||
|
} else {
|
||||||
|
result.append(String.format(
|
||||||
|
"Split[%s] includes the Oracle data-chunks:\n", this.splitId));
|
||||||
|
for (OraOopOracleDataChunk dataChunk : getDataChunks()) {
|
||||||
|
result.append(dataChunk.toString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return result.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected int getSplitId() {
|
||||||
|
return this.splitId;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void setSplitId(int newSplitId) {
|
||||||
|
this.splitId = newSplitId;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void setSplitLocation(String newSplitLocation) {
|
||||||
|
this.splitLocation = newSplitLocation;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void setTotalNumberOfBlocksInAllSplits(
|
||||||
|
int newTotalNumberOfBlocksInAllSplits) {
|
||||||
|
this.totalNumberOfBlocksInAllSplits = newTotalNumberOfBlocksInAllSplits;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,468 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.Connection;
|
||||||
|
import java.sql.ResultSet;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBInputFormat;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBInputFormat.DBInputSplit;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DataDrivenDBRecordReader;
|
||||||
|
import org.apache.sqoop.manager.oracle.OraOopConstants.
|
||||||
|
OraOopTableImportWhereClauseLocation;
|
||||||
|
import org.apache.sqoop.manager.oracle.OraOopUtilities.OraOopStatsReports;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* NOTES:
|
||||||
|
*
|
||||||
|
* T is the output-type of this record reader.
|
||||||
|
*
|
||||||
|
* getFieldNames() is overridden to insert an "data_chunk_id" column
|
||||||
|
* containing the id (integer) of the Oracle data-chunk the data
|
||||||
|
* was obtained from. This is used to calculate the "percentage complete"
|
||||||
|
* for this mapper.
|
||||||
|
*
|
||||||
|
* getSelectQuery() is overridden to inject the actual data_chunk_id number
|
||||||
|
* into the query that is executed (for the data-chunk being processed).
|
||||||
|
*
|
||||||
|
* This class extends DBRecordReader. Unfortunately, DBRecordReader does
|
||||||
|
* not expose its results property (of type ResultSet), so we have to
|
||||||
|
* override executeQuery() in order to obtain a reference to the data
|
||||||
|
* obtained when the SQL generated by getSelectQuery() is executed.
|
||||||
|
*/
|
||||||
|
class OraOopDBRecordReader<T extends SqoopRecord> extends
|
||||||
|
DataDrivenDBRecordReader<T> {
|
||||||
|
|
||||||
|
private static final OraOopLog LOG = OraOopLogFactory
|
||||||
|
.getLog(OraOopDBRecordReader.class);
|
||||||
|
|
||||||
|
private OraOopDBInputSplit dbInputSplit; // <- The split this record-reader is
|
||||||
|
// working on.
|
||||||
|
private int numberOfBlocksInThisSplit; // <- The number of Oracle blocks in
|
||||||
|
// this Oracle data-chunk.
|
||||||
|
private int numberOfBlocksProcessedInThisSplit; // <- How many Oracle blocks
|
||||||
|
// we've processed with this
|
||||||
|
// record-reader.
|
||||||
|
private String currentDataChunkId; // <- The id of the current data-chunk
|
||||||
|
// being processed
|
||||||
|
private ResultSet results; // <- The ResultSet containing the data from the
|
||||||
|
// query returned by getSelectQuery()
|
||||||
|
private int columnIndexDataChunkIdZeroBased = -1; // <- The zero-based column
|
||||||
|
// index of the
|
||||||
|
// data_chunk_id column.
|
||||||
|
private boolean progressCalculationErrorLogged; // <- Whether we've logged a
|
||||||
|
// problem with the progress
|
||||||
|
// calculation during
|
||||||
|
// nextKeyValue().
|
||||||
|
private Object oraOopOraStats; // <- A reference to the Oracle statistics
|
||||||
|
// object that is being tracked for this Oracle
|
||||||
|
// session.
|
||||||
|
private boolean profilingEnabled; // <- Whether to collect profiling metrics
|
||||||
|
private long timeSpentInNextKeyValueInNanoSeconds; // <- Total time spent in
|
||||||
|
// super.nextKeyValue()
|
||||||
|
|
||||||
|
public OraOopDBRecordReader(DBInputFormat.DBInputSplit split,
|
||||||
|
Class<T> inputClass, Configuration conf, Connection conn,
|
||||||
|
DBConfiguration dbConfig, String cond, String[] fields, String table)
|
||||||
|
throws SQLException {
|
||||||
|
|
||||||
|
super(split, inputClass, conf, conn, dbConfig, cond, fields, table,
|
||||||
|
"ORACLE-ORAOOP");
|
||||||
|
|
||||||
|
OraOopUtilities.enableDebugLoggingIfRequired(conf);
|
||||||
|
|
||||||
|
this.dbInputSplit = castSplit(split);
|
||||||
|
|
||||||
|
String thisOracleInstanceName =
|
||||||
|
OraOopOracleQueries.getCurrentOracleInstanceName(conn);
|
||||||
|
LOG.info(String.format(
|
||||||
|
"This record reader is connected to Oracle via the JDBC URL: \n"
|
||||||
|
+ "\t\"%s\"\n" + "\tto the Oracle instance: \"%s\"", conn
|
||||||
|
.toString(), thisOracleInstanceName));
|
||||||
|
|
||||||
|
OracleConnectionFactory.initializeOracleConnection(conn, conf);
|
||||||
|
|
||||||
|
if (OraOopUtilities.userWantsOracleSessionStatisticsReports(conf)) {
|
||||||
|
this.oraOopOraStats = OraOopUtilities.startSessionSnapshot(conn);
|
||||||
|
}
|
||||||
|
|
||||||
|
this.numberOfBlocksInThisSplit =
|
||||||
|
this.dbInputSplit.getTotalNumberOfBlocksInThisSplit();
|
||||||
|
this.numberOfBlocksProcessedInThisSplit = 0;
|
||||||
|
|
||||||
|
this.profilingEnabled = conf.getBoolean("oraoop.profiling.enabled", false);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static OraOopDBInputSplit castSplit(DBInputSplit split) {
|
||||||
|
|
||||||
|
// Check there's a split available...
|
||||||
|
if (split == null) {
|
||||||
|
throw new IllegalArgumentException("The DBInputSplit cannot be null.");
|
||||||
|
}
|
||||||
|
|
||||||
|
// Check that the split is the correct type...
|
||||||
|
Class<?> desiredSplitClass = OraOopDBInputSplit.class;
|
||||||
|
if (!(split.getClass() == desiredSplitClass)) {
|
||||||
|
String errMsg =
|
||||||
|
String.format("The type of Split available within %s "
|
||||||
|
+ "should be an instance of class %s, "
|
||||||
|
+ "but is actually an instance of class %s", OraOopUtilities
|
||||||
|
.getCurrentMethodName(), desiredSplitClass.getName(), split
|
||||||
|
.getClass().getName());
|
||||||
|
throw new RuntimeException(errMsg);
|
||||||
|
}
|
||||||
|
|
||||||
|
// TODO Cast this using desiredSplitClass, so we only need 1 line of code
|
||||||
|
// that
|
||||||
|
// identifies the type of the split class...
|
||||||
|
// inputSplit = (desiredSplitClass)this.getSplit();
|
||||||
|
return (OraOopDBInputSplit) split;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String[] getFieldNames() {
|
||||||
|
|
||||||
|
String[] fieldNames = super.getFieldNames();
|
||||||
|
ArrayList<String> result = new ArrayList<String>();
|
||||||
|
|
||||||
|
for (int idx = 0; idx < fieldNames.length; idx++) {
|
||||||
|
result.add(fieldNames[idx]);
|
||||||
|
}
|
||||||
|
|
||||||
|
result.add(OraOopConstants.COLUMN_NAME_DATA_CHUNK_ID);
|
||||||
|
this.columnIndexDataChunkIdZeroBased = result.size() - 1;
|
||||||
|
|
||||||
|
return result.toArray(new String[result.size()]);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getSelectQuery() {
|
||||||
|
|
||||||
|
boolean consistentRead =
|
||||||
|
this.getDBConf().getConf().getBoolean(
|
||||||
|
OraOopConstants.ORAOOP_IMPORT_CONSISTENT_READ, false);
|
||||||
|
long consistentReadScn =
|
||||||
|
this.getDBConf().getConf().getLong(
|
||||||
|
OraOopConstants.ORAOOP_IMPORT_CONSISTENT_READ_SCN, 0L);
|
||||||
|
if (consistentRead && consistentReadScn == 0L) {
|
||||||
|
throw new RuntimeException("Could not get SCN for consistent read.");
|
||||||
|
}
|
||||||
|
|
||||||
|
StringBuilder query = new StringBuilder();
|
||||||
|
|
||||||
|
if (this.dbInputSplit.getDataChunks() == null) {
|
||||||
|
String errMsg =
|
||||||
|
String.format("The %s does not contain any data-chunks, within %s.",
|
||||||
|
this.dbInputSplit.getClass().getName(), OraOopUtilities
|
||||||
|
.getCurrentMethodName());
|
||||||
|
throw new RuntimeException(errMsg);
|
||||||
|
}
|
||||||
|
|
||||||
|
OraOopConstants.OraOopTableImportWhereClauseLocation whereClauseLocation =
|
||||||
|
OraOopUtilities.getOraOopTableImportWhereClauseLocation(this
|
||||||
|
.getDBConf().getConf(),
|
||||||
|
OraOopConstants.OraOopTableImportWhereClauseLocation.SUBSPLIT);
|
||||||
|
|
||||||
|
OracleTable tableContext = getOracleTableContext();
|
||||||
|
OracleTableColumns tableColumns = null;
|
||||||
|
try {
|
||||||
|
|
||||||
|
Configuration conf = this.getDBConf().getConf();
|
||||||
|
|
||||||
|
tableColumns =
|
||||||
|
OraOopOracleQueries.getTableColumns(getConnection(), tableContext,
|
||||||
|
OraOopUtilities.omitLobAndLongColumnsDuringImport(conf),
|
||||||
|
OraOopUtilities.recallSqoopJobType(conf)
|
||||||
|
, true // <- onlyOraOopSupportedTypes
|
||||||
|
, true // <- omitOraOopPseudoColumns
|
||||||
|
);
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
LOG.error(String.format(
|
||||||
|
"Unable to obtain the data-types of the columns in table %s.\n"
|
||||||
|
+ "Error:\n%s", tableContext.toString(), ex.getMessage()));
|
||||||
|
throw new RuntimeException(ex);
|
||||||
|
}
|
||||||
|
|
||||||
|
int numberOfDataChunks = this.dbInputSplit.getNumberOfDataChunks();
|
||||||
|
for (int idx = 0; idx < numberOfDataChunks; idx++) {
|
||||||
|
|
||||||
|
OraOopOracleDataChunk dataChunk =
|
||||||
|
this.dbInputSplit.getDataChunks().get(idx);
|
||||||
|
|
||||||
|
if (idx > 0) {
|
||||||
|
query.append("UNION ALL \n");
|
||||||
|
}
|
||||||
|
|
||||||
|
query.append(getColumnNamesClause(tableColumns, dataChunk.getId())) // <-
|
||||||
|
// SELECT
|
||||||
|
// clause
|
||||||
|
.append("\n");
|
||||||
|
|
||||||
|
query.append(" FROM ").append(this.getTableName()).append(" ");
|
||||||
|
|
||||||
|
if (consistentRead) {
|
||||||
|
query.append("AS OF SCN ").append(consistentReadScn).append(" ");
|
||||||
|
}
|
||||||
|
|
||||||
|
query.append(getPartitionClauseForDataChunk(this.dbInputSplit, idx))
|
||||||
|
.append(" t").append("\n");
|
||||||
|
|
||||||
|
query.append(" WHERE (").append(
|
||||||
|
getWhereClauseForDataChunk(this.dbInputSplit, idx)).append(")\n");
|
||||||
|
|
||||||
|
// If the user wants the WHERE clause applied to each data-chunk...
|
||||||
|
if (whereClauseLocation
|
||||||
|
== OraOopTableImportWhereClauseLocation.SUBSPLIT) {
|
||||||
|
String conditions = this.getConditions();
|
||||||
|
if (conditions != null && conditions.length() > 0) {
|
||||||
|
query.append(" AND (").append(conditions).append(")\n");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
// If the user wants the WHERE clause applied to the whole split...
|
||||||
|
if (whereClauseLocation == OraOopTableImportWhereClauseLocation.SPLIT) {
|
||||||
|
String conditions = this.getConditions();
|
||||||
|
if (conditions != null && conditions.length() > 0) {
|
||||||
|
|
||||||
|
// Insert a "select everything" line at the start of the SQL query...
|
||||||
|
query.insert(0, getColumnNamesClause(tableColumns, null) + " FROM (\n");
|
||||||
|
|
||||||
|
// ...and then apply the WHERE clause to all the UNIONed sub-queries...
|
||||||
|
query.append(")\n").append("WHERE\n").append(conditions).append("\n");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
LOG.info("SELECT QUERY = \n" + query.toString());
|
||||||
|
|
||||||
|
return query.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
private String getColumnNamesClause(OracleTableColumns tableColumns,
|
||||||
|
String dataChunkId) {
|
||||||
|
|
||||||
|
StringBuilder result = new StringBuilder();
|
||||||
|
|
||||||
|
result.append("SELECT ");
|
||||||
|
result.append(OraOopUtilities.getImportHint(this.getDBConf().getConf()));
|
||||||
|
|
||||||
|
String[] fieldNames = this.getFieldNames();
|
||||||
|
|
||||||
|
int firstFieldIndex = 0;
|
||||||
|
int lastFieldIndex = fieldNames.length - 1;
|
||||||
|
for (int i = firstFieldIndex; i <= lastFieldIndex; i++) {
|
||||||
|
if (i > firstFieldIndex) {
|
||||||
|
result.append(",");
|
||||||
|
}
|
||||||
|
String fieldName = fieldNames[i];
|
||||||
|
|
||||||
|
OracleTableColumn oracleTableColumn =
|
||||||
|
tableColumns.findColumnByName(fieldName);
|
||||||
|
if (oracleTableColumn != null) {
|
||||||
|
if (oracleTableColumn.getDataType().equals(
|
||||||
|
OraOopConstants.Oracle.URITYPE)) {
|
||||||
|
fieldName = String.format("uritype.geturl(%s)", fieldName);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// If this field is the "data_chunk_id" that we inserted during
|
||||||
|
// getFields()
|
||||||
|
// then we need to insert the value of that data_chunk_id now...
|
||||||
|
if (i == this.columnIndexDataChunkIdZeroBased
|
||||||
|
&& fieldName == OraOopConstants.COLUMN_NAME_DATA_CHUNK_ID) {
|
||||||
|
if (dataChunkId != null && !dataChunkId.isEmpty()) {
|
||||||
|
fieldName =
|
||||||
|
String.format("'%s' %s", dataChunkId,
|
||||||
|
OraOopConstants.COLUMN_NAME_DATA_CHUNK_ID);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
result.append(fieldName);
|
||||||
|
}
|
||||||
|
return result.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
private String getPartitionClauseForDataChunk(OraOopDBInputSplit split,
|
||||||
|
int dataChunkIndex) {
|
||||||
|
OraOopOracleDataChunk dataChunk = split.getDataChunks().get(dataChunkIndex);
|
||||||
|
return dataChunk.getPartitionClause();
|
||||||
|
}
|
||||||
|
|
||||||
|
private String getWhereClauseForDataChunk(OraOopDBInputSplit split,
|
||||||
|
int dataChunkIndex) {
|
||||||
|
|
||||||
|
OraOopOracleDataChunk dataChunk = split.getDataChunks().get(dataChunkIndex);
|
||||||
|
return dataChunk.getWhereClause();
|
||||||
|
}
|
||||||
|
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
@Override
|
||||||
|
public long getPos() throws IOException {
|
||||||
|
|
||||||
|
// This split contains multiple data-chunks.
|
||||||
|
// Each data-chunk contains multiple blocks.
|
||||||
|
// Return the number of blocks that have been processed by this split...
|
||||||
|
return numberOfBlocksProcessedInThisSplit;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
@Override
|
||||||
|
public float getProgress() throws IOException {
|
||||||
|
|
||||||
|
return numberOfBlocksProcessedInThisSplit
|
||||||
|
/ (float) numberOfBlocksInThisSplit;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean nextKeyValue() throws IOException {
|
||||||
|
|
||||||
|
boolean result = false;
|
||||||
|
try {
|
||||||
|
|
||||||
|
long startTime = 0;
|
||||||
|
if (this.profilingEnabled) {
|
||||||
|
startTime = System.nanoTime();
|
||||||
|
}
|
||||||
|
|
||||||
|
result = super.nextKeyValue();
|
||||||
|
|
||||||
|
if (this.profilingEnabled) {
|
||||||
|
this.timeSpentInNextKeyValueInNanoSeconds +=
|
||||||
|
System.nanoTime() - startTime;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Keep track of which data-chunk we're processing, and therefore how many
|
||||||
|
// Oracle blocks we've processed. This can be used to calculate our
|
||||||
|
// "percentage complete"...
|
||||||
|
if (result && this.results != null) {
|
||||||
|
|
||||||
|
String thisDataChunkId = null;
|
||||||
|
try {
|
||||||
|
// ColumnIndexes are 1-based in jdbc...
|
||||||
|
thisDataChunkId =
|
||||||
|
this.results.getString(this.columnIndexDataChunkIdZeroBased + 1);
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
if (!progressCalculationErrorLogged) {
|
||||||
|
// This prevents us from flooding the log with the same message
|
||||||
|
// thousands of times...
|
||||||
|
progressCalculationErrorLogged = true;
|
||||||
|
|
||||||
|
LOG.warn(String
|
||||||
|
.format(
|
||||||
|
"Unable to obtain the value of the %s column in method %s.\n"
|
||||||
|
+ "\tthis.columnIndexDataChunkIdZeroBased = %d (NB: "
|
||||||
|
+ "jdbc field indexes are 1-based)\n\tAs a consequence, "
|
||||||
|
+ "progress for the record-reader cannot be calculated.\n"
|
||||||
|
+ "\tError=\n%s",
|
||||||
|
OraOopConstants.COLUMN_NAME_DATA_CHUNK_ID, OraOopUtilities
|
||||||
|
.getCurrentMethodName(),
|
||||||
|
this.columnIndexDataChunkIdZeroBased, ex.getMessage()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (thisDataChunkId != null
|
||||||
|
&& !thisDataChunkId.equals(this.currentDataChunkId)) {
|
||||||
|
if (this.currentDataChunkId != null
|
||||||
|
&& !this.currentDataChunkId.isEmpty()) {
|
||||||
|
OraOopOracleDataChunk dataChunk =
|
||||||
|
this.dbInputSplit.findDataChunkById(thisDataChunkId);
|
||||||
|
if (dataChunk != null) {
|
||||||
|
this.numberOfBlocksProcessedInThisSplit +=
|
||||||
|
dataChunk.getNumberOfBlocks();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
this.currentDataChunkId = thisDataChunkId;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch (IOException ex) {
|
||||||
|
if (OraOopUtilities.oracleSessionHasBeenKilled(ex)) {
|
||||||
|
LOG.info("\n*********************************************************"
|
||||||
|
+ "\nThe Oracle session in use has been killed by a 3rd party."
|
||||||
|
+ "\n*********************************************************");
|
||||||
|
}
|
||||||
|
throw ex;
|
||||||
|
}
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected ResultSet executeQuery(String query) throws SQLException {
|
||||||
|
|
||||||
|
try {
|
||||||
|
this.results = super.executeQuery(query);
|
||||||
|
return this.results;
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
LOG.error(String.format("Error in %s while executing the SQL query:\n"
|
||||||
|
+ "%s\n\n" + "%s", OraOopUtilities.getCurrentMethodName(), query, ex
|
||||||
|
.getMessage()));
|
||||||
|
throw ex;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException {
|
||||||
|
|
||||||
|
if (this.profilingEnabled) {
|
||||||
|
LOG.info(String.format(
|
||||||
|
"Time spent in super.nextKeyValue() = %s seconds.",
|
||||||
|
this.timeSpentInNextKeyValueInNanoSeconds / Math.pow(10, 9)));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (OraOopUtilities.userWantsOracleSessionStatisticsReports(getDBConf()
|
||||||
|
.getConf())) {
|
||||||
|
OraOopStatsReports reports =
|
||||||
|
OraOopUtilities.stopSessionSnapshot(this.oraOopOraStats);
|
||||||
|
this.oraOopOraStats = null;
|
||||||
|
|
||||||
|
LOG.info(String.format("Oracle Statistics Report for OraOop:\n\n%s",
|
||||||
|
reports.getPerformanceReport()));
|
||||||
|
|
||||||
|
String fileName =
|
||||||
|
String.format("oracle-stats-csv-%d", this.dbInputSplit.getSplitId());
|
||||||
|
OraOopUtilities.writeOutputFile(this.getDBConf().getConf(), fileName,
|
||||||
|
reports.getCsvReport());
|
||||||
|
|
||||||
|
fileName =
|
||||||
|
String.format("oracle-stats-%d", this.dbInputSplit.getSplitId());
|
||||||
|
OraOopUtilities.writeOutputFile(this.getDBConf().getConf(), fileName,
|
||||||
|
reports.getPerformanceReport());
|
||||||
|
}
|
||||||
|
|
||||||
|
super.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
public OracleTable getOracleTableContext() {
|
||||||
|
|
||||||
|
Configuration conf = this.getDBConf().getConf();
|
||||||
|
OracleTable result =
|
||||||
|
new OracleTable(conf.get(OraOopConstants.ORAOOP_TABLE_OWNER), conf
|
||||||
|
.get(OraOopConstants.ORAOOP_TABLE_NAME));
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,359 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.Connection;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configurable;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.io.LongWritable;
|
||||||
|
import org.apache.hadoop.mapreduce.InputSplit;
|
||||||
|
import org.apache.hadoop.mapreduce.JobContext;
|
||||||
|
import org.apache.hadoop.mapreduce.RecordReader;
|
||||||
|
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DataDrivenDBInputFormat;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Reads data from Oracle table - data is divided between mappers based on ROWID
|
||||||
|
* split.
|
||||||
|
*
|
||||||
|
* @param <T> Output type of the record reader
|
||||||
|
*/
|
||||||
|
public class OraOopDataDrivenDBInputFormat<T extends SqoopRecord> extends
|
||||||
|
DataDrivenDBInputFormat<T> implements Configurable {
|
||||||
|
|
||||||
|
public static final OraOopLog LOG = OraOopLogFactory
|
||||||
|
.getLog(OraOopDataDrivenDBInputFormat.class.getName());
|
||||||
|
|
||||||
|
public OraOopDataDrivenDBInputFormat() {
|
||||||
|
super();
|
||||||
|
OraOopUtilities.checkJavaSecurityEgd();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<InputSplit> getSplits(JobContext jobContext) throws IOException {
|
||||||
|
|
||||||
|
int desiredNumberOfMappers = getDesiredNumberOfMappers(jobContext);
|
||||||
|
|
||||||
|
// Resolve the Oracle owner and name of the table we're importing...
|
||||||
|
OracleTable table = identifyOracleTableFromJobContext(jobContext);
|
||||||
|
List<String> partitionList = getPartitionList(jobContext);
|
||||||
|
|
||||||
|
// Get our Oracle connection...
|
||||||
|
Connection connection = getConnection();
|
||||||
|
|
||||||
|
List<InputSplit> splits = null;
|
||||||
|
try {
|
||||||
|
OracleConnectionFactory.initializeOracleConnection(connection, getConf());
|
||||||
|
|
||||||
|
// The number of chunks generated will *not* be a multiple of the number
|
||||||
|
// of splits,
|
||||||
|
// to ensure that each split doesn't always get data from the start of
|
||||||
|
// each data-file...
|
||||||
|
int numberOfChunksPerOracleDataFile = (desiredNumberOfMappers * 2) + 1;
|
||||||
|
|
||||||
|
// Get the Oracle data-chunks for the table...
|
||||||
|
List<? extends OraOopOracleDataChunk> dataChunks;
|
||||||
|
if (OraOopUtilities.getOraOopOracleDataChunkMethod(getConf()).equals(
|
||||||
|
OraOopConstants.OraOopOracleDataChunkMethod.PARTITION)) {
|
||||||
|
dataChunks =
|
||||||
|
OraOopOracleQueries.getOracleDataChunksPartition(connection, table,
|
||||||
|
partitionList);
|
||||||
|
} else {
|
||||||
|
dataChunks =
|
||||||
|
OraOopOracleQueries.getOracleDataChunksExtent(jobContext
|
||||||
|
.getConfiguration(), connection, table, partitionList,
|
||||||
|
numberOfChunksPerOracleDataFile);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (dataChunks.size() == 0) {
|
||||||
|
String errMsg;
|
||||||
|
if (OraOopUtilities.getOraOopOracleDataChunkMethod(getConf()).equals(
|
||||||
|
OraOopConstants.OraOopOracleDataChunkMethod.PARTITION)) {
|
||||||
|
errMsg =
|
||||||
|
String
|
||||||
|
.format(
|
||||||
|
"The table %s does not contain any partitions and you "
|
||||||
|
+ "have specified to chunk the table by partitions.",
|
||||||
|
table.getName());
|
||||||
|
} else {
|
||||||
|
errMsg =
|
||||||
|
String.format("The table %s does not contain any data.", table
|
||||||
|
.getName());
|
||||||
|
}
|
||||||
|
LOG.fatal(errMsg);
|
||||||
|
throw new RuntimeException(errMsg);
|
||||||
|
} else {
|
||||||
|
OraOopConstants.OraOopOracleBlockToSplitAllocationMethod
|
||||||
|
blockAllocationMethod = OraOopUtilities
|
||||||
|
.getOraOopOracleBlockToSplitAllocationMethod(
|
||||||
|
jobContext.getConfiguration(),
|
||||||
|
OraOopConstants.
|
||||||
|
OraOopOracleBlockToSplitAllocationMethod.ROUNDROBIN);
|
||||||
|
|
||||||
|
// Group the Oracle data-chunks into splits...
|
||||||
|
splits =
|
||||||
|
groupTableDataChunksIntoSplits(dataChunks, desiredNumberOfMappers,
|
||||||
|
blockAllocationMethod);
|
||||||
|
|
||||||
|
String oraoopLocations =
|
||||||
|
jobContext.getConfiguration().get("oraoop.locations", "");
|
||||||
|
String[] locations = oraoopLocations.split(",");
|
||||||
|
for (int idx = 0; idx < locations.length; idx++) {
|
||||||
|
if (idx < splits.size()) {
|
||||||
|
String location = locations[idx].trim();
|
||||||
|
if (!location.isEmpty()) {
|
||||||
|
((OraOopDBInputSplit) splits.get(idx)).setSplitLocation(location);
|
||||||
|
|
||||||
|
LOG.info(String
|
||||||
|
.format("Split[%d] has been assigned location \"%s\".", idx,
|
||||||
|
location));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
throw new IOException(ex);
|
||||||
|
}
|
||||||
|
|
||||||
|
return splits;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected RecordReader<LongWritable, T> createDBRecordReader(
|
||||||
|
DBInputSplit split, Configuration conf) throws IOException {
|
||||||
|
|
||||||
|
// This code is now running on a Datanode in the Hadoop cluster, so we need
|
||||||
|
// to enable debug logging in this JVM...
|
||||||
|
OraOopUtilities.enableDebugLoggingIfRequired(conf);
|
||||||
|
|
||||||
|
// Retrieve the JDBC URL that should be used by this mapper.
|
||||||
|
// We achieve this by modifying the JDBC URL property in the configuration,
|
||||||
|
// prior to the
|
||||||
|
// OraOopDBRecordReader (or its ancestors) using the configuration to
|
||||||
|
// establish a connection
|
||||||
|
// to the database - via DBConfiguration.getConnection()...
|
||||||
|
OraOopDBInputSplit oraOopSplit = OraOopDBRecordReader.castSplit(split);
|
||||||
|
int mapperId = oraOopSplit.getSplitId();
|
||||||
|
String mapperJdbcUrlPropertyName =
|
||||||
|
OraOopUtilities.getMapperJdbcUrlPropertyName(mapperId, conf);
|
||||||
|
|
||||||
|
// Get this mapper's JDBC URL
|
||||||
|
String mapperJdbcUrl = conf.get(mapperJdbcUrlPropertyName, null);
|
||||||
|
LOG.debug(String.format("Mapper %d has a JDBC URL of: %s", mapperId,
|
||||||
|
mapperJdbcUrl == null ? "<null>" : mapperJdbcUrl));
|
||||||
|
|
||||||
|
DBConfiguration dbConf = getDBConf();
|
||||||
|
|
||||||
|
if (mapperJdbcUrl != null) {
|
||||||
|
// Just changing the URL_PROPERTY in the conf object does not work - as
|
||||||
|
// dbConf.getConf()
|
||||||
|
// seems to refer to a separate instance of the configuration properties.
|
||||||
|
// Therefore, we
|
||||||
|
// need to update the URL_PROPERTY in dbConf so that we connect to the
|
||||||
|
// appropriate instance
|
||||||
|
// in the Oracle RAC. To help avoid confusion, we'll also update the
|
||||||
|
// URL_PROPERTY in the
|
||||||
|
// conf object to match...
|
||||||
|
dbConf.getConf().set(DBConfiguration.URL_PROPERTY, mapperJdbcUrl);
|
||||||
|
conf.set(DBConfiguration.URL_PROPERTY, mapperJdbcUrl);
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
Class<T> inputClass = (Class<T>) (dbConf.getInputClass());
|
||||||
|
|
||||||
|
try {
|
||||||
|
// Use Oracle-specific db reader
|
||||||
|
|
||||||
|
// this.getConnection() will return the connection created when the
|
||||||
|
// DBInputFormat ancestor
|
||||||
|
// was created. This connection will be based on the URL_PROPERTY that was
|
||||||
|
// current at that
|
||||||
|
// time. We've just changed the URL_PROPERTY (if this is an Oracle RAC)
|
||||||
|
// and therefore need
|
||||||
|
// to use dbConf.getConnection() so that a new connection is created using
|
||||||
|
// the current
|
||||||
|
// value of the URL_PROPERTY...
|
||||||
|
|
||||||
|
return new OraOopDBRecordReader<T>(split, inputClass, conf, dbConf
|
||||||
|
.getConnection(), dbConf, dbConf.getInputConditions(), dbConf
|
||||||
|
.getInputFieldNames(), dbConf.getInputTableName());
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
throw new IOException(ex);
|
||||||
|
} catch (ClassNotFoundException ex) {
|
||||||
|
throw new IOException(ex);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private OracleTable identifyOracleTableFromJobContext(JobContext jobContext) {
|
||||||
|
|
||||||
|
OracleTable result = new OracleTable();
|
||||||
|
|
||||||
|
String dbUserName =
|
||||||
|
jobContext.getConfiguration().get(DBConfiguration.USERNAME_PROPERTY);
|
||||||
|
String tableName = getDBConf().getInputTableName();
|
||||||
|
|
||||||
|
result =
|
||||||
|
OraOopUtilities.decodeOracleTableName(dbUserName, tableName, jobContext
|
||||||
|
.getConfiguration());
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
private int getDesiredNumberOfMappers(JobContext jobContext) {
|
||||||
|
|
||||||
|
int desiredNumberOfMappers =
|
||||||
|
jobContext.getConfiguration().getInt(
|
||||||
|
OraOopConstants.ORAOOP_DESIRED_NUMBER_OF_MAPPERS, -1);
|
||||||
|
|
||||||
|
int minMappersAcceptedByOraOop =
|
||||||
|
OraOopUtilities.getMinNumberOfImportMappersAcceptedByOraOop(jobContext
|
||||||
|
.getConfiguration());
|
||||||
|
|
||||||
|
if (desiredNumberOfMappers < minMappersAcceptedByOraOop) {
|
||||||
|
LOG.warn(String.format("%s should not be used to perform a sqoop import "
|
||||||
|
+ "when the number of mappers is %d\n "
|
||||||
|
+ "i.e. OraOopManagerFactory.accept() should only appect jobs "
|
||||||
|
+ "where the number of mappers is at least %d",
|
||||||
|
OraOopConstants.ORAOOP_PRODUCT_NAME, desiredNumberOfMappers,
|
||||||
|
minMappersAcceptedByOraOop));
|
||||||
|
}
|
||||||
|
|
||||||
|
return desiredNumberOfMappers;
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<String> getPartitionList(JobContext jobContext) {
|
||||||
|
LOG.debug(OraOopConstants.ORAOOP_IMPORT_PARTITION_LIST
|
||||||
|
+ " = "
|
||||||
|
+ jobContext.getConfiguration().get(
|
||||||
|
OraOopConstants.ORAOOP_IMPORT_PARTITION_LIST));
|
||||||
|
List<String> result =
|
||||||
|
OraOopUtilities.splitOracleStringList(jobContext.getConfiguration()
|
||||||
|
.get(OraOopConstants.ORAOOP_IMPORT_PARTITION_LIST));
|
||||||
|
if (result != null && result.size() > 0) {
|
||||||
|
LOG.debug("Partition filter list: " + result.toString());
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected
|
||||||
|
List<InputSplit>
|
||||||
|
groupTableDataChunksIntoSplits(
|
||||||
|
List<? extends OraOopOracleDataChunk> dataChunks,
|
||||||
|
int desiredNumberOfSplits,
|
||||||
|
OraOopConstants.OraOopOracleBlockToSplitAllocationMethod
|
||||||
|
blockAllocationMethod) {
|
||||||
|
|
||||||
|
int numberOfDataChunks = dataChunks.size();
|
||||||
|
int actualNumberOfSplits =
|
||||||
|
Math.min(numberOfDataChunks, desiredNumberOfSplits);
|
||||||
|
int totalNumberOfBlocksInAllDataChunks = 0;
|
||||||
|
for (OraOopOracleDataChunk dataChunk : dataChunks) {
|
||||||
|
totalNumberOfBlocksInAllDataChunks += dataChunk.getNumberOfBlocks();
|
||||||
|
}
|
||||||
|
|
||||||
|
String debugMsg = String.format(
|
||||||
|
"The table being imported by sqoop has %d blocks "
|
||||||
|
+ "that have been divided into %d chunks "
|
||||||
|
+ "which will be processed in %d splits. "
|
||||||
|
+ "The chunks will be allocated to the splits using the method : %s",
|
||||||
|
totalNumberOfBlocksInAllDataChunks, numberOfDataChunks,
|
||||||
|
actualNumberOfSplits, blockAllocationMethod.toString());
|
||||||
|
LOG.info(debugMsg);
|
||||||
|
|
||||||
|
List<InputSplit> splits = new ArrayList<InputSplit>(actualNumberOfSplits);
|
||||||
|
|
||||||
|
for (int i = 0; i < actualNumberOfSplits; i++) {
|
||||||
|
OraOopDBInputSplit split = new OraOopDBInputSplit();
|
||||||
|
split.setSplitId(i);
|
||||||
|
split.setTotalNumberOfBlocksInAllSplits(
|
||||||
|
totalNumberOfBlocksInAllDataChunks);
|
||||||
|
splits.add(split);
|
||||||
|
}
|
||||||
|
|
||||||
|
switch (blockAllocationMethod) {
|
||||||
|
|
||||||
|
case RANDOM:
|
||||||
|
// Randomize the order of the data chunks and then "fall through" into
|
||||||
|
// the ROUNDROBIN block below...
|
||||||
|
Collections.shuffle(dataChunks);
|
||||||
|
|
||||||
|
// NB: No "break;" statement here - we're intentionally falling into the
|
||||||
|
// ROUNDROBIN block below...
|
||||||
|
|
||||||
|
//$FALL-THROUGH$
|
||||||
|
case ROUNDROBIN:
|
||||||
|
int idxSplitRoundRobin = 0;
|
||||||
|
for (OraOopOracleDataChunk dataChunk : dataChunks) {
|
||||||
|
|
||||||
|
if (idxSplitRoundRobin >= splits.size()) {
|
||||||
|
idxSplitRoundRobin = 0;
|
||||||
|
}
|
||||||
|
OraOopDBInputSplit split =
|
||||||
|
(OraOopDBInputSplit) splits.get(idxSplitRoundRobin++);
|
||||||
|
|
||||||
|
split.getDataChunks().add(dataChunk);
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
|
||||||
|
case SEQUENTIAL:
|
||||||
|
double dataChunksPerSplit = dataChunks.size() / (double) splits.size();
|
||||||
|
int dataChunksAllocatedToSplits = 0;
|
||||||
|
|
||||||
|
int idxSplitSeq = 0;
|
||||||
|
for (OraOopOracleDataChunk dataChunk : dataChunks) {
|
||||||
|
|
||||||
|
OraOopDBInputSplit split
|
||||||
|
= (OraOopDBInputSplit) splits.get(idxSplitSeq);
|
||||||
|
split.getDataChunks().add(dataChunk);
|
||||||
|
|
||||||
|
dataChunksAllocatedToSplits++;
|
||||||
|
|
||||||
|
if (dataChunksAllocatedToSplits
|
||||||
|
>= (dataChunksPerSplit * (idxSplitSeq + 1))
|
||||||
|
&& idxSplitSeq < splits.size()) {
|
||||||
|
idxSplitSeq++;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
|
||||||
|
default:
|
||||||
|
throw new RuntimeException("Block allocation method not implemented.");
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
for (int idx = 0; idx < splits.size(); idx++) {
|
||||||
|
LOG.debug("\n\t"
|
||||||
|
+ ((OraOopDBInputSplit) splits.get(idx)).getDebugDetails());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return splits;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
64
src/java/org/apache/sqoop/manager/oracle/OraOopGenerics.java
Normal file
64
src/java/org/apache/sqoop/manager/oracle/OraOopGenerics.java
Normal file
@ -0,0 +1,64 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generic class to hold list of objects.
|
||||||
|
*/
|
||||||
|
public class OraOopGenerics {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generic class to hold list of objects.
|
||||||
|
*/
|
||||||
|
public static class ObjectList<T> {
|
||||||
|
|
||||||
|
private List<T> objects;
|
||||||
|
|
||||||
|
public ObjectList() {
|
||||||
|
|
||||||
|
this.objects = new ArrayList<T>();
|
||||||
|
}
|
||||||
|
|
||||||
|
public void add(T item) {
|
||||||
|
|
||||||
|
this.objects.add(item);
|
||||||
|
}
|
||||||
|
|
||||||
|
public int size() {
|
||||||
|
|
||||||
|
return this.objects.size();
|
||||||
|
}
|
||||||
|
|
||||||
|
public T get(int index) {
|
||||||
|
|
||||||
|
return this.objects.get(index);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Iterator<T> iterator() {
|
||||||
|
|
||||||
|
return this.objects.iterator();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
232
src/java/org/apache/sqoop/manager/oracle/OraOopJdbcUrl.java
Normal file
232
src/java/org/apache/sqoop/manager/oracle/OraOopJdbcUrl.java
Normal file
@ -0,0 +1,232 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle;
|
||||||
|
|
||||||
|
import org.apache.sqoop.manager.oracle.OraOopUtilities.JdbcOracleThinConnection;
|
||||||
|
import org.apache.sqoop.manager.oracle.OraOopUtilities
|
||||||
|
.JdbcOracleThinConnectionParsingError;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Parses the Oracle connection string.
|
||||||
|
*/
|
||||||
|
public class OraOopJdbcUrl {
|
||||||
|
|
||||||
|
private String jdbcConnectString;
|
||||||
|
|
||||||
|
public OraOopJdbcUrl(String jdbcConnectString) {
|
||||||
|
|
||||||
|
if (jdbcConnectString == null) {
|
||||||
|
throw new IllegalArgumentException(
|
||||||
|
"The jdbcConnectionString argument must not be null.");
|
||||||
|
}
|
||||||
|
|
||||||
|
if (jdbcConnectString.isEmpty()) {
|
||||||
|
throw new IllegalArgumentException(
|
||||||
|
"The jdbcConnectionString argument must not be empty.");
|
||||||
|
}
|
||||||
|
|
||||||
|
this.jdbcConnectString = jdbcConnectString;
|
||||||
|
}
|
||||||
|
|
||||||
|
public JdbcOracleThinConnection parseJdbcOracleThinConnectionString()
|
||||||
|
throws JdbcOracleThinConnectionParsingError {
|
||||||
|
|
||||||
|
/*
|
||||||
|
* http://wiki.oracle.com/page/JDBC
|
||||||
|
*
|
||||||
|
* There are different flavours of JDBC connections for Oracle, including:
|
||||||
|
* Thin E.g. jdbc:oracle:thin:@localhost.locadomain:1521:orcl
|
||||||
|
*
|
||||||
|
* A pure Java driver used on the client side that does not need an Oracle
|
||||||
|
* client installation. It is recommended that you use this driver unless
|
||||||
|
* you need support for non-TCP/IP networks because it provides for maximum
|
||||||
|
* portability and performance.
|
||||||
|
*
|
||||||
|
* Oracle Call Interface driver (OCI). E.g. jdbc:oracle:oci8:@orcl.world
|
||||||
|
* //<- "orcl.world" is a TNS entry
|
||||||
|
*
|
||||||
|
* This uses the Oracle client installation libraries and interfaces. If you
|
||||||
|
* want to support connection pooling or client side caching of requests,
|
||||||
|
* use this driver. You will also need this driver if you are using
|
||||||
|
* transparent application failover (TAF) from your application as well as
|
||||||
|
* strong authentication like Kerberos and PKI certificates.
|
||||||
|
*
|
||||||
|
* JDBC-ODBC bridge. E.g. jdbc:odbc:mydatabase //<- "mydatabase" is an ODBC
|
||||||
|
* data source.
|
||||||
|
*
|
||||||
|
* This uses the ODBC driver in Windows to connect to the database.
|
||||||
|
*/
|
||||||
|
|
||||||
|
String hostName = null;
|
||||||
|
int port = 0;
|
||||||
|
String sid = null;
|
||||||
|
String service = null;
|
||||||
|
|
||||||
|
String jdbcUrl = this.jdbcConnectString.trim();
|
||||||
|
|
||||||
|
// If there are any parameters included at the end of the connection URL,
|
||||||
|
// let's remove them now...
|
||||||
|
int paramsIdx = jdbcUrl.indexOf("?");
|
||||||
|
if (paramsIdx > -1) {
|
||||||
|
jdbcUrl = jdbcUrl.substring(0, paramsIdx);
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* The format of an Oracle jdbc URL is one of:
|
||||||
|
* jdbc:oracle:<driver-type>:@<host>:<port>:<sid>
|
||||||
|
* jdbc:oracle:<driver-type>:@<host>:<port>/<service>
|
||||||
|
* jdbc:oracle:<driver-type>:@<host>:<port>/<service>?<parameters>
|
||||||
|
* jdbc:oracle:<driver-type>:@//<host>:<port>/<service>
|
||||||
|
* jdbc:oracle:<driver-type>:@//<host>:<port>/<service>?<parameters>
|
||||||
|
*/
|
||||||
|
|
||||||
|
// Split the URL on its ":" characters...
|
||||||
|
String[] jdbcFragments = jdbcUrl.trim().split(":");
|
||||||
|
|
||||||
|
// Clean up each fragment of the URL...
|
||||||
|
for (int idx = 0; idx < jdbcFragments.length; idx++) {
|
||||||
|
jdbcFragments[idx] = jdbcFragments[idx].trim();
|
||||||
|
}
|
||||||
|
|
||||||
|
// Check we can proceed...
|
||||||
|
if (jdbcFragments.length < 5 || jdbcFragments.length > 6) {
|
||||||
|
throw new JdbcOracleThinConnectionParsingError(
|
||||||
|
String.format(
|
||||||
|
"There should be 5 or 6 colon-separated pieces of data in the JDBC "
|
||||||
|
+ "URL, such as:\n\tjdbc:oracle:<driver-type>:@<host>:<port>:<sid>\n"
|
||||||
|
+ "\tjdbc:oracle:<driver-type>:@<host>:<port>/<service>\n"
|
||||||
|
+ "\tjdbc:oracle:<driver-type>:@<host>:<port>/<service>?<parameters>\n"
|
||||||
|
+ "The JDBC URL specified was:\n"
|
||||||
|
+ "%s\n"
|
||||||
|
+ "which contains %d pieces of colon-separated data.",
|
||||||
|
this.jdbcConnectString, jdbcFragments.length));
|
||||||
|
}
|
||||||
|
|
||||||
|
// jdbc
|
||||||
|
if (!jdbcFragments[0].equalsIgnoreCase("jdbc")) {
|
||||||
|
throw new JdbcOracleThinConnectionParsingError(
|
||||||
|
"The first item in the colon-separated JDBC URL must be \"jdbc\".");
|
||||||
|
}
|
||||||
|
|
||||||
|
// jdbc:oracle
|
||||||
|
if (!jdbcFragments[1].equalsIgnoreCase("oracle")) {
|
||||||
|
throw new JdbcOracleThinConnectionParsingError(
|
||||||
|
"The second item in the colon-separated JDBC URL must be \"oracle\".");
|
||||||
|
}
|
||||||
|
|
||||||
|
// jdbc:oracle:thin
|
||||||
|
if (!jdbcFragments[2].equalsIgnoreCase("thin")) {
|
||||||
|
throw new JdbcOracleThinConnectionParsingError(
|
||||||
|
String
|
||||||
|
.format(
|
||||||
|
"The Oracle \"thin\" JDBC driver is not being used.\n"
|
||||||
|
+ "The third item in the colon-separated JDBC URL must "
|
||||||
|
+ "be \"thin\", not \"%s\".",
|
||||||
|
jdbcFragments[2]));
|
||||||
|
}
|
||||||
|
|
||||||
|
// jdbc:oracle:thin:@<host>
|
||||||
|
hostName = jdbcFragments[3];
|
||||||
|
if (hostName.isEmpty() || hostName.equalsIgnoreCase("@")) {
|
||||||
|
throw new JdbcOracleThinConnectionParsingError(
|
||||||
|
"The fourth item in the colon-separated JDBC URL (the host name) "
|
||||||
|
+ "must not be empty.");
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!hostName.startsWith("@")) {
|
||||||
|
throw new JdbcOracleThinConnectionParsingError(
|
||||||
|
"The fourth item in the colon-separated JDBC URL (the host name) "
|
||||||
|
+ "must a prefixed with the \"@\" character.");
|
||||||
|
}
|
||||||
|
|
||||||
|
String portStr = "";
|
||||||
|
switch (jdbcFragments.length) {
|
||||||
|
case 6:
|
||||||
|
// jdbc:oracle:<driver-type>:@<host>:<port>:<sid>
|
||||||
|
portStr = jdbcFragments[4];
|
||||||
|
sid = jdbcFragments[5];
|
||||||
|
break;
|
||||||
|
|
||||||
|
case 5:
|
||||||
|
// jdbc:oracle:<driver-type>:@<host>:<port>/<service>
|
||||||
|
String[] portAndService = jdbcFragments[4].split("/");
|
||||||
|
if (portAndService.length != 2) {
|
||||||
|
throw new JdbcOracleThinConnectionParsingError(
|
||||||
|
"The fifth colon-separated item in the JDBC URL "
|
||||||
|
+ "(<port>/<service>) must contain two items "
|
||||||
|
+ "separated by a \"/\".");
|
||||||
|
}
|
||||||
|
portStr = portAndService[0].trim();
|
||||||
|
service = portAndService[1].trim();
|
||||||
|
break;
|
||||||
|
|
||||||
|
default:
|
||||||
|
throw new JdbcOracleThinConnectionParsingError("Internal error parsing "
|
||||||
|
+ "JDBC connection string.");
|
||||||
|
}
|
||||||
|
|
||||||
|
if (portStr.isEmpty()) {
|
||||||
|
throw new JdbcOracleThinConnectionParsingError(
|
||||||
|
"The fifth item in the colon-separated JDBC URL (the port) must not "
|
||||||
|
+ "be empty.");
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
port = Integer.parseInt(portStr);
|
||||||
|
} catch (NumberFormatException ex) {
|
||||||
|
throw new JdbcOracleThinConnectionParsingError(
|
||||||
|
String
|
||||||
|
.format(
|
||||||
|
"The fifth item in the colon-separated JDBC URL (the port) "
|
||||||
|
+ "must be a valid number.\n"
|
||||||
|
+ "\"%s\" could not be parsed as an integer.", portStr));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (port <= 0) {
|
||||||
|
throw new JdbcOracleThinConnectionParsingError(
|
||||||
|
String
|
||||||
|
.format(
|
||||||
|
"The fifth item in the colon-separated JDBC URL (the port) "
|
||||||
|
+ "must be greater than zero.\n"
|
||||||
|
+ "\"%s\" was specified.", portStr));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (sid == null && service == null) {
|
||||||
|
throw new JdbcOracleThinConnectionParsingError(
|
||||||
|
"The JDBC URL does not contain a SID or SERVICE. The URL should look "
|
||||||
|
+ "like one of these:\n\tjdbc:oracle:<driver-type>:@<host>:<port>:<sid>\n"
|
||||||
|
+ "\tjdbc:oracle:<driver-type>:@<host>:<port>/<service>\n"
|
||||||
|
+ "\tjdbc:oracle:<driver-type>:@<host>:<port>/<service>?<parameters>\n"
|
||||||
|
+ "\tjdbc:oracle:<driver-type>:@//<host>:<port>/<service>\n"
|
||||||
|
+ "\tjdbc:oracle:<driver-type>:@<host>:<port>/<service>?<parameters>");
|
||||||
|
}
|
||||||
|
|
||||||
|
// Remove the "@" prefix of the hostname
|
||||||
|
JdbcOracleThinConnection result =
|
||||||
|
new JdbcOracleThinConnection(hostName.replaceFirst("^[@][/]{0,2}", "")
|
||||||
|
, port, sid, service);
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getConnectionUrl() {
|
||||||
|
return this.jdbcConnectString;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
235
src/java/org/apache/sqoop/manager/oracle/OraOopLog.java
Normal file
235
src/java/org/apache/sqoop/manager/oracle/OraOopLog.java
Normal file
@ -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;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -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));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -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()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
1126
src/java/org/apache/sqoop/manager/oracle/OraOopManagerFactory.java
Normal file
1126
src/java/org/apache/sqoop/manager/oracle/OraOopManagerFactory.java
Normal file
File diff suppressed because it is too large
Load Diff
@ -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;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -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();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
1687
src/java/org/apache/sqoop/manager/oracle/OraOopOracleQueries.java
Normal file
1687
src/java/org/apache/sqoop/manager/oracle/OraOopOracleQueries.java
Normal file
File diff suppressed because it is too large
Load Diff
@ -0,0 +1,713 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.math.BigDecimal;
|
||||||
|
import java.sql.Connection;
|
||||||
|
import java.sql.PreparedStatement;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.sql.Timestamp;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.mapreduce.JobContext;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.mapreduce.AsyncSqlOutputFormat;
|
||||||
|
import com.cloudera.sqoop.mapreduce.ExportOutputFormat;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
||||||
|
|
||||||
|
abstract class OraOopOutputFormatBase<K extends SqoopRecord, V> extends
|
||||||
|
ExportOutputFormat<K, V> {
|
||||||
|
|
||||||
|
private static final OraOopLog LOG = OraOopLogFactory
|
||||||
|
.getLog(OraOopOutputFormatBase.class);
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void checkOutputSpecs(JobContext context) throws IOException,
|
||||||
|
InterruptedException {
|
||||||
|
|
||||||
|
super.checkOutputSpecs(context);
|
||||||
|
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
|
||||||
|
// This code is now running on a Datanode in the Hadoop cluster, so we
|
||||||
|
// need to enable debug logging in this JVM...
|
||||||
|
OraOopUtilities.enableDebugLoggingIfRequired(conf);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected int getMapperId(TaskAttemptContext context) {
|
||||||
|
|
||||||
|
return context.getTaskAttemptID().getTaskID().getId();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void applyMapperJdbcUrl(TaskAttemptContext context, int mapperId) {
|
||||||
|
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
|
||||||
|
// Retrieve the JDBC URL that should be used by this mapper.
|
||||||
|
// We achieve this by modifying the JDBC URL property in the
|
||||||
|
// configuration, prior to the OraOopDBRecordWriter's (ancestral)
|
||||||
|
// constructor using the configuration to establish a connection
|
||||||
|
// to the database - via DBConfiguration.getConnection()...
|
||||||
|
String mapperJdbcUrlPropertyName =
|
||||||
|
OraOopUtilities.getMapperJdbcUrlPropertyName(mapperId, conf);
|
||||||
|
|
||||||
|
// Get this mapper's JDBC URL
|
||||||
|
String mapperJdbcUrl = conf.get(mapperJdbcUrlPropertyName, null);
|
||||||
|
|
||||||
|
LOG.debug(String.format("Mapper %d has a JDBC URL of: %s", mapperId,
|
||||||
|
mapperJdbcUrl == null ? "<null>" : mapperJdbcUrl));
|
||||||
|
|
||||||
|
if (mapperJdbcUrl != null) {
|
||||||
|
conf.set(DBConfiguration.URL_PROPERTY, mapperJdbcUrl);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected boolean canUseOracleAppendValuesHint(TaskAttemptContext context) {
|
||||||
|
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
|
||||||
|
// Should we use the APPEND_VALUES Oracle hint?...
|
||||||
|
// (Yes, if this is Oracle 11.2 or above)...
|
||||||
|
OracleVersion oracleVersion =
|
||||||
|
new OracleVersion(conf.getInt(
|
||||||
|
OraOopConstants.ORAOOP_ORACLE_DATABASE_VERSION_MAJOR, 0), conf
|
||||||
|
.getInt(OraOopConstants.ORAOOP_ORACLE_DATABASE_VERSION_MINOR, 0),
|
||||||
|
0, 0, "");
|
||||||
|
|
||||||
|
boolean result = oracleVersion.isGreaterThanOrEqualTo(11, 2, 0, 0);
|
||||||
|
|
||||||
|
// If there is a BINARY_DOUBLE or BINARY_FLOAT column, then we'll avoid
|
||||||
|
// using
|
||||||
|
// the APPEND_VALUES hint. If there is a NULL in the HDFS file, then we'll
|
||||||
|
// encounter
|
||||||
|
// "ORA-12838: cannot read/modify an object after modifying it in parallel"
|
||||||
|
// due to the JDBC driver issuing the INSERT statement twice to the database
|
||||||
|
// without a COMMIT in between (as was observed via WireShark).
|
||||||
|
// We're not sure why this happens - we just know how to avoid it.
|
||||||
|
if (result) {
|
||||||
|
boolean binaryDoubleColumnExists =
|
||||||
|
conf.getBoolean(OraOopConstants.TABLE_CONTAINS_BINARY_DOUBLE_COLUMN,
|
||||||
|
false);
|
||||||
|
boolean binaryFloatColumnExists =
|
||||||
|
conf.getBoolean(OraOopConstants.TABLE_CONTAINS_BINARY_FLOAT_COLUMN,
|
||||||
|
false);
|
||||||
|
if (binaryDoubleColumnExists || binaryFloatColumnExists) {
|
||||||
|
result = false;
|
||||||
|
LOG.info("The APPEND_VALUES Oracle hint will not be used for the "
|
||||||
|
+ "INSERT SQL statement, as the Oracle table "
|
||||||
|
+ "contains either a BINARY_DOUBLE or BINARY_FLOAT column.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected boolean allowUserToOverrideUseOfTheOracleAppendValuesHint(
|
||||||
|
TaskAttemptContext context, boolean useAppendValuesOracleHint) {
|
||||||
|
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
|
||||||
|
boolean result = useAppendValuesOracleHint;
|
||||||
|
|
||||||
|
// Has the user forced the use of APPEND_VALUES either on or off?...
|
||||||
|
switch (OraOopUtilities.getOracleAppendValuesHintUsage(conf)) {
|
||||||
|
|
||||||
|
case OFF:
|
||||||
|
result = false;
|
||||||
|
LOG.debug(String
|
||||||
|
.format(
|
||||||
|
"Use of the APPEND_VALUES Oracle hint has been forced OFF. "
|
||||||
|
+ "(It was %s to used).",
|
||||||
|
useAppendValuesOracleHint ? "going" : "not going"));
|
||||||
|
break;
|
||||||
|
|
||||||
|
case ON:
|
||||||
|
result = true;
|
||||||
|
LOG.debug(String
|
||||||
|
.format(
|
||||||
|
"Use of the APPEND_VALUES Oracle hint has been forced ON. "
|
||||||
|
+ "(It was %s to used).",
|
||||||
|
useAppendValuesOracleHint ? "going" : "not going"));
|
||||||
|
break;
|
||||||
|
|
||||||
|
case AUTO:
|
||||||
|
LOG.debug(String.format("The APPEND_VALUES Oracle hint %s be used.",
|
||||||
|
result ? "will" : "will not"));
|
||||||
|
break;
|
||||||
|
|
||||||
|
default:
|
||||||
|
throw new RuntimeException("Invalid value for APPEND_VALUES.");
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void updateBatchSizeInConfigurationToAllowOracleAppendValuesHint(
|
||||||
|
TaskAttemptContext context) {
|
||||||
|
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
|
||||||
|
// If using APPEND_VALUES, check the batch size and commit frequency...
|
||||||
|
int originalBatchesPerCommit =
|
||||||
|
conf.getInt(AsyncSqlOutputFormat.STATEMENTS_PER_TRANSACTION_KEY, 0);
|
||||||
|
if (originalBatchesPerCommit != 1) {
|
||||||
|
conf.setInt(AsyncSqlOutputFormat.STATEMENTS_PER_TRANSACTION_KEY, 1);
|
||||||
|
LOG.info(String
|
||||||
|
.format(
|
||||||
|
"The number of batch-inserts to perform per commit has been "
|
||||||
|
+ "changed from %d to %d. This is in response "
|
||||||
|
+ "to the Oracle APPEND_VALUES hint being used.",
|
||||||
|
originalBatchesPerCommit, 1));
|
||||||
|
}
|
||||||
|
|
||||||
|
int originalBatchSize =
|
||||||
|
conf.getInt(AsyncSqlOutputFormat.RECORDS_PER_STATEMENT_KEY, 0);
|
||||||
|
int minAppendValuesBatchSize =
|
||||||
|
OraOopUtilities.getMinAppendValuesBatchSize(conf);
|
||||||
|
if (originalBatchSize < minAppendValuesBatchSize) {
|
||||||
|
conf.setInt(AsyncSqlOutputFormat.RECORDS_PER_STATEMENT_KEY,
|
||||||
|
minAppendValuesBatchSize);
|
||||||
|
LOG.info(String
|
||||||
|
.format(
|
||||||
|
"The number of rows per batch-insert has been changed from %d "
|
||||||
|
+ "to %d. This is in response "
|
||||||
|
+ "to the Oracle APPEND_VALUES hint being used.",
|
||||||
|
originalBatchSize, minAppendValuesBatchSize));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
abstract class OraOopDBRecordWriterBase extends
|
||||||
|
ExportOutputFormat<K, V>.ExportRecordWriter<K, V> {
|
||||||
|
|
||||||
|
protected OracleTable oracleTable; // <- If exporting into a partitioned
|
||||||
|
// table, this table will be unique for
|
||||||
|
// this mapper
|
||||||
|
private OracleTableColumns oracleTableColumns; // <- The columns in the
|
||||||
|
// table we're inserting rows
|
||||||
|
// into
|
||||||
|
protected int mapperId; // <- The index of this Hadoop mapper
|
||||||
|
protected boolean tableHasMapperRowNumberColumn; // <- Whether the export
|
||||||
|
// table contain the column
|
||||||
|
// ORAOOP_MAPPER_ROW
|
||||||
|
protected long mapperRowNumber; // <- The 1-based row number being processed
|
||||||
|
// by this mapper. It's inserted into the
|
||||||
|
// "ORAOOP_MAPPER_ROW" column
|
||||||
|
|
||||||
|
public OraOopDBRecordWriterBase(TaskAttemptContext context, int mapperId)
|
||||||
|
throws ClassNotFoundException, SQLException {
|
||||||
|
|
||||||
|
super(context);
|
||||||
|
this.mapperId = mapperId;
|
||||||
|
this.mapperRowNumber = 1;
|
||||||
|
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
|
||||||
|
// Log any info that might be useful to us...
|
||||||
|
logBatchSettings();
|
||||||
|
|
||||||
|
// Connect to Oracle...
|
||||||
|
Connection connection = this.getConnection();
|
||||||
|
|
||||||
|
String thisOracleInstanceName =
|
||||||
|
OraOopOracleQueries.getCurrentOracleInstanceName(connection);
|
||||||
|
LOG.info(String.format(
|
||||||
|
"This record writer is connected to Oracle via the JDBC URL: \n"
|
||||||
|
+ "\t\"%s\"\n" + "\tto the Oracle instance: \"%s\"", connection
|
||||||
|
.toString(), thisOracleInstanceName));
|
||||||
|
|
||||||
|
// Initialize the Oracle session...
|
||||||
|
OracleConnectionFactory.initializeOracleConnection(connection, conf);
|
||||||
|
connection.setAutoCommit(false);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void setOracleTableColumns(
|
||||||
|
OracleTableColumns newOracleTableColumns) {
|
||||||
|
|
||||||
|
this.oracleTableColumns = newOracleTableColumns;
|
||||||
|
this.tableHasMapperRowNumberColumn =
|
||||||
|
this.oracleTableColumns.findColumnByName(
|
||||||
|
OraOopConstants.COLUMN_NAME_EXPORT_MAPPER_ROW) != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected OracleTableColumns getOracleTableColumns() {
|
||||||
|
|
||||||
|
return this.oracleTableColumns;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void getExportTableAndColumns(TaskAttemptContext context)
|
||||||
|
throws SQLException {
|
||||||
|
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
|
||||||
|
String schema =
|
||||||
|
context.getConfiguration().get(OraOopConstants.ORAOOP_TABLE_OWNER);
|
||||||
|
String localTableName =
|
||||||
|
context.getConfiguration().get(OraOopConstants.ORAOOP_TABLE_NAME);
|
||||||
|
|
||||||
|
if (schema == null || schema.isEmpty() || localTableName == null
|
||||||
|
|| localTableName.isEmpty()) {
|
||||||
|
throw new RuntimeException(
|
||||||
|
"Unable to recall the schema and name of the Oracle table "
|
||||||
|
+ "being exported.");
|
||||||
|
}
|
||||||
|
|
||||||
|
this.oracleTable = new OracleTable(schema, localTableName);
|
||||||
|
|
||||||
|
setOracleTableColumns(OraOopOracleQueries.getTableColumns(this
|
||||||
|
.getConnection(), this.oracleTable, OraOopUtilities
|
||||||
|
.omitLobAndLongColumnsDuringImport(conf), OraOopUtilities
|
||||||
|
.recallSqoopJobType(conf), true // <- onlyOraOopSupportedTypes
|
||||||
|
, false // <- omitOraOopPseudoColumns
|
||||||
|
));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected PreparedStatement getPreparedStatement(
|
||||||
|
List<SqoopRecord> userRecords) throws SQLException {
|
||||||
|
|
||||||
|
Connection connection = this.getConnection();
|
||||||
|
|
||||||
|
String sql = getBatchSqlStatement();
|
||||||
|
LOG.debug(String.format("Prepared Statement SQL:\n%s", sql));
|
||||||
|
|
||||||
|
PreparedStatement statement;
|
||||||
|
|
||||||
|
try {
|
||||||
|
// Synchronize on connection to ensure this does not conflict
|
||||||
|
// with the operations in the update thread.
|
||||||
|
synchronized (connection) {
|
||||||
|
statement = connection.prepareStatement(sql);
|
||||||
|
}
|
||||||
|
|
||||||
|
configurePreparedStatement(statement, userRecords);
|
||||||
|
} catch (Exception ex) {
|
||||||
|
if (ex instanceof SQLException) {
|
||||||
|
throw (SQLException) ex;
|
||||||
|
} else {
|
||||||
|
LOG.error(String.format("The following error occurred during %s",
|
||||||
|
OraOopUtilities.getCurrentMethodName()), ex);
|
||||||
|
throw new SQLException(ex);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return statement;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected boolean isBatchExec() {
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getInsertStatement(int numRows) {
|
||||||
|
|
||||||
|
throw new UnsupportedOperationException(String.format(
|
||||||
|
"%s should not be called, as %s operates in batch mode.",
|
||||||
|
OraOopUtilities.getCurrentMethodName(), this.getClass().getName()));
|
||||||
|
}
|
||||||
|
|
||||||
|
protected String getBatchInsertSqlStatement(String oracleHint) {
|
||||||
|
|
||||||
|
// String[] columnNames = this.getColumnNames();
|
||||||
|
StringBuilder sqlNames = new StringBuilder();
|
||||||
|
StringBuilder sqlValues = new StringBuilder();
|
||||||
|
|
||||||
|
/*
|
||||||
|
* NOTE: "this.oracleTableColumns" may contain a different list of columns
|
||||||
|
* than "this.getColumnNames()". This is because: (1)
|
||||||
|
* "this.getColumnNames()" includes columns with data-types that are not
|
||||||
|
* supported by OraOop. (2) "this.oracleTableColumns" includes any
|
||||||
|
* pseudo-columns that we've added to the export table (and don't exist in
|
||||||
|
* the HDFS file being read). For example, if exporting to a partitioned
|
||||||
|
* table (that OraOop created), there are two pseudo-columns we added to
|
||||||
|
* the table to identify the export job and the mapper.
|
||||||
|
*/
|
||||||
|
|
||||||
|
int colCount = 0;
|
||||||
|
for (int idx = 0; idx < this.oracleTableColumns.size(); idx++) {
|
||||||
|
OracleTableColumn oracleTableColumn = this.oracleTableColumns.get(idx);
|
||||||
|
String columnName = oracleTableColumn.getName();
|
||||||
|
|
||||||
|
// column names...
|
||||||
|
if (colCount > 0) {
|
||||||
|
sqlNames.append("\n,");
|
||||||
|
}
|
||||||
|
sqlNames.append(columnName);
|
||||||
|
|
||||||
|
// column values...
|
||||||
|
if (colCount > 0) {
|
||||||
|
sqlValues.append("\n,");
|
||||||
|
}
|
||||||
|
|
||||||
|
String pseudoColumnValue =
|
||||||
|
generateInsertValueForPseudoColumn(columnName);
|
||||||
|
|
||||||
|
String bindVarName = null;
|
||||||
|
|
||||||
|
if (pseudoColumnValue != null) {
|
||||||
|
bindVarName = pseudoColumnValue;
|
||||||
|
} else if (oracleTableColumn.getOracleType() == OraOopOracleQueries
|
||||||
|
.getOracleType("STRUCT")) {
|
||||||
|
if (oracleTableColumn.getDataType().equals(
|
||||||
|
OraOopConstants.Oracle.URITYPE)) {
|
||||||
|
bindVarName =
|
||||||
|
String.format("urifactory.getUri(%s)",
|
||||||
|
columnNameToBindVariable(columnName));
|
||||||
|
}
|
||||||
|
} else if (getConf().getBoolean(
|
||||||
|
OraOopConstants.ORAOOP_MAP_TIMESTAMP_AS_STRING,
|
||||||
|
OraOopConstants.ORAOOP_MAP_TIMESTAMP_AS_STRING_DEFAULT)) {
|
||||||
|
if (oracleTableColumn.getOracleType() == OraOopOracleQueries
|
||||||
|
.getOracleType("DATE")) {
|
||||||
|
bindVarName =
|
||||||
|
String.format("to_date(%s, 'yyyy-mm-dd hh24:mi:ss')",
|
||||||
|
columnNameToBindVariable(columnName));
|
||||||
|
} else if (oracleTableColumn.getOracleType() == OraOopOracleQueries
|
||||||
|
.getOracleType("TIMESTAMP")) {
|
||||||
|
bindVarName =
|
||||||
|
String.format("to_timestamp(%s, 'yyyy-mm-dd hh24:mi:ss.ff')",
|
||||||
|
columnNameToBindVariable(columnName));
|
||||||
|
} else if (oracleTableColumn.getOracleType() == OraOopOracleQueries
|
||||||
|
.getOracleType("TIMESTAMPTZ")) {
|
||||||
|
bindVarName =
|
||||||
|
String.format(
|
||||||
|
"to_timestamp_tz(%s, 'yyyy-mm-dd hh24:mi:ss.ff TZR')",
|
||||||
|
columnNameToBindVariable(columnName));
|
||||||
|
} else if (oracleTableColumn.getOracleType() == OraOopOracleQueries
|
||||||
|
.getOracleType("TIMESTAMPLTZ")) {
|
||||||
|
bindVarName =
|
||||||
|
String.format(
|
||||||
|
"to_timestamp_tz(%s, 'yyyy-mm-dd hh24:mi:ss.ff TZR')",
|
||||||
|
columnNameToBindVariable(columnName));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (bindVarName == null) {
|
||||||
|
bindVarName = columnNameToBindVariable(columnName);
|
||||||
|
}
|
||||||
|
|
||||||
|
sqlValues.append(bindVarName);
|
||||||
|
|
||||||
|
colCount++;
|
||||||
|
}
|
||||||
|
|
||||||
|
String sql =
|
||||||
|
String.format("insert %s into %s\n" + "(%s)\n" + "values\n"
|
||||||
|
+ "(%s)\n", oracleHint, this.oracleTable.toString(), sqlNames
|
||||||
|
.toString(), sqlValues.toString());
|
||||||
|
|
||||||
|
LOG.info("Batch-Mode insert statement:\n" + sql);
|
||||||
|
return sql;
|
||||||
|
}
|
||||||
|
|
||||||
|
abstract void configurePreparedStatement(
|
||||||
|
PreparedStatement preparedStatement, List<SqoopRecord> userRecords)
|
||||||
|
throws SQLException;
|
||||||
|
|
||||||
|
private void setBindValueAtName(PreparedStatement statement,
|
||||||
|
String bindValueName, Object bindValue, OracleTableColumn column)
|
||||||
|
throws SQLException {
|
||||||
|
if (column.getOracleType()
|
||||||
|
== OraOopOracleQueries.getOracleType("NUMBER")) {
|
||||||
|
OraOopOracleQueries.setBigDecimalAtName(statement, bindValueName,
|
||||||
|
(BigDecimal) bindValue);
|
||||||
|
} else if (column.getOracleType() == OraOopOracleQueries
|
||||||
|
.getOracleType("VARCHAR")) {
|
||||||
|
OraOopOracleQueries.setStringAtName(statement, bindValueName,
|
||||||
|
(String) bindValue);
|
||||||
|
} else if (column.getOracleType() == OraOopOracleQueries
|
||||||
|
.getOracleType("TIMESTAMP")
|
||||||
|
|| column.getOracleType() == OraOopOracleQueries
|
||||||
|
.getOracleType("TIMESTAMPTZ")
|
||||||
|
|| column.getOracleType() == OraOopOracleQueries
|
||||||
|
.getOracleType("TIMESTAMPLTZ")) {
|
||||||
|
Object objValue = bindValue;
|
||||||
|
if (objValue instanceof Timestamp) {
|
||||||
|
Timestamp value = (Timestamp) objValue;
|
||||||
|
OraOopOracleQueries.setTimestampAtName(statement, bindValueName,
|
||||||
|
value);
|
||||||
|
} else {
|
||||||
|
String value = (String) objValue;
|
||||||
|
|
||||||
|
if (value == null || value.equalsIgnoreCase("null")) {
|
||||||
|
value = "";
|
||||||
|
}
|
||||||
|
|
||||||
|
OraOopOracleQueries.setStringAtName(statement, bindValueName, value);
|
||||||
|
}
|
||||||
|
} else if (column.getOracleType() == OraOopOracleQueries
|
||||||
|
.getOracleType("BINARY_DOUBLE")) {
|
||||||
|
Double value = (Double) bindValue;
|
||||||
|
if (value != null) {
|
||||||
|
OraOopOracleQueries.setBinaryDoubleAtName(statement, bindValueName,
|
||||||
|
value);
|
||||||
|
} else {
|
||||||
|
OraOopOracleQueries.setObjectAtName(statement, bindValueName, null);
|
||||||
|
}
|
||||||
|
} else if (column.getOracleType() == OraOopOracleQueries
|
||||||
|
.getOracleType("BINARY_FLOAT")) {
|
||||||
|
Float value = (Float) bindValue;
|
||||||
|
if (value != null) {
|
||||||
|
OraOopOracleQueries.setBinaryFloatAtName(statement, bindValueName,
|
||||||
|
value);
|
||||||
|
} else {
|
||||||
|
OraOopOracleQueries.setObjectAtName(statement, bindValueName, null);
|
||||||
|
}
|
||||||
|
} else if (column.getOracleType() == OraOopOracleQueries
|
||||||
|
.getOracleType("STRUCT")) { // <- E.g. URITYPE
|
||||||
|
if (column.getDataType().equals(OraOopConstants.Oracle.URITYPE)) {
|
||||||
|
String value = (String) bindValue;
|
||||||
|
OraOopOracleQueries.setStringAtName(statement, bindValueName, value);
|
||||||
|
} else {
|
||||||
|
String msg =
|
||||||
|
String.format(
|
||||||
|
"%s needs to be updated to cope with the data-type: %s "
|
||||||
|
+ "where the Oracle data_type is \"%s\".",
|
||||||
|
OraOopUtilities.getCurrentMethodName(), column.getDataType(),
|
||||||
|
column.getOracleType());
|
||||||
|
LOG.error(msg);
|
||||||
|
throw new UnsupportedOperationException(msg);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// LOB data-types are currently not supported during
|
||||||
|
// a Sqoop Export.
|
||||||
|
// JIRA: SQOOP-117
|
||||||
|
// OraOopConstants.SUPPORTED_EXPORT_ORACLE_DATA_TYPES_CLAUSE
|
||||||
|
// will already have excluded all LOB columns.
|
||||||
|
|
||||||
|
// case oracle.jdbc.OracleTypes.CLOB:
|
||||||
|
// {
|
||||||
|
// oracle.sql.CLOB clob = new
|
||||||
|
// oracle.sql.CLOB(connection);
|
||||||
|
// Object value = fieldMap.get(colName);
|
||||||
|
// //clob.set
|
||||||
|
// statement.setCLOBAtName(bindValueName, clob);
|
||||||
|
// break;
|
||||||
|
// }
|
||||||
|
String msg =
|
||||||
|
String.format(
|
||||||
|
"%s may need to be updated to cope with the data-type: %s",
|
||||||
|
OraOopUtilities.getCurrentMethodName(), column.getOracleType());
|
||||||
|
LOG.debug(msg);
|
||||||
|
|
||||||
|
OraOopOracleQueries
|
||||||
|
.setObjectAtName(statement, bindValueName, bindValue);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void configurePreparedStatementColumns(
|
||||||
|
PreparedStatement statement, Map<String, Object> fieldMap)
|
||||||
|
throws SQLException {
|
||||||
|
|
||||||
|
String bindValueName;
|
||||||
|
|
||||||
|
if (this.tableHasMapperRowNumberColumn) {
|
||||||
|
bindValueName =
|
||||||
|
columnNameToBindVariable(
|
||||||
|
OraOopConstants.COLUMN_NAME_EXPORT_MAPPER_ROW).replaceFirst(
|
||||||
|
":", "");
|
||||||
|
try {
|
||||||
|
OraOopOracleQueries.setLongAtName(statement, bindValueName,
|
||||||
|
this.mapperRowNumber);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
this.mapperRowNumber++;
|
||||||
|
}
|
||||||
|
|
||||||
|
Iterator<String> columnNameIterator = fieldMap.keySet().iterator();
|
||||||
|
while (columnNameIterator.hasNext()) {
|
||||||
|
String colName = columnNameIterator.next();
|
||||||
|
bindValueName = columnNameToBindVariable(colName).replaceFirst(":", "");
|
||||||
|
|
||||||
|
OracleTableColumn oracleTableColumn =
|
||||||
|
oracleTableColumns.findColumnByName(colName);
|
||||||
|
setBindValueAtName(statement, bindValueName, fieldMap.get(colName),
|
||||||
|
oracleTableColumn);
|
||||||
|
}
|
||||||
|
statement.addBatch();
|
||||||
|
}
|
||||||
|
|
||||||
|
abstract String getBatchSqlStatement();
|
||||||
|
|
||||||
|
protected String columnNameToBindVariable(String columnName) {
|
||||||
|
|
||||||
|
return ":" + columnName;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void write(K key, V value) throws InterruptedException, IOException {
|
||||||
|
|
||||||
|
try {
|
||||||
|
super.write(key, value);
|
||||||
|
} catch (IOException ex) {
|
||||||
|
// This IOException may contain a SQLException that occurred
|
||||||
|
// during the batch insert...
|
||||||
|
showSqlBatchErrorDetails(ex);
|
||||||
|
throw ex;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close(TaskAttemptContext context) throws IOException,
|
||||||
|
InterruptedException {
|
||||||
|
|
||||||
|
try {
|
||||||
|
super.close(context);
|
||||||
|
} catch (IOException ex) {
|
||||||
|
// This IOException may contain a SQLException that occurred
|
||||||
|
// during the batch insert...
|
||||||
|
showSqlBatchErrorDetails(ex);
|
||||||
|
throw ex;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void showSqlBatchErrorDetails(Exception exception) {
|
||||||
|
|
||||||
|
if (OraOopUtilities.oracleSessionHasBeenKilled(exception)) {
|
||||||
|
LOG.info("\n*********************************************************"
|
||||||
|
+ "\nThe Oracle session in use has been killed by a 3rd party."
|
||||||
|
+ "\n*********************************************************");
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Unfortunately, BatchUpdateException.getUpdateCounts() only returns
|
||||||
|
* information about UPDATE statements (not INSERT) statements. Since
|
||||||
|
* we're only performing INSERT statements, there's no extra information
|
||||||
|
* we can provide to the user at this point.
|
||||||
|
*/
|
||||||
|
|
||||||
|
// if(exception == null)
|
||||||
|
// return;
|
||||||
|
//
|
||||||
|
// if(exception instanceof BatchUpdateException) {
|
||||||
|
// BatchUpdateException ex = (BatchUpdateException)exception;
|
||||||
|
//
|
||||||
|
// int[] updateCounts = ex.getUpdateCounts();
|
||||||
|
// LOG.error("The number of successful updates was: " +
|
||||||
|
// updateCounts.length);
|
||||||
|
//
|
||||||
|
// // Recurse for chained exceptions...
|
||||||
|
// SQLException nextEx = ex.getNextException();
|
||||||
|
// while(nextEx != null) {
|
||||||
|
// showSqlBatchErrorDetails(nextEx);
|
||||||
|
// nextEx = nextEx.getNextException();
|
||||||
|
// }
|
||||||
|
// }
|
||||||
|
//
|
||||||
|
// // Recurse for nested exceptions...
|
||||||
|
// Throwable cause = exception.getCause();
|
||||||
|
// if(cause instanceof Exception)
|
||||||
|
// showSqlBatchErrorDetails((Exception)cause);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
protected Object getJobSysDate(TaskAttemptContext context) {
|
||||||
|
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
return OraOopUtilities.recallOracleDateTime(conf,
|
||||||
|
OraOopConstants.ORAOOP_JOB_SYSDATE);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected OracleTable createUniqueMapperTable(TaskAttemptContext context)
|
||||||
|
throws SQLException {
|
||||||
|
|
||||||
|
// This mapper inserts data into a unique table before either:
|
||||||
|
// - exchanging it into a subpartition of the 'real' export table; or
|
||||||
|
// - merging it into the 'real' export table.
|
||||||
|
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
|
||||||
|
Object sysDateTime = getJobSysDate(context);
|
||||||
|
|
||||||
|
String schema = conf.get(OraOopConstants.ORAOOP_TABLE_OWNER);
|
||||||
|
String localTableName = conf.get(OraOopConstants.ORAOOP_TABLE_NAME);
|
||||||
|
|
||||||
|
OracleTable templateTable = new OracleTable(schema, localTableName);
|
||||||
|
|
||||||
|
OracleTable mapperTable =
|
||||||
|
OraOopUtilities.generateExportTableMapperTableName(this.mapperId,
|
||||||
|
sysDateTime, null);
|
||||||
|
|
||||||
|
// If this mapper is being reattempted in response to a failure, we need
|
||||||
|
// to delete the
|
||||||
|
// temporary table created by the previous attempt...
|
||||||
|
OraOopOracleQueries.dropTable(this.getConnection(), mapperTable);
|
||||||
|
|
||||||
|
String temporaryTableStorageClause =
|
||||||
|
OraOopUtilities.getTemporaryTableStorageClause(conf);
|
||||||
|
|
||||||
|
OraOopOracleQueries.createExportTableForMapper(this.getConnection(),
|
||||||
|
mapperTable, temporaryTableStorageClause, templateTable
|
||||||
|
, false); // <- addOraOopPartitionColumns
|
||||||
|
|
||||||
|
LOG.debug(String.format("Created temporary mapper table %s", mapperTable
|
||||||
|
.toString()));
|
||||||
|
|
||||||
|
return mapperTable;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected String generateInsertValueForPseudoColumn(String columnName) {
|
||||||
|
|
||||||
|
if (columnName
|
||||||
|
.equalsIgnoreCase(OraOopConstants.COLUMN_NAME_EXPORT_PARTITION)) {
|
||||||
|
|
||||||
|
String partitionValueStr =
|
||||||
|
this.getConf().get(
|
||||||
|
OraOopConstants.ORAOOP_EXPORT_PARTITION_DATE_VALUE, null);
|
||||||
|
if (partitionValueStr == null) {
|
||||||
|
throw new RuntimeException(
|
||||||
|
"Unable to recall the value of the partition date-time.");
|
||||||
|
}
|
||||||
|
|
||||||
|
return String.format("to_date('%s', '%s')", partitionValueStr,
|
||||||
|
OraOopConstants.ORAOOP_EXPORT_PARTITION_DATE_FORMAT);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (columnName
|
||||||
|
.equalsIgnoreCase(OraOopConstants.COLUMN_NAME_EXPORT_SUBPARTITION)) {
|
||||||
|
return Integer.toString(this.mapperId);
|
||||||
|
}
|
||||||
|
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void logBatchSettings() {
|
||||||
|
|
||||||
|
LOG.info(String.format("The number of rows per batch is: %d",
|
||||||
|
this.rowsPerStmt));
|
||||||
|
|
||||||
|
int stmtsPerTx =
|
||||||
|
this.getConf().getInt(
|
||||||
|
AsyncSqlOutputFormat.STATEMENTS_PER_TRANSACTION_KEY,
|
||||||
|
AsyncSqlOutputFormat.DEFAULT_STATEMENTS_PER_TRANSACTION);
|
||||||
|
|
||||||
|
LOG.info(String.format("The number of batches per commit is: %d",
|
||||||
|
stmtsPerTx));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,263 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.PreparedStatement;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.mapreduce.RecordWriter;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Insert into an Oracle table based on emitted keys.
|
||||||
|
*/
|
||||||
|
public class OraOopOutputFormatInsert<K extends SqoopRecord, V> extends
|
||||||
|
OraOopOutputFormatBase<K, V> {
|
||||||
|
|
||||||
|
private static final OraOopLog LOG = OraOopLogFactory
|
||||||
|
.getLog(OraOopOutputFormatInsert.class);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Type of insert to use - direct or partition exchange load.
|
||||||
|
*/
|
||||||
|
public enum InsertMode {
|
||||||
|
DirectInsert, ExchangePartition
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
||||||
|
throws IOException {
|
||||||
|
OraOopUtilities.checkJavaSecurityEgd();
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
|
||||||
|
int mapperId = this.getMapperId(context);
|
||||||
|
applyMapperJdbcUrl(context, mapperId);
|
||||||
|
|
||||||
|
// Is each mapper inserting rows into a unique table?...
|
||||||
|
InsertMode insertMode = OraOopUtilities.getExportInsertMode(conf);
|
||||||
|
|
||||||
|
// Should we use the APPEND_VALUES Oracle hint?...
|
||||||
|
boolean useAppendValuesOracleHint = false;
|
||||||
|
if (insertMode == InsertMode.ExchangePartition) {
|
||||||
|
// NB: "Direct inserts" cannot utilize APPEND_VALUES, otherwise Oracle
|
||||||
|
// will serialize
|
||||||
|
// the N mappers, causing a lot of lock contention.
|
||||||
|
useAppendValuesOracleHint = this.canUseOracleAppendValuesHint(context);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Has the user forced the use of APPEND_VALUES either on or off?...
|
||||||
|
useAppendValuesOracleHint =
|
||||||
|
allowUserToOverrideUseOfTheOracleAppendValuesHint(context,
|
||||||
|
useAppendValuesOracleHint);
|
||||||
|
|
||||||
|
// If using APPEND_VALUES, check the batch size and commit frequency...
|
||||||
|
if (useAppendValuesOracleHint) {
|
||||||
|
updateBatchSizeInConfigurationToAllowOracleAppendValuesHint(context);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Create the Record Writer...
|
||||||
|
OraOopDBRecordWriterInsert result = null;
|
||||||
|
try {
|
||||||
|
result =
|
||||||
|
new OraOopDBRecordWriterInsert(context, mapperId, insertMode,
|
||||||
|
useAppendValuesOracleHint);
|
||||||
|
} catch (NoClassDefFoundError ex) {
|
||||||
|
throw new IOException(String.format(
|
||||||
|
"Unable to create an instance of OraOopDBRecordWriterInsert.\n"
|
||||||
|
+ "The classpath is:\n%s", OraOopUtilities.getJavaClassPath()),
|
||||||
|
ex);
|
||||||
|
} catch (Exception ex) {
|
||||||
|
throw new IOException(ex);
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
result.getExportTableAndColumns(context);
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
throw new IOException(ex);
|
||||||
|
}
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Insert into an Oracle table based on emitted keys.
|
||||||
|
*/
|
||||||
|
public class OraOopDBRecordWriterInsert extends OraOopDBRecordWriterBase {
|
||||||
|
|
||||||
|
private String sqlStatement; // <- The SQL used when inserting batches of
|
||||||
|
// rows into the Oracle table
|
||||||
|
private InsertMode insertMode; // <- The modus operandi of this class. i.e.
|
||||||
|
// Whether we insert into the Oracle table
|
||||||
|
// directly, or insert data into a separate
|
||||||
|
// table and then perform an EXCHANGE
|
||||||
|
// PARTITION statement.
|
||||||
|
private boolean useAppendValuesOracleHint; // <- Whether to use the
|
||||||
|
// " /*+APPEND_VALUES*/ " hint
|
||||||
|
// within the Oracle SQL
|
||||||
|
// statement we generate
|
||||||
|
private String subPartitionName; // <- The name of the subpartition in the
|
||||||
|
// "main table" that this mappers unique
|
||||||
|
// table will be exchanged with
|
||||||
|
|
||||||
|
public OraOopDBRecordWriterInsert(TaskAttemptContext context, int mapperId,
|
||||||
|
InsertMode insertMode, boolean useAppendValuesOracleHint)
|
||||||
|
throws ClassNotFoundException, SQLException {
|
||||||
|
|
||||||
|
super(context, mapperId);
|
||||||
|
this.insertMode = insertMode;
|
||||||
|
this.useAppendValuesOracleHint = useAppendValuesOracleHint;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void getExportTableAndColumns(TaskAttemptContext context)
|
||||||
|
throws SQLException {
|
||||||
|
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
|
||||||
|
switch (this.insertMode) {
|
||||||
|
|
||||||
|
case DirectInsert:
|
||||||
|
super.getExportTableAndColumns(context);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case ExchangePartition:
|
||||||
|
// This mapper inserts data into a unique table before exchanging it
|
||||||
|
// into
|
||||||
|
// a subpartition of the 'real' export table...
|
||||||
|
|
||||||
|
this.oracleTable = createUniqueMapperTable(context);
|
||||||
|
setOracleTableColumns(OraOopOracleQueries.getTableColumns(this
|
||||||
|
.getConnection(), this.oracleTable, OraOopUtilities
|
||||||
|
.omitLobAndLongColumnsDuringImport(conf), OraOopUtilities
|
||||||
|
.recallSqoopJobType(conf), true // <- onlyOraOopSupportedTypes
|
||||||
|
, false) // <- omitOraOopPseudoColumns
|
||||||
|
);
|
||||||
|
|
||||||
|
this.subPartitionName =
|
||||||
|
OraOopUtilities.generateExportTableSubPartitionName(
|
||||||
|
this.mapperId, this.getJobSysDate(context), conf);
|
||||||
|
|
||||||
|
break;
|
||||||
|
|
||||||
|
default:
|
||||||
|
throw new RuntimeException(String.format(
|
||||||
|
"Update %s to cater for the insertMode \"%s\".", OraOopUtilities
|
||||||
|
.getCurrentMethodName(), this.insertMode.toString()));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void closeConnection(TaskAttemptContext context)
|
||||||
|
throws SQLException {
|
||||||
|
|
||||||
|
// If this mapper is inserting data into a unique table, we'll now
|
||||||
|
// move this data into the main export table...
|
||||||
|
if (this.insertMode == InsertMode.ExchangePartition) {
|
||||||
|
|
||||||
|
// Perform an "exchange subpartition" operation on the "main table"
|
||||||
|
// to convert this table into a subpartition of the "main table"...
|
||||||
|
exchangePartitionUniqueMapperTableDataIntoMainExportTable(context);
|
||||||
|
|
||||||
|
LOG.debug(String.format("Dropping temporary mapper table %s",
|
||||||
|
this.oracleTable.toString()));
|
||||||
|
OraOopOracleQueries.dropTable(this.getConnection(), this.oracleTable);
|
||||||
|
}
|
||||||
|
|
||||||
|
super.closeConnection(context);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void exchangePartitionUniqueMapperTableDataIntoMainExportTable(
|
||||||
|
TaskAttemptContext context) throws SQLException {
|
||||||
|
|
||||||
|
String schema =
|
||||||
|
context.getConfiguration().get(OraOopConstants.ORAOOP_TABLE_OWNER);
|
||||||
|
String localTableName =
|
||||||
|
context.getConfiguration().get(OraOopConstants.ORAOOP_TABLE_NAME);
|
||||||
|
OracleTable mainTable = new OracleTable(schema, localTableName);
|
||||||
|
|
||||||
|
try {
|
||||||
|
long start = System.nanoTime();
|
||||||
|
|
||||||
|
OraOopOracleQueries.exchangeSubpartition(this.getConnection(),
|
||||||
|
mainTable, this.subPartitionName, this.oracleTable);
|
||||||
|
|
||||||
|
double timeInSec = (System.nanoTime() - start) / Math.pow(10, 9);
|
||||||
|
LOG.info(String
|
||||||
|
.format(
|
||||||
|
"Time spent performing an \"exchange subpartition with "
|
||||||
|
+ "table\": %f sec.",
|
||||||
|
timeInSec));
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
throw new SQLException(
|
||||||
|
String
|
||||||
|
.format(
|
||||||
|
"Unable to perform an \"exchange subpartition\" operation "
|
||||||
|
+ "for the table %s, for the subpartition named "
|
||||||
|
+ "\"%s\" with the table named \"%s\".",
|
||||||
|
mainTable.toString(), this.subPartitionName,
|
||||||
|
this.oracleTable.toString()), ex);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getBatchSqlStatement() {
|
||||||
|
|
||||||
|
if (sqlStatement == null) {
|
||||||
|
this.sqlStatement =
|
||||||
|
getBatchInsertSqlStatement(this.useAppendValuesOracleHint
|
||||||
|
? "/*+APPEND_VALUES*/" : "");
|
||||||
|
}
|
||||||
|
|
||||||
|
return this.sqlStatement;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
void configurePreparedStatement(PreparedStatement statement,
|
||||||
|
List<SqoopRecord> userRecords) throws SQLException {
|
||||||
|
|
||||||
|
Map<String, Object> fieldMap;
|
||||||
|
try {
|
||||||
|
for (SqoopRecord record : userRecords) {
|
||||||
|
fieldMap = record.getFieldMap();
|
||||||
|
|
||||||
|
configurePreparedStatementColumns(statement, fieldMap);
|
||||||
|
}
|
||||||
|
|
||||||
|
} catch (Exception ex) {
|
||||||
|
if (ex instanceof SQLException) {
|
||||||
|
throw (SQLException) ex;
|
||||||
|
} else {
|
||||||
|
LOG.error(String.format("The following error occurred during %s",
|
||||||
|
OraOopUtilities.getCurrentMethodName()), ex);
|
||||||
|
throw new SQLException(ex);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,418 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.PreparedStatement;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.mapreduce.RecordWriter;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import org.apache.sqoop.manager.oracle.OraOopOracleQueries.
|
||||||
|
CreateExportChangesTableOptions;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Update an Oracle table based on emitted keys.
|
||||||
|
*/
|
||||||
|
public class OraOopOutputFormatUpdate<K extends SqoopRecord, V> extends
|
||||||
|
OraOopOutputFormatBase<K, V> {
|
||||||
|
|
||||||
|
private static final OraOopLog LOG = OraOopLogFactory
|
||||||
|
.getLog(OraOopOutputFormatUpdate.class);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Type of export - straight update or merge (update-insert).
|
||||||
|
*/
|
||||||
|
public enum UpdateMode {
|
||||||
|
Update, Merge
|
||||||
|
}
|
||||||
|
|
||||||
|
private enum ExportTableUpdateTechnique {
|
||||||
|
ReInsertUpdatedRows, ReInsertUpdatedRowsAndNewRows, UpdateSql, MergeSql
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
||||||
|
throws IOException {
|
||||||
|
OraOopUtilities.checkJavaSecurityEgd();
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
|
||||||
|
// Get the unique JDBC URL to use for this mapper and update the
|
||||||
|
// configuration property
|
||||||
|
// so that the URL is actually used...
|
||||||
|
int mapperId = this.getMapperId(context);
|
||||||
|
applyMapperJdbcUrl(context, mapperId);
|
||||||
|
|
||||||
|
UpdateMode updateMode = OraOopUtilities.getExportUpdateMode(conf);
|
||||||
|
|
||||||
|
boolean useAppendValuesOracleHint = false;
|
||||||
|
|
||||||
|
if (updateMode == UpdateMode.Merge || updateMode == UpdateMode.Update) {
|
||||||
|
// Should we use the APPEND_VALUES Oracle hint?...
|
||||||
|
useAppendValuesOracleHint = this.canUseOracleAppendValuesHint(context);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Has the user forced the use of APPEND_VALUES either on or off?...
|
||||||
|
useAppendValuesOracleHint =
|
||||||
|
allowUserToOverrideUseOfTheOracleAppendValuesHint(context,
|
||||||
|
useAppendValuesOracleHint);
|
||||||
|
|
||||||
|
// If using APPEND_VALUES, check the batch size and commit frequency...
|
||||||
|
if (useAppendValuesOracleHint) {
|
||||||
|
updateBatchSizeInConfigurationToAllowOracleAppendValuesHint(context);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Create the Record Writer...
|
||||||
|
OraOopDBRecordWriterUpdate result = null;
|
||||||
|
try {
|
||||||
|
result =
|
||||||
|
new OraOopDBRecordWriterUpdate(context, mapperId, updateMode,
|
||||||
|
useAppendValuesOracleHint);
|
||||||
|
} catch (NoClassDefFoundError ex) {
|
||||||
|
throw new IOException(String.format(
|
||||||
|
"Unable to create an instance of OraOopDBRecordWriterUpdate.\n"
|
||||||
|
+ "The classpath is:\n%s", OraOopUtilities.getJavaClassPath()),
|
||||||
|
ex);
|
||||||
|
} catch (Exception ex) {
|
||||||
|
throw new IOException(ex);
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
result.getExportTableAndColumns(context);
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
throw new IOException(ex);
|
||||||
|
}
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Update an Oracle table based on emitted keys.
|
||||||
|
*/
|
||||||
|
public class OraOopDBRecordWriterUpdate extends OraOopDBRecordWriterBase {
|
||||||
|
|
||||||
|
private String sqlStatement; // <- The SQL used when updating batches of
|
||||||
|
// rows into the Oracle table
|
||||||
|
private String[] updateColumnNames; // <- The name of the column(s) used to
|
||||||
|
// match a row in the HDFS file to a row
|
||||||
|
// in the Oracle table. i.e. What as
|
||||||
|
// specified in the "--update-key" sqoop
|
||||||
|
// argument.
|
||||||
|
private UpdateMode updateMode; // <- The modus operandi of this class. i.e.
|
||||||
|
// Whether we update the Oracle table
|
||||||
|
// directly, or insert data into a separate
|
||||||
|
// table and then apply a SQL MERGE
|
||||||
|
// statement.
|
||||||
|
private boolean useAppendValuesOracleHint; // <- Whether to use the
|
||||||
|
// " /*+APPEND_VALUES*/ " hint
|
||||||
|
// within the Oracle SQL
|
||||||
|
// statement we generate
|
||||||
|
private boolean tableHasOraOopPartitions; // <- Indicates whether the export
|
||||||
|
// table has partitions that were
|
||||||
|
// creted by OraOop
|
||||||
|
private long numberOfRowsSkipped; // <- The number of rows encountered
|
||||||
|
// during configurePreparedStatement()
|
||||||
|
// that had a NULL value for (one of) the
|
||||||
|
// update columns. This row was therefore
|
||||||
|
// skipped.
|
||||||
|
|
||||||
|
public OraOopDBRecordWriterUpdate(TaskAttemptContext context, int mapperId,
|
||||||
|
UpdateMode updateMode, boolean useAppendValuesOracleHint)
|
||||||
|
throws ClassNotFoundException, SQLException {
|
||||||
|
|
||||||
|
super(context, mapperId);
|
||||||
|
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
|
||||||
|
this.updateColumnNames =
|
||||||
|
OraOopUtilities.getExportUpdateKeyColumnNames(conf);
|
||||||
|
this.useAppendValuesOracleHint = useAppendValuesOracleHint;
|
||||||
|
this.updateMode = updateMode;
|
||||||
|
this.tableHasOraOopPartitions =
|
||||||
|
conf.getBoolean(OraOopConstants.EXPORT_TABLE_HAS_ORAOOP_PARTITIONS,
|
||||||
|
false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void getExportTableAndColumns(TaskAttemptContext context)
|
||||||
|
throws SQLException {
|
||||||
|
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
|
||||||
|
this.oracleTable = createUniqueMapperTable(context);
|
||||||
|
setOracleTableColumns(OraOopOracleQueries.getTableColumns(this
|
||||||
|
.getConnection(), this.oracleTable, OraOopUtilities
|
||||||
|
.omitLobAndLongColumnsDuringImport(conf), OraOopUtilities
|
||||||
|
.recallSqoopJobType(conf), true // <- onlyOraOopSupportedTypes
|
||||||
|
, false) // <- omitOraOopPseudoColumns
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void closeConnection(TaskAttemptContext context)
|
||||||
|
throws SQLException {
|
||||||
|
|
||||||
|
try {
|
||||||
|
|
||||||
|
if (this.numberOfRowsSkipped > 0) {
|
||||||
|
LOG.warn(String.format(
|
||||||
|
"%d records were skipped due to a NULL value within one of the "
|
||||||
|
+ "update-key column(s).\nHaving a NULL value prevents a record "
|
||||||
|
+ "from being able to be matched to a row in the Oracle table.",
|
||||||
|
this.numberOfRowsSkipped));
|
||||||
|
}
|
||||||
|
|
||||||
|
// Now update the "main" export table with data that was inserted into
|
||||||
|
// this mapper's table...
|
||||||
|
updateMainExportTableFromUniqueMapperTable(context,
|
||||||
|
this.updateColumnNames);
|
||||||
|
|
||||||
|
LOG.debug(String.format("Dropping temporary mapper table %s",
|
||||||
|
this.oracleTable.toString()));
|
||||||
|
OraOopOracleQueries.dropTable(this.getConnection(), this.oracleTable);
|
||||||
|
} finally {
|
||||||
|
super.closeConnection(context);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private ExportTableUpdateTechnique getExportTableUpdateTechnique() {
|
||||||
|
|
||||||
|
ExportTableUpdateTechnique result;
|
||||||
|
|
||||||
|
if (this.tableHasOraOopPartitions) {
|
||||||
|
switch (this.updateMode) {
|
||||||
|
|
||||||
|
case Update:
|
||||||
|
result = ExportTableUpdateTechnique.ReInsertUpdatedRows;
|
||||||
|
break;
|
||||||
|
|
||||||
|
case Merge:
|
||||||
|
result = ExportTableUpdateTechnique.ReInsertUpdatedRowsAndNewRows;
|
||||||
|
break;
|
||||||
|
|
||||||
|
default:
|
||||||
|
throw new RuntimeException(String.format(
|
||||||
|
"Update %s to cater for the updateMode \"%s\".",
|
||||||
|
OraOopUtilities.getCurrentMethodName(), this.updateMode
|
||||||
|
.toString()));
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
switch (this.updateMode) {
|
||||||
|
|
||||||
|
case Update:
|
||||||
|
result = ExportTableUpdateTechnique.UpdateSql;
|
||||||
|
break;
|
||||||
|
|
||||||
|
case Merge:
|
||||||
|
result = ExportTableUpdateTechnique.MergeSql;
|
||||||
|
break;
|
||||||
|
|
||||||
|
default:
|
||||||
|
throw new RuntimeException(String.format(
|
||||||
|
"Update %s to cater for the updateMode \"%s\".",
|
||||||
|
OraOopUtilities.getCurrentMethodName(), this.updateMode
|
||||||
|
.toString()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void updateMainExportTableFromUniqueMapperTable(
|
||||||
|
TaskAttemptContext context, String[] mergeColumnNames)
|
||||||
|
throws SQLException {
|
||||||
|
|
||||||
|
String schema =
|
||||||
|
context.getConfiguration().get(OraOopConstants.ORAOOP_TABLE_OWNER);
|
||||||
|
String localTableName =
|
||||||
|
context.getConfiguration().get(OraOopConstants.ORAOOP_TABLE_NAME);
|
||||||
|
OracleTable targetTable = new OracleTable(schema, localTableName);
|
||||||
|
|
||||||
|
Object sysDateTime = getJobSysDate(context);
|
||||||
|
OracleTable changesTable =
|
||||||
|
OraOopUtilities.generateExportTableMapperTableName(Integer
|
||||||
|
.toString(this.mapperId)
|
||||||
|
+ "_CHG", sysDateTime, null);
|
||||||
|
|
||||||
|
OraOopOracleQueries.CreateExportChangesTableOptions changesTableOptions;
|
||||||
|
boolean parallelizationEnabled =
|
||||||
|
OraOopUtilities.enableOracleParallelProcessingDuringExport(context
|
||||||
|
.getConfiguration());
|
||||||
|
|
||||||
|
ExportTableUpdateTechnique exportTableUpdateTechnique =
|
||||||
|
getExportTableUpdateTechnique();
|
||||||
|
switch (exportTableUpdateTechnique) {
|
||||||
|
|
||||||
|
case ReInsertUpdatedRows:
|
||||||
|
case UpdateSql:
|
||||||
|
changesTableOptions =
|
||||||
|
CreateExportChangesTableOptions.OnlyRowsThatDiffer;
|
||||||
|
break;
|
||||||
|
|
||||||
|
case ReInsertUpdatedRowsAndNewRows:
|
||||||
|
case MergeSql:
|
||||||
|
changesTableOptions =
|
||||||
|
CreateExportChangesTableOptions.RowsThatDifferPlusNewRows;
|
||||||
|
break;
|
||||||
|
|
||||||
|
default:
|
||||||
|
throw new RuntimeException(String.format(
|
||||||
|
"Update %s to cater for the ExportTableUpdateTechnique \"%s\".",
|
||||||
|
OraOopUtilities.getCurrentMethodName(),
|
||||||
|
exportTableUpdateTechnique.toString()));
|
||||||
|
}
|
||||||
|
|
||||||
|
String temporaryTableStorageClause =
|
||||||
|
OraOopUtilities.getTemporaryTableStorageClause(context
|
||||||
|
.getConfiguration());
|
||||||
|
|
||||||
|
try {
|
||||||
|
int changeTableRowCount =
|
||||||
|
OraOopOracleQueries.createExportChangesTable(this.getConnection(),
|
||||||
|
changesTable, temporaryTableStorageClause, this.oracleTable,
|
||||||
|
targetTable, this.updateColumnNames, changesTableOptions,
|
||||||
|
parallelizationEnabled);
|
||||||
|
|
||||||
|
if (changeTableRowCount == 0) {
|
||||||
|
LOG.debug(String.format(
|
||||||
|
"The changes-table does not contain any rows. %s is now exiting.",
|
||||||
|
OraOopUtilities.getCurrentMethodName()));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
switch (exportTableUpdateTechnique) {
|
||||||
|
|
||||||
|
case ReInsertUpdatedRows:
|
||||||
|
case ReInsertUpdatedRowsAndNewRows:
|
||||||
|
|
||||||
|
OraOopOracleQueries.deleteRowsFromTable(this.getConnection(),
|
||||||
|
targetTable, changesTable, this.updateColumnNames,
|
||||||
|
parallelizationEnabled);
|
||||||
|
|
||||||
|
OraOopOracleQueries.insertRowsIntoExportTable(this.getConnection(),
|
||||||
|
targetTable, changesTable, sysDateTime, this.mapperId,
|
||||||
|
parallelizationEnabled);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case UpdateSql:
|
||||||
|
|
||||||
|
long start = System.nanoTime();
|
||||||
|
|
||||||
|
OraOopOracleQueries.updateTable(this.getConnection(), targetTable,
|
||||||
|
changesTable, this.updateColumnNames, this
|
||||||
|
.getOracleTableColumns(), sysDateTime, this.mapperId,
|
||||||
|
parallelizationEnabled);
|
||||||
|
|
||||||
|
double timeInSec = (System.nanoTime() - start) / Math.pow(10, 9);
|
||||||
|
LOG.info(String.format("Time spent performing an update: %f sec.",
|
||||||
|
timeInSec));
|
||||||
|
break;
|
||||||
|
|
||||||
|
case MergeSql:
|
||||||
|
|
||||||
|
long mergeStart = System.nanoTime();
|
||||||
|
|
||||||
|
OraOopOracleQueries.mergeTable(this.getConnection(), targetTable,
|
||||||
|
changesTable, this.updateColumnNames, this
|
||||||
|
.getOracleTableColumns(), sysDateTime, this.mapperId,
|
||||||
|
parallelizationEnabled);
|
||||||
|
|
||||||
|
double mergeTimeInSec = (System.nanoTime() - mergeStart)
|
||||||
|
/ Math.pow(10, 9);
|
||||||
|
LOG.info(String.format("Time spent performing a merge: %f sec.",
|
||||||
|
mergeTimeInSec));
|
||||||
|
|
||||||
|
break;
|
||||||
|
|
||||||
|
default:
|
||||||
|
throw new RuntimeException(
|
||||||
|
String.format(
|
||||||
|
"Update %s to cater for the ExportTableUpdateTechnique \"%s\".",
|
||||||
|
OraOopUtilities.getCurrentMethodName(),
|
||||||
|
exportTableUpdateTechnique.toString()));
|
||||||
|
}
|
||||||
|
|
||||||
|
this.getConnection().commit();
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
this.getConnection().rollback();
|
||||||
|
throw ex;
|
||||||
|
} finally {
|
||||||
|
OraOopOracleQueries.dropTable(this.getConnection(), changesTable);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getBatchSqlStatement() {
|
||||||
|
|
||||||
|
if (sqlStatement == null) {
|
||||||
|
this.sqlStatement =
|
||||||
|
getBatchInsertSqlStatement(
|
||||||
|
this.useAppendValuesOracleHint ? "/*+APPEND_VALUES*/"
|
||||||
|
: "");
|
||||||
|
}
|
||||||
|
|
||||||
|
return this.sqlStatement;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
void configurePreparedStatement(PreparedStatement statement,
|
||||||
|
List<SqoopRecord> userRecords) throws SQLException {
|
||||||
|
|
||||||
|
Map<String, Object> fieldMap;
|
||||||
|
try {
|
||||||
|
for (SqoopRecord record : userRecords) {
|
||||||
|
fieldMap = record.getFieldMap();
|
||||||
|
|
||||||
|
boolean updateKeyValueIsNull = false;
|
||||||
|
for (int idx = 0; idx < this.updateColumnNames.length; idx++) {
|
||||||
|
String updateColumnName = this.updateColumnNames[idx];
|
||||||
|
Object updateKeyValue = fieldMap.get(updateColumnName);
|
||||||
|
if (updateKeyValue == null) {
|
||||||
|
this.numberOfRowsSkipped++;
|
||||||
|
updateKeyValueIsNull = true;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (updateKeyValueIsNull) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
configurePreparedStatementColumns(statement, fieldMap);
|
||||||
|
}
|
||||||
|
|
||||||
|
} catch (Exception ex) {
|
||||||
|
if (ex instanceof SQLException) {
|
||||||
|
throw (SQLException) ex;
|
||||||
|
} else {
|
||||||
|
LOG.error(String.format("The following error occurred during %s",
|
||||||
|
OraOopUtilities.getCurrentMethodName()), ex);
|
||||||
|
throw new SQLException(ex);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
1461
src/java/org/apache/sqoop/manager/oracle/OraOopUtilities.java
Normal file
1461
src/java/org/apache/sqoop/manager/oracle/OraOopUtilities.java
Normal file
File diff suppressed because it is too large
Load Diff
@ -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;
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,217 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle;
|
||||||
|
|
||||||
|
import java.sql.Connection;
|
||||||
|
import java.sql.DriverManager;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.sql.Statement;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Properties;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create and initialize connections to Oracle RDBMS.
|
||||||
|
*/
|
||||||
|
public class OracleConnectionFactory {
|
||||||
|
|
||||||
|
protected OracleConnectionFactory() {
|
||||||
|
}
|
||||||
|
|
||||||
|
protected static final OraOopLog LOG = OraOopLogFactory
|
||||||
|
.getLog(OracleConnectionFactory.class.getName());
|
||||||
|
|
||||||
|
public static Connection createOracleJdbcConnection(
|
||||||
|
String jdbcDriverClassName, String jdbcUrl, String username,
|
||||||
|
String password) throws SQLException {
|
||||||
|
Properties props = null;
|
||||||
|
return createOracleJdbcConnection(jdbcDriverClassName, jdbcUrl, username,
|
||||||
|
password, props);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Connection createOracleJdbcConnection(
|
||||||
|
String jdbcDriverClassName, String jdbcUrl, String username,
|
||||||
|
String password, Properties additionalProps) throws SQLException {
|
||||||
|
|
||||||
|
loadJdbcDriver(jdbcDriverClassName);
|
||||||
|
Connection connection =
|
||||||
|
createConnection(jdbcUrl, username, password, additionalProps);
|
||||||
|
|
||||||
|
// Only OraOopDBRecordReader will call initializeOracleConnection(), as
|
||||||
|
// we only need to initialize the session(s) prior to the mapper starting
|
||||||
|
// it's job.
|
||||||
|
// i.e. We don't need to initialize the sessions in order to get the
|
||||||
|
// table's data-files etc.
|
||||||
|
|
||||||
|
// initializeOracleConnection(connection, conf);
|
||||||
|
|
||||||
|
return connection;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void loadJdbcDriver(String jdbcDriverClassName) {
|
||||||
|
|
||||||
|
try {
|
||||||
|
Class.forName(jdbcDriverClassName);
|
||||||
|
} catch (ClassNotFoundException ex) {
|
||||||
|
String errorMsg =
|
||||||
|
"Unable to load the jdbc driver class : " + jdbcDriverClassName;
|
||||||
|
LOG.error(errorMsg);
|
||||||
|
throw new RuntimeException(errorMsg);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static Connection createConnection(String jdbcUrl, String username,
|
||||||
|
String password, Properties additionalProps) throws SQLException {
|
||||||
|
|
||||||
|
Properties props = new Properties();
|
||||||
|
if (username != null) {
|
||||||
|
props.put("user", username);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (password != null) {
|
||||||
|
props.put("password", password);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (additionalProps != null && additionalProps.size() > 0) {
|
||||||
|
props.putAll(additionalProps);
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
return DriverManager.getConnection(jdbcUrl, props);
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
String errorMsg = String.format(
|
||||||
|
"Unable to obtain a JDBC connection to the URL \"%s\" as user \"%s\": ",
|
||||||
|
jdbcUrl, (username != null) ? username : "[null]");
|
||||||
|
LOG.error(errorMsg, ex);
|
||||||
|
throw ex;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void initializeOracleConnection(Connection connection,
|
||||||
|
org.apache.hadoop.conf.Configuration conf) throws SQLException {
|
||||||
|
|
||||||
|
connection.setTransactionIsolation(Connection.TRANSACTION_READ_COMMITTED);
|
||||||
|
|
||||||
|
OraOopOracleQueries.setConnectionTimeZone(connection, conf);
|
||||||
|
|
||||||
|
setSessionClientInfo(connection, conf);
|
||||||
|
|
||||||
|
OraOopOracleQueries.setJdbcFetchSize(connection, conf);
|
||||||
|
|
||||||
|
executeOraOopSessionInitializationStatements(connection, conf);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected static void setSessionClientInfo(Connection connection,
|
||||||
|
org.apache.hadoop.conf.Configuration conf) {
|
||||||
|
|
||||||
|
String sql = "";
|
||||||
|
try {
|
||||||
|
sql =
|
||||||
|
"begin \n"
|
||||||
|
+ " dbms_application_info.set_module(module_name => "
|
||||||
|
+ "'%s', action_name => '%s'); \n"
|
||||||
|
+ "end;";
|
||||||
|
|
||||||
|
String oracleSessionActionName =
|
||||||
|
conf.get(OraOopConstants.ORACLE_SESSION_ACTION_NAME);
|
||||||
|
|
||||||
|
sql =
|
||||||
|
String.format(sql, OraOopConstants.ORACLE_SESSION_MODULE_NAME,
|
||||||
|
oracleSessionActionName);
|
||||||
|
|
||||||
|
Statement statement = connection.createStatement();
|
||||||
|
statement.execute(sql);
|
||||||
|
LOG.info("Initializing Oracle session with SQL :\n" + sql);
|
||||||
|
} catch (Exception ex) {
|
||||||
|
LOG.error(String.format("An error occurred while attempting to execute "
|
||||||
|
+ "the following Oracle session-initialization statement:" + "\n%s"
|
||||||
|
+ "\nError:" + "\n%s", sql, ex.getMessage()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected static void executeOraOopSessionInitializationStatements(
|
||||||
|
Connection connection, org.apache.hadoop.conf.Configuration conf) {
|
||||||
|
|
||||||
|
List<String> statements = parseOraOopSessionInitializationStatements(conf);
|
||||||
|
|
||||||
|
if (statements.size() == 0) {
|
||||||
|
LOG.warn(String
|
||||||
|
.format(
|
||||||
|
"No Oracle 'session initialization' statements were found to "
|
||||||
|
+ "execute.\nCheck that your %s and/or %s files are correctly "
|
||||||
|
+ "installed in the ${SQOOP_HOME}/conf directory.",
|
||||||
|
OraOopConstants.ORAOOP_SITE_TEMPLATE_FILENAME,
|
||||||
|
OraOopConstants.ORAOOP_SITE_FILENAME));
|
||||||
|
} else {
|
||||||
|
for (String statement : statements) {
|
||||||
|
try {
|
||||||
|
connection.createStatement().execute(statement);
|
||||||
|
LOG.info("Initializing Oracle session with SQL : " + statement);
|
||||||
|
} catch (Exception ex) {
|
||||||
|
LOG.error(String.format(
|
||||||
|
"An error occurred while attempting to execute "
|
||||||
|
+ "the following Oracle session-initialization statement:"
|
||||||
|
+ "\n%s" + "\nError:" + "\n%s", statement, ex.getMessage()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected static List<String> parseOraOopSessionInitializationStatements(
|
||||||
|
org.apache.hadoop.conf.Configuration conf) {
|
||||||
|
|
||||||
|
ArrayList<String> result = new ArrayList<String>();
|
||||||
|
|
||||||
|
if (conf == null) {
|
||||||
|
throw new IllegalArgumentException(
|
||||||
|
"No configuration argument must be specified.");
|
||||||
|
}
|
||||||
|
|
||||||
|
String sessionInitializationStatements =
|
||||||
|
conf.get(OraOopConstants.ORAOOP_SESSION_INITIALIZATION_STATEMENTS);
|
||||||
|
if (sessionInitializationStatements != null
|
||||||
|
&& !sessionInitializationStatements.isEmpty()) {
|
||||||
|
String[] initializationStatements =
|
||||||
|
sessionInitializationStatements.split(";");
|
||||||
|
for (String initializationStatement : initializationStatements) {
|
||||||
|
initializationStatement = initializationStatement.trim();
|
||||||
|
if (initializationStatement != null
|
||||||
|
&& !initializationStatement.isEmpty()
|
||||||
|
&& !initializationStatement
|
||||||
|
.startsWith(OraOopConstants.Oracle.
|
||||||
|
ORACLE_SQL_STATEMENT_COMMENT_TOKEN)) {
|
||||||
|
|
||||||
|
LOG.debug(String
|
||||||
|
.format(
|
||||||
|
"initializationStatement (quoted & pre-expression "
|
||||||
|
+ "evaluation) = \"%s\"",
|
||||||
|
initializationStatement));
|
||||||
|
|
||||||
|
initializationStatement =
|
||||||
|
OraOopUtilities.replaceConfigurationExpression(
|
||||||
|
initializationStatement, conf);
|
||||||
|
|
||||||
|
result.add(initializationStatement);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
68
src/java/org/apache/sqoop/manager/oracle/OracleTable.java
Normal file
68
src/java/org/apache/sqoop/manager/oracle/OracleTable.java
Normal file
@ -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;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,43 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle;
|
||||||
|
|
||||||
|
import java.util.Iterator;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Contains a list of Oracle columns.
|
||||||
|
*/
|
||||||
|
public class OracleTableColumns extends
|
||||||
|
OraOopGenerics.ObjectList<OracleTableColumn> {
|
||||||
|
|
||||||
|
public OracleTableColumn findColumnByName(String columnName) {
|
||||||
|
|
||||||
|
OracleTableColumn result;
|
||||||
|
|
||||||
|
Iterator<OracleTableColumn> iterator = this.iterator();
|
||||||
|
while (iterator.hasNext()) {
|
||||||
|
result = iterator.next();
|
||||||
|
if (result.getName().equals(columnName)) {
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -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;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,62 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle;
|
||||||
|
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.regex.Matcher;
|
||||||
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Contains a list of Oracle table partitions.
|
||||||
|
*/
|
||||||
|
public class OracleTablePartitions extends
|
||||||
|
OraOopGenerics.ObjectList<OracleTablePartition> {
|
||||||
|
|
||||||
|
public OracleTablePartition findPartitionByName(String partitionName) {
|
||||||
|
|
||||||
|
OracleTablePartition result;
|
||||||
|
|
||||||
|
Iterator<OracleTablePartition> iterator = this.iterator();
|
||||||
|
while (iterator.hasNext()) {
|
||||||
|
result = iterator.next();
|
||||||
|
if (result.getName().equals(partitionName)) {
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OracleTablePartition findPartitionByRegEx(String regEx) {
|
||||||
|
|
||||||
|
OracleTablePartition result;
|
||||||
|
|
||||||
|
Pattern pattern = Pattern.compile(regEx);
|
||||||
|
|
||||||
|
Iterator<OracleTablePartition> iterator = this.iterator();
|
||||||
|
while (iterator.hasNext()) {
|
||||||
|
result = iterator.next();
|
||||||
|
Matcher matcher = pattern.matcher(result.getName());
|
||||||
|
if (matcher.find()) {
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
84
src/java/org/apache/sqoop/manager/oracle/OracleVersion.java
Normal file
84
src/java/org/apache/sqoop/manager/oracle/OracleVersion.java
Normal file
@ -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;
|
||||||
|
}
|
||||||
|
}
|
@ -46,6 +46,16 @@ public final class OracleUtils {
|
|||||||
public static final String ORACLE_SECONDARY_USER_NAME = "SQOOPTEST2";
|
public static final String ORACLE_SECONDARY_USER_NAME = "SQOOPTEST2";
|
||||||
public static final String ORACLE_SECONDARY_USER_PASS = "ABCDEF";
|
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() { }
|
private OracleUtils() { }
|
||||||
|
|
||||||
public static void setOracleAuth(SqoopOptions options) {
|
public static void setOracleAuth(SqoopOptions options) {
|
||||||
|
@ -220,6 +220,8 @@ public void setUp() {
|
|||||||
manager = testServer.getManager();
|
manager = testServer.getManager();
|
||||||
} else {
|
} else {
|
||||||
Configuration conf = getConf();
|
Configuration conf = getConf();
|
||||||
|
//Need to disable OraOop for existing tests
|
||||||
|
conf.set("oraoop.disabled", "true");
|
||||||
SqoopOptions opts = getSqoopOptions(conf);
|
SqoopOptions opts = getSqoopOptions(conf);
|
||||||
opts.setConnectString(getConnectString());
|
opts.setConnectString(getConnectString());
|
||||||
opts.setTableName(getTableName());
|
opts.setTableName(getTableName());
|
||||||
|
@ -317,6 +317,8 @@ protected List<String> runExport(String [] argv) throws IOException {
|
|||||||
try {
|
try {
|
||||||
ExportTool exporter = new ExportTool();
|
ExportTool exporter = new ExportTool();
|
||||||
Configuration conf = getConf();
|
Configuration conf = getConf();
|
||||||
|
//Need to disable OraOop for existing tests
|
||||||
|
conf.set("oraoop.disabled", "true");
|
||||||
SqoopOptions opts = getSqoopOptions(conf);
|
SqoopOptions opts = getSqoopOptions(conf);
|
||||||
Sqoop sqoop = new Sqoop(exporter, conf, opts);
|
Sqoop sqoop = new Sqoop(exporter, conf, opts);
|
||||||
ret = Sqoop.runSqoop(sqoop, argv);
|
ret = Sqoop.runSqoop(sqoop, argv);
|
||||||
|
@ -115,6 +115,8 @@ protected void verifyImport(String expectedVal, String [] importCols) {
|
|||||||
removeTableDir();
|
removeTableDir();
|
||||||
|
|
||||||
Configuration conf = getConf();
|
Configuration conf = getConf();
|
||||||
|
//Need to disable OraOop for existing tests
|
||||||
|
conf.set("oraoop.disabled", "true");
|
||||||
SqoopOptions opts = getSqoopOptions(conf);
|
SqoopOptions opts = getSqoopOptions(conf);
|
||||||
|
|
||||||
// run the tool through the normal entry-point.
|
// run the tool through the normal entry-point.
|
||||||
@ -210,6 +212,8 @@ protected void runImport(SqoopTool tool, String [] argv) throws IOException {
|
|||||||
int ret;
|
int ret;
|
||||||
try {
|
try {
|
||||||
Configuration conf = getConf();
|
Configuration conf = getConf();
|
||||||
|
//Need to disable OraOop for existing tests
|
||||||
|
conf.set("oraoop.disabled", "true");
|
||||||
SqoopOptions opts = getSqoopOptions(conf);
|
SqoopOptions opts = getSqoopOptions(conf);
|
||||||
Sqoop sqoop = new Sqoop(tool, conf, opts);
|
Sqoop sqoop = new Sqoop(tool, conf, opts);
|
||||||
ret = Sqoop.runSqoop(sqoop, argv);
|
ret = Sqoop.runSqoop(sqoop, argv);
|
||||||
|
49
src/test/oraoop/create_users.sql
Normal file
49
src/test/oraoop/create_users.sql
Normal file
@ -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;
|
126
src/test/oraoop/pkg_tst_product_gen.pbk
Normal file
126
src/test/oraoop/pkg_tst_product_gen.pbk
Normal file
@ -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;
|
45
src/test/oraoop/pkg_tst_product_gen.psk
Normal file
45
src/test/oraoop/pkg_tst_product_gen.psk
Normal file
@ -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;
|
90
src/test/oraoop/table_tst_product.xml
Normal file
90
src/test/oraoop/table_tst_product.xml
Normal file
@ -0,0 +1,90 @@
|
|||||||
|
<?xml version="1.0" encoding="UTF-8"?>
|
||||||
|
<!--
|
||||||
|
Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
or more contributor license agreements. See the NOTICE file
|
||||||
|
distributed with this work for additional information
|
||||||
|
regarding copyright ownership. The ASF licenses this file
|
||||||
|
to you under the Apache License, Version 2.0 (the
|
||||||
|
"License"); you may not use this file except in compliance
|
||||||
|
with the License. You may obtain a copy of the License at
|
||||||
|
|
||||||
|
http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
|
||||||
|
Unless required by applicable law or agreed to in writing, software
|
||||||
|
distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
See the License for the specific language governing permissions and
|
||||||
|
limitations under the License.
|
||||||
|
-->
|
||||||
|
<table>
|
||||||
|
<name>TST_PRODUCT</name>
|
||||||
|
<columns>
|
||||||
|
<column>
|
||||||
|
<name>product_id</name>
|
||||||
|
<dataType>INTEGER</dataType>
|
||||||
|
<dataExpression>id</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>supplier_code</name>
|
||||||
|
<dataType>VARCHAR2 (30)</dataType>
|
||||||
|
<dataExpression>TO_CHAR (id - MOD (id, 5000),'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>product_code</name>
|
||||||
|
<dataType>VARCHAR2 (30)</dataType>
|
||||||
|
<dataExpression>TO_CHAR (MOD (id, 100000), 'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>product_descr</name>
|
||||||
|
<dataType>VARCHAR2 (255)</dataType>
|
||||||
|
<dataExpression>DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 100)))</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>product_long_descr</name>
|
||||||
|
<dataType>VARCHAR2 (4000)</dataType>
|
||||||
|
<dataExpression>DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 200)))</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>product_cost_price</name>
|
||||||
|
<dataType>NUMBER</dataType>
|
||||||
|
<dataExpression>ROUND (DBMS_RANDOM.VALUE (0, 100000), 2)</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>sell_from_date</name>
|
||||||
|
<dataType>DATE</dataType>
|
||||||
|
<dataExpression>TRUNC (SYSDATE + DBMS_RANDOM.VALUE (-365, 365))</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>sell_price</name>
|
||||||
|
<dataType>NUMBER</dataType>
|
||||||
|
<dataExpression>ROUND (DBMS_RANDOM.VALUE (0, 200000), 2)</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>create_user</name>
|
||||||
|
<dataType>VARCHAR2 (30)</dataType>
|
||||||
|
<dataExpression>DBMS_RANDOM.string ('U', 30)</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>create_time</name>
|
||||||
|
<dataType>TIMESTAMP</dataType>
|
||||||
|
<dataExpression>TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>last_update_user</name>
|
||||||
|
<dataType>VARCHAR2 (30)</dataType>
|
||||||
|
<dataExpression>DBMS_RANDOM.string ('U', 30)</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>last_update_time</name>
|
||||||
|
<dataType>TIMESTAMP</dataType>
|
||||||
|
<dataExpression>TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')</dataExpression>
|
||||||
|
</column>
|
||||||
|
</columns>
|
||||||
|
<primaryKeyColumns>
|
||||||
|
<primaryKeyColumn>product_id</primaryKeyColumn>
|
||||||
|
</primaryKeyColumns>
|
||||||
|
<uniqueKeyColumns>
|
||||||
|
<uniqueKeyColumn>supplier_code</uniqueKeyColumn>
|
||||||
|
<uniqueKeyColumn>product_code</uniqueKeyColumn>
|
||||||
|
</uniqueKeyColumns>
|
||||||
|
</table>
|
103
src/test/oraoop/table_tst_product_part.xml
Normal file
103
src/test/oraoop/table_tst_product_part.xml
Normal file
@ -0,0 +1,103 @@
|
|||||||
|
<?xml version="1.0" encoding="UTF-8"?>
|
||||||
|
<!--
|
||||||
|
Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
or more contributor license agreements. See the NOTICE file
|
||||||
|
distributed with this work for additional information
|
||||||
|
regarding copyright ownership. The ASF licenses this file
|
||||||
|
to you under the Apache License, Version 2.0 (the
|
||||||
|
"License"); you may not use this file except in compliance
|
||||||
|
with the License. You may obtain a copy of the License at
|
||||||
|
|
||||||
|
http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
|
||||||
|
Unless required by applicable law or agreed to in writing, software
|
||||||
|
distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
See the License for the specific language governing permissions and
|
||||||
|
limitations under the License.
|
||||||
|
-->
|
||||||
|
<table>
|
||||||
|
<name>TST_PRODUCT_PART</name>
|
||||||
|
<columns>
|
||||||
|
<column>
|
||||||
|
<name>product_id</name>
|
||||||
|
<dataType>INTEGER</dataType>
|
||||||
|
<dataExpression>id</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>supplier_code</name>
|
||||||
|
<dataType>VARCHAR2 (30)</dataType>
|
||||||
|
<dataExpression>TO_CHAR (id - MOD (id, 5000),'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>product_code</name>
|
||||||
|
<dataType>VARCHAR2 (30)</dataType>
|
||||||
|
<dataExpression>TO_CHAR (MOD (id, 100000), 'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>product_descr</name>
|
||||||
|
<dataType>VARCHAR2 (255)</dataType>
|
||||||
|
<dataExpression>DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 100)))</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>product_long_descr</name>
|
||||||
|
<dataType>VARCHAR2 (4000)</dataType>
|
||||||
|
<dataExpression>DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 200)))</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>product_cost_price</name>
|
||||||
|
<dataType>NUMBER</dataType>
|
||||||
|
<dataExpression>ROUND (DBMS_RANDOM.VALUE (0, 100000), 2)</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>sell_from_date</name>
|
||||||
|
<dataType>DATE</dataType>
|
||||||
|
<dataExpression>TRUNC (SYSDATE + DBMS_RANDOM.VALUE (-365, 365))</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>sell_price</name>
|
||||||
|
<dataType>NUMBER</dataType>
|
||||||
|
<dataExpression>ROUND (DBMS_RANDOM.VALUE (0, 200000), 2)</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>create_user</name>
|
||||||
|
<dataType>VARCHAR2 (30)</dataType>
|
||||||
|
<dataExpression>DBMS_RANDOM.string ('U', 30)</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>create_time</name>
|
||||||
|
<dataType>TIMESTAMP</dataType>
|
||||||
|
<dataExpression>TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>last_update_user</name>
|
||||||
|
<dataType>VARCHAR2 (30)</dataType>
|
||||||
|
<dataExpression>DBMS_RANDOM.string ('U', 30)</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>last_update_time</name>
|
||||||
|
<dataType>TIMESTAMP</dataType>
|
||||||
|
<dataExpression>TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')</dataExpression>
|
||||||
|
</column>
|
||||||
|
</columns>
|
||||||
|
<primaryKeyColumns>
|
||||||
|
<primaryKeyColumn>product_id</primaryKeyColumn>
|
||||||
|
</primaryKeyColumns>
|
||||||
|
<uniqueKeyColumns>
|
||||||
|
<uniqueKeyColumn>supplier_code</uniqueKeyColumn>
|
||||||
|
<uniqueKeyColumn>product_code</uniqueKeyColumn>
|
||||||
|
</uniqueKeyColumns>
|
||||||
|
<partitionClause>
|
||||||
|
PARTITION BY RANGE(sell_from_date)
|
||||||
|
(
|
||||||
|
PARTITION tst_product_part_1 values less than (to_date(''' || to_char(add_months(sysdate,-9),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
|
||||||
|
PARTITION tst_product_part_2 values less than (to_date(''' || to_char(add_months(sysdate,-6),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
|
||||||
|
PARTITION tst_product_part_3 values less than (to_date(''' || to_char(add_months(sysdate,-3),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
|
||||||
|
PARTITION tst_product_part_4 values less than (to_date(''' || to_char(sysdate,'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
|
||||||
|
PARTITION tst_product_part_5 values less than (to_date(''' || to_char(add_months(sysdate,3),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
|
||||||
|
PARTITION "tst_product_pa#rt_6" values less than (to_date(''' || to_char(add_months(sysdate,6),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
|
||||||
|
PARTITION "tst_product_part_7" values less than (to_date(''' || to_char(add_months(sysdate,9),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
|
||||||
|
PARTITION tst_product_part_max values less than (MAXVALUE)
|
||||||
|
)
|
||||||
|
</partitionClause>
|
||||||
|
</table>
|
90
src/test/oraoop/table_tst_product_special_chars.xml
Normal file
90
src/test/oraoop/table_tst_product_special_chars.xml
Normal file
@ -0,0 +1,90 @@
|
|||||||
|
<?xml version="1.0" encoding="UTF-8"?>
|
||||||
|
<!--
|
||||||
|
Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
or more contributor license agreements. See the NOTICE file
|
||||||
|
distributed with this work for additional information
|
||||||
|
regarding copyright ownership. The ASF licenses this file
|
||||||
|
to you under the Apache License, Version 2.0 (the
|
||||||
|
"License"); you may not use this file except in compliance
|
||||||
|
with the License. You may obtain a copy of the License at
|
||||||
|
|
||||||
|
http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
|
||||||
|
Unless required by applicable law or agreed to in writing, software
|
||||||
|
distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
See the License for the specific language governing permissions and
|
||||||
|
limitations under the License.
|
||||||
|
-->
|
||||||
|
<table>
|
||||||
|
<name>T5+_Pr#duct</name>
|
||||||
|
<columns>
|
||||||
|
<column>
|
||||||
|
<name>product_id</name>
|
||||||
|
<dataType>INTEGER</dataType>
|
||||||
|
<dataExpression>id</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>supplier_code</name>
|
||||||
|
<dataType>VARCHAR2 (30)</dataType>
|
||||||
|
<dataExpression>TO_CHAR (id - MOD (id, 5000),'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>product_code</name>
|
||||||
|
<dataType>VARCHAR2 (30)</dataType>
|
||||||
|
<dataExpression>TO_CHAR (MOD (id, 100000), 'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>product_descr</name>
|
||||||
|
<dataType>VARCHAR2 (255)</dataType>
|
||||||
|
<dataExpression>DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 100)))</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>product_long_descr</name>
|
||||||
|
<dataType>VARCHAR2 (4000)</dataType>
|
||||||
|
<dataExpression>DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 200)))</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>product_cost_price</name>
|
||||||
|
<dataType>NUMBER</dataType>
|
||||||
|
<dataExpression>ROUND (DBMS_RANDOM.VALUE (0, 100000), 2)</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>sell_from_date</name>
|
||||||
|
<dataType>DATE</dataType>
|
||||||
|
<dataExpression>TRUNC (SYSDATE + DBMS_RANDOM.VALUE (-365, 365))</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>sell_price</name>
|
||||||
|
<dataType>NUMBER</dataType>
|
||||||
|
<dataExpression>ROUND (DBMS_RANDOM.VALUE (0, 200000), 2)</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>create_user</name>
|
||||||
|
<dataType>VARCHAR2 (30)</dataType>
|
||||||
|
<dataExpression>DBMS_RANDOM.string ('U', 30)</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>create_time</name>
|
||||||
|
<dataType>TIMESTAMP</dataType>
|
||||||
|
<dataExpression>TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>last_update_user</name>
|
||||||
|
<dataType>VARCHAR2 (30)</dataType>
|
||||||
|
<dataExpression>DBMS_RANDOM.string ('U', 30)</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>last_update_time</name>
|
||||||
|
<dataType>TIMESTAMP</dataType>
|
||||||
|
<dataExpression>TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')</dataExpression>
|
||||||
|
</column>
|
||||||
|
</columns>
|
||||||
|
<primaryKeyColumns>
|
||||||
|
<primaryKeyColumn>product_id</primaryKeyColumn>
|
||||||
|
</primaryKeyColumns>
|
||||||
|
<uniqueKeyColumns>
|
||||||
|
<uniqueKeyColumn>supplier_code</uniqueKeyColumn>
|
||||||
|
<uniqueKeyColumn>product_code</uniqueKeyColumn>
|
||||||
|
</uniqueKeyColumns>
|
||||||
|
</table>
|
105
src/test/oraoop/table_tst_product_subpart.xml
Normal file
105
src/test/oraoop/table_tst_product_subpart.xml
Normal file
@ -0,0 +1,105 @@
|
|||||||
|
<?xml version="1.0" encoding="UTF-8"?>
|
||||||
|
<!--
|
||||||
|
Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
or more contributor license agreements. See the NOTICE file
|
||||||
|
distributed with this work for additional information
|
||||||
|
regarding copyright ownership. The ASF licenses this file
|
||||||
|
to you under the Apache License, Version 2.0 (the
|
||||||
|
"License"); you may not use this file except in compliance
|
||||||
|
with the License. You may obtain a copy of the License at
|
||||||
|
|
||||||
|
http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
|
||||||
|
Unless required by applicable law or agreed to in writing, software
|
||||||
|
distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
See the License for the specific language governing permissions and
|
||||||
|
limitations under the License.
|
||||||
|
-->
|
||||||
|
<table>
|
||||||
|
<name>TST_PRODUCT_SUBPART</name>
|
||||||
|
<columns>
|
||||||
|
<column>
|
||||||
|
<name>product_id</name>
|
||||||
|
<dataType>INTEGER</dataType>
|
||||||
|
<dataExpression>id</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>supplier_code</name>
|
||||||
|
<dataType>VARCHAR2 (30)</dataType>
|
||||||
|
<dataExpression>TO_CHAR (id - MOD (id, 5000),'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>product_code</name>
|
||||||
|
<dataType>VARCHAR2 (30)</dataType>
|
||||||
|
<dataExpression>TO_CHAR (MOD (id, 100000), 'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>product_descr</name>
|
||||||
|
<dataType>VARCHAR2 (255)</dataType>
|
||||||
|
<dataExpression>DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 100)))</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>product_long_descr</name>
|
||||||
|
<dataType>VARCHAR2 (4000)</dataType>
|
||||||
|
<dataExpression>DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 200)))</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>product_cost_price</name>
|
||||||
|
<dataType>NUMBER</dataType>
|
||||||
|
<dataExpression>ROUND (DBMS_RANDOM.VALUE (0, 100000), 2)</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>sell_from_date</name>
|
||||||
|
<dataType>DATE</dataType>
|
||||||
|
<dataExpression>TRUNC (SYSDATE + DBMS_RANDOM.VALUE (-365, 365))</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>sell_price</name>
|
||||||
|
<dataType>NUMBER</dataType>
|
||||||
|
<dataExpression>ROUND (DBMS_RANDOM.VALUE (0, 200000), 2)</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>create_user</name>
|
||||||
|
<dataType>VARCHAR2 (30)</dataType>
|
||||||
|
<dataExpression>DBMS_RANDOM.string ('U', 30)</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>create_time</name>
|
||||||
|
<dataType>TIMESTAMP</dataType>
|
||||||
|
<dataExpression>TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>last_update_user</name>
|
||||||
|
<dataType>VARCHAR2 (30)</dataType>
|
||||||
|
<dataExpression>DBMS_RANDOM.string ('U', 30)</dataExpression>
|
||||||
|
</column>
|
||||||
|
<column>
|
||||||
|
<name>last_update_time</name>
|
||||||
|
<dataType>TIMESTAMP</dataType>
|
||||||
|
<dataExpression>TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')</dataExpression>
|
||||||
|
</column>
|
||||||
|
</columns>
|
||||||
|
<primaryKeyColumns>
|
||||||
|
<primaryKeyColumn>product_id</primaryKeyColumn>
|
||||||
|
</primaryKeyColumns>
|
||||||
|
<uniqueKeyColumns>
|
||||||
|
<uniqueKeyColumn>supplier_code</uniqueKeyColumn>
|
||||||
|
<uniqueKeyColumn>product_code</uniqueKeyColumn>
|
||||||
|
</uniqueKeyColumns>
|
||||||
|
<partitionClause>
|
||||||
|
PARTITION BY RANGE(sell_from_date)
|
||||||
|
SUBPARTITION BY HASH(supplier_code)
|
||||||
|
SUBPARTITIONS 2
|
||||||
|
(
|
||||||
|
PARTITION tst_product_part_1 values less than (to_date(''' || to_char(add_months(sysdate,-9),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
|
||||||
|
PARTITION tst_product_part_2 values less than (to_date(''' || to_char(add_months(sysdate,-6),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
|
||||||
|
PARTITION tst_product_part_3 values less than (to_date(''' || to_char(add_months(sysdate,-3),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
|
||||||
|
PARTITION tst_product_part_4 values less than (to_date(''' || to_char(sysdate,'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
|
||||||
|
PARTITION tst_product_part_5 values less than (to_date(''' || to_char(add_months(sysdate,3),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
|
||||||
|
PARTITION tst_product_part_6 values less than (to_date(''' || to_char(add_months(sysdate,6),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
|
||||||
|
PARTITION tst_product_part_7 values less than (to_date(''' || to_char(add_months(sysdate,9),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
|
||||||
|
PARTITION tst_product_part_max values less than (MAXVALUE)
|
||||||
|
)
|
||||||
|
</partitionClause>
|
||||||
|
</table>
|
68
src/test/org/apache/sqoop/manager/oracle/ExportTest.java
Normal file
68
src/test/org/apache/sqoop/manager/oracle/ExportTest.java
Normal file
@ -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();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
241
src/test/org/apache/sqoop/manager/oracle/ImportTest.java
Normal file
241
src/test/org/apache/sqoop/manager/oracle/ImportTest.java
Normal file
@ -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();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -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();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
321
src/test/org/apache/sqoop/manager/oracle/OraOopTestCase.java
Normal file
321
src/test/org/apache/sqoop/manager/oracle/OraOopTestCase.java
Normal file
@ -0,0 +1,321 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle;
|
||||||
|
|
||||||
|
import java.io.StringWriter;
|
||||||
|
import java.sql.Connection;
|
||||||
|
import java.sql.DriverManager;
|
||||||
|
import java.sql.PreparedStatement;
|
||||||
|
import java.sql.ResultSet;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.regex.Matcher;
|
||||||
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
|
import junit.framework.Assert;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.log4j.Layout;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
import org.apache.log4j.PatternLayout;
|
||||||
|
import org.apache.log4j.WriterAppender;
|
||||||
|
|
||||||
|
import com.cloudera.sqoop.Sqoop;
|
||||||
|
import com.cloudera.sqoop.manager.OracleUtils;
|
||||||
|
|
||||||
|
import org.apache.sqoop.manager.oracle.util.HadoopFiles;
|
||||||
|
import org.apache.sqoop.manager.oracle.util.OracleData;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Base test case for OraOop to handle common functions.
|
||||||
|
*/
|
||||||
|
public abstract class OraOopTestCase {
|
||||||
|
|
||||||
|
private static final OraOopLog LOG = OraOopLogFactory.getLog(
|
||||||
|
OraOopTestCase.class.getName());
|
||||||
|
|
||||||
|
private String sqoopGenLibDirectory = System.getProperty("user.dir")
|
||||||
|
+ "/target/tmp/lib";
|
||||||
|
private String sqoopGenSrcDirectory = System.getProperty("user.dir")
|
||||||
|
+ "/target/tmp/src";
|
||||||
|
private String sqoopTargetDirectory = "target/tmp/";
|
||||||
|
private String sqoopGenClassName = "org.apache.sqoop.gen.OraOopTestClass";
|
||||||
|
|
||||||
|
private Connection conn;
|
||||||
|
|
||||||
|
protected ClassLoader classLoader;
|
||||||
|
{
|
||||||
|
classLoader = Thread.currentThread().getContextClassLoader();
|
||||||
|
if (classLoader == null) {
|
||||||
|
classLoader = OraOopTestCase.class.getClassLoader();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static {
|
||||||
|
Configuration
|
||||||
|
.addDefaultResource(OraOopConstants.ORAOOP_SITE_TEMPLATE_FILENAME);
|
||||||
|
Configuration.addDefaultResource(OraOopConstants.ORAOOP_SITE_FILENAME);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected String getSqoopTargetDirectory() {
|
||||||
|
return sqoopTargetDirectory;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void setSqoopTargetDirectory(String newSqoopTargetDirectory) {
|
||||||
|
this.sqoopTargetDirectory = newSqoopTargetDirectory;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected String getSqoopGenLibDirectory() {
|
||||||
|
return sqoopGenLibDirectory;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected String getSqoopGenSrcDirectory() {
|
||||||
|
return sqoopGenSrcDirectory;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected String getSqoopGenClassName() {
|
||||||
|
return sqoopGenClassName;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected Connection getTestEnvConnection() throws SQLException {
|
||||||
|
if (this.conn == null) {
|
||||||
|
this.conn =
|
||||||
|
DriverManager.getConnection(OracleUtils.CONNECT_STRING,
|
||||||
|
OracleUtils.ORACLE_USER_NAME, OracleUtils.ORACLE_USER_PASS);
|
||||||
|
}
|
||||||
|
return this.conn;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void closeTestEnvConnection() {
|
||||||
|
try {
|
||||||
|
if (this.conn != null) {
|
||||||
|
this.conn.close();
|
||||||
|
}
|
||||||
|
} catch (SQLException e) {
|
||||||
|
// Tried to close connection but failed - continue anyway
|
||||||
|
}
|
||||||
|
this.conn = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void createTable(String fileName) {
|
||||||
|
try {
|
||||||
|
Connection localConn = getTestEnvConnection();
|
||||||
|
int parallelProcesses = OracleData.getParallelProcesses(localConn);
|
||||||
|
int rowsPerSlave =
|
||||||
|
OracleUtils.INTEGRATIONTEST_NUM_ROWS / parallelProcesses;
|
||||||
|
try {
|
||||||
|
long startTime = System.currentTimeMillis();
|
||||||
|
OracleData.createTable(localConn, fileName, parallelProcesses,
|
||||||
|
rowsPerSlave);
|
||||||
|
LOG.debug("Created and loaded table in "
|
||||||
|
+ ((System.currentTimeMillis() - startTime) / 1000) + " seconds.");
|
||||||
|
} catch (SQLException e) {
|
||||||
|
if (e.getErrorCode() == 955) {
|
||||||
|
LOG.debug("Table already exists - using existing data");
|
||||||
|
} else {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected int countTable(String inputTableName, List<String> partitionList) {
|
||||||
|
String tableName = inputTableName;
|
||||||
|
if (tableName.startsWith("\"\"") && tableName.endsWith("\"\"")) {
|
||||||
|
// Table names may be double quoted to work around Sqoop issue
|
||||||
|
tableName = tableName.substring(1, tableName.length() - 1);
|
||||||
|
}
|
||||||
|
String sql = null;
|
||||||
|
int numRows = 0;
|
||||||
|
if (partitionList != null && partitionList.size() > 0) {
|
||||||
|
sql = "SELECT sum(cnt) FROM (";
|
||||||
|
int i = 0;
|
||||||
|
for (String partition : partitionList) {
|
||||||
|
i++;
|
||||||
|
if (i > 1) {
|
||||||
|
sql += " UNION ALL ";
|
||||||
|
}
|
||||||
|
sql +=
|
||||||
|
"SELECT count(*) cnt FROM " + tableName + " PARTITION(\""
|
||||||
|
+ partition + "\")";
|
||||||
|
}
|
||||||
|
sql += ")";
|
||||||
|
} else {
|
||||||
|
sql = "SELECT count(*) FROM " + tableName;
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
PreparedStatement stmt =
|
||||||
|
this.getTestEnvConnection().prepareStatement(sql);
|
||||||
|
stmt.execute();
|
||||||
|
ResultSet results = stmt.getResultSet();
|
||||||
|
results.next();
|
||||||
|
numRows = results.getInt(1);
|
||||||
|
} catch (SQLException e) {
|
||||||
|
throw new RuntimeException("Could not count number of rows in table "
|
||||||
|
+ tableName, e);
|
||||||
|
}
|
||||||
|
return numRows;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected Configuration getSqoopConf() {
|
||||||
|
Configuration sqoopConf = new Configuration();
|
||||||
|
return sqoopConf;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected int runImport(String tableName, Configuration sqoopConf,
|
||||||
|
boolean sequenceFile) {
|
||||||
|
Logger rootLogger = Logger.getRootLogger();
|
||||||
|
rootLogger.removeAllAppenders();
|
||||||
|
StringWriter stringWriter = new StringWriter();
|
||||||
|
Layout layout = new PatternLayout("%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n");
|
||||||
|
WriterAppender writerAppender = new WriterAppender(layout, stringWriter);
|
||||||
|
rootLogger.addAppender(writerAppender);
|
||||||
|
|
||||||
|
List<String> sqoopArgs = new ArrayList<String>();
|
||||||
|
|
||||||
|
sqoopArgs.add("import");
|
||||||
|
|
||||||
|
if (sequenceFile) {
|
||||||
|
sqoopArgs.add("--as-sequencefile");
|
||||||
|
}
|
||||||
|
|
||||||
|
sqoopArgs.add("--connect");
|
||||||
|
sqoopArgs.add(OracleUtils.CONNECT_STRING);
|
||||||
|
|
||||||
|
sqoopArgs.add("--username");
|
||||||
|
sqoopArgs.add(OracleUtils.ORACLE_USER_NAME);
|
||||||
|
|
||||||
|
sqoopArgs.add("--password");
|
||||||
|
sqoopArgs.add(OracleUtils.ORACLE_USER_PASS);
|
||||||
|
|
||||||
|
sqoopArgs.add("--table");
|
||||||
|
sqoopArgs.add(tableName);
|
||||||
|
|
||||||
|
sqoopArgs.add("--target-dir");
|
||||||
|
sqoopArgs.add(this.sqoopTargetDirectory);
|
||||||
|
|
||||||
|
sqoopArgs.add("--class-name");
|
||||||
|
sqoopArgs.add(getSqoopGenClassName());
|
||||||
|
|
||||||
|
sqoopArgs.add("--bindir");
|
||||||
|
sqoopArgs.add(this.sqoopGenLibDirectory);
|
||||||
|
|
||||||
|
sqoopArgs.add("--outdir");
|
||||||
|
sqoopArgs.add(this.sqoopGenSrcDirectory);
|
||||||
|
|
||||||
|
if (OracleUtils.NUM_MAPPERS != 0) {
|
||||||
|
sqoopArgs.add("--num-mappers");
|
||||||
|
sqoopArgs.add(Integer.toString(OracleUtils.NUM_MAPPERS));
|
||||||
|
}
|
||||||
|
|
||||||
|
int rowsInTable =
|
||||||
|
countTable(tableName, OraOopUtilities.splitOracleStringList(sqoopConf
|
||||||
|
.get(OraOopConstants.ORAOOP_IMPORT_PARTITION_LIST)));
|
||||||
|
|
||||||
|
int retCode =
|
||||||
|
Sqoop.runTool(sqoopArgs.toArray(new String[sqoopArgs.size()]),
|
||||||
|
sqoopConf);
|
||||||
|
int rowsImported = 0;
|
||||||
|
if (retCode == 0) {
|
||||||
|
String logString = stringWriter.toString();
|
||||||
|
Pattern pattern =
|
||||||
|
Pattern.compile(
|
||||||
|
"(INFO mapreduce.ImportJobBase: Retrieved )([0-9]+)( records.)");
|
||||||
|
Matcher matcher = pattern.matcher(logString);
|
||||||
|
while (matcher.find()) {
|
||||||
|
rowsImported = Integer.parseInt(matcher.group(2));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (retCode != 0 || rowsInTable != rowsImported) {
|
||||||
|
System.out.println(stringWriter.toString());
|
||||||
|
}
|
||||||
|
Assert.assertEquals("Incorrect number of rows imported", rowsInTable,
|
||||||
|
rowsImported);
|
||||||
|
return retCode;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected int runExportFromTemplateTable(String templateTableName,
|
||||||
|
String tableName) {
|
||||||
|
List<String> sqoopArgs = new ArrayList<String>();
|
||||||
|
|
||||||
|
sqoopArgs.add("export");
|
||||||
|
|
||||||
|
sqoopArgs.add("--connect");
|
||||||
|
sqoopArgs.add(OracleUtils.CONNECT_STRING);
|
||||||
|
|
||||||
|
sqoopArgs.add("--username");
|
||||||
|
sqoopArgs.add(OracleUtils.ORACLE_USER_NAME);
|
||||||
|
|
||||||
|
sqoopArgs.add("--password");
|
||||||
|
sqoopArgs.add(OracleUtils.ORACLE_USER_PASS);
|
||||||
|
|
||||||
|
sqoopArgs.add("--table");
|
||||||
|
sqoopArgs.add(tableName);
|
||||||
|
|
||||||
|
sqoopArgs.add("--export-dir");
|
||||||
|
sqoopArgs.add(this.sqoopTargetDirectory);
|
||||||
|
|
||||||
|
sqoopArgs.add("--class-name");
|
||||||
|
sqoopArgs.add(getSqoopGenClassName());
|
||||||
|
|
||||||
|
sqoopArgs.add("--bindir");
|
||||||
|
sqoopArgs.add(this.sqoopGenLibDirectory);
|
||||||
|
|
||||||
|
sqoopArgs.add("--outdir");
|
||||||
|
sqoopArgs.add(this.sqoopGenSrcDirectory);
|
||||||
|
|
||||||
|
Configuration sqoopConf = getSqoopConf();
|
||||||
|
|
||||||
|
sqoopConf.set("oraoop.template.table", templateTableName);
|
||||||
|
sqoopConf.setBoolean("oraoop.drop.table", true);
|
||||||
|
sqoopConf.setBoolean("oraoop.nologging", true);
|
||||||
|
sqoopConf.setBoolean("oraoop.partitioned", false);
|
||||||
|
|
||||||
|
return Sqoop.runTool(sqoopArgs.toArray(new String[sqoopArgs.size()]),
|
||||||
|
sqoopConf);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected int runCompareTables(Connection connection, String table1,
|
||||||
|
String table2) throws SQLException {
|
||||||
|
PreparedStatement stmt;
|
||||||
|
stmt = connection.prepareStatement(
|
||||||
|
"select count(*) from (select * from (select * from "
|
||||||
|
+ table1
|
||||||
|
+ " minus select * from "
|
||||||
|
+ table2
|
||||||
|
+ ") union all select * from (select * from "
|
||||||
|
+ table2
|
||||||
|
+ " minus select * from " + table1 + "))");
|
||||||
|
ResultSet results = stmt.executeQuery();
|
||||||
|
results.next();
|
||||||
|
int numDifferences = results.getInt(1);
|
||||||
|
return numDifferences;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void cleanupFolders() throws Exception {
|
||||||
|
HadoopFiles.delete(new Path(getSqoopTargetDirectory()), true);
|
||||||
|
HadoopFiles.delete(new Path(getSqoopGenSrcDirectory()), true);
|
||||||
|
HadoopFiles.delete(new Path(getSqoopGenLibDirectory()), true);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -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) ";
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,520 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle;
|
||||||
|
|
||||||
|
import static org.junit.Assert.*;
|
||||||
|
|
||||||
|
import java.sql.Connection;
|
||||||
|
import java.sql.PreparedStatement;
|
||||||
|
import java.sql.ResultSet;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.text.SimpleDateFormat;
|
||||||
|
import java.util.Date;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import com.cloudera.sqoop.manager.OracleUtils;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test OracleConnectionFactory class including initialization statements.
|
||||||
|
*/
|
||||||
|
public class OracleConnectionFactoryTest extends OraOopTestCase {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This is just to expose methods in OracleConnectionFactory.
|
||||||
|
*/
|
||||||
|
public class Exposer extends OracleConnectionFactory {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSetJdbcFetchSize() {
|
||||||
|
setAndCheckJdbcFetchSize(45);
|
||||||
|
setAndCheckJdbcFetchSize(2000);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void setAndCheckJdbcFetchSize(int jdbcFetchSize) {
|
||||||
|
|
||||||
|
try {
|
||||||
|
Connection conn = getConnection();
|
||||||
|
|
||||||
|
String uniqueJunk =
|
||||||
|
(new SimpleDateFormat("yyyyMMddHHmmsszzz")).format(new Date())
|
||||||
|
+ jdbcFetchSize;
|
||||||
|
|
||||||
|
org.apache.hadoop.conf.Configuration conf = new Configuration();
|
||||||
|
conf.setInt(OraOopConstants.ORACLE_ROW_FETCH_SIZE, jdbcFetchSize);
|
||||||
|
|
||||||
|
// Prevent setJdbcFetchSize() from logging information about the
|
||||||
|
// fetch-size
|
||||||
|
// changing. Otherwise, the junit output will be polluted with messages
|
||||||
|
// about
|
||||||
|
// things that aren't actually a problem...
|
||||||
|
boolean logIsBeingCached =
|
||||||
|
OracleConnectionFactory.LOG.getCacheLogEntries();
|
||||||
|
OracleConnectionFactory.LOG.setCacheLogEntries(true);
|
||||||
|
|
||||||
|
OraOopOracleQueries.setJdbcFetchSize(conn, conf);
|
||||||
|
|
||||||
|
OracleConnectionFactory.LOG.setCacheLogEntries(logIsBeingCached);
|
||||||
|
|
||||||
|
String uniqueSql =
|
||||||
|
String.format("select /*%s*/ * from dba_objects", uniqueJunk);
|
||||||
|
// Usually dba_objects will have a lot of rows
|
||||||
|
ResultSet resultSet1 = conn.createStatement().executeQuery(uniqueSql);
|
||||||
|
while (resultSet1.next()) {
|
||||||
|
// Nothing to do
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
ResultSet resultSet2 =
|
||||||
|
conn.createStatement().executeQuery(OraOopTestConstants.SQL_TABLE);
|
||||||
|
boolean sqlFound = false;
|
||||||
|
double rowsPerFetch = 0;
|
||||||
|
while (resultSet2.next()) {
|
||||||
|
String sqlText = resultSet2.getString("SQL_TEXT");
|
||||||
|
if (sqlText.contains(uniqueJunk)) {
|
||||||
|
sqlFound = true;
|
||||||
|
rowsPerFetch = resultSet2.getDouble("ROWS_PER_FETCH");
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!sqlFound) {
|
||||||
|
Assert
|
||||||
|
.fail("Unable to find the performance metrics for the SQL "
|
||||||
|
+ "statement being used to check the JDBC fetch size.");
|
||||||
|
}
|
||||||
|
|
||||||
|
if (rowsPerFetch < jdbcFetchSize * 0.95
|
||||||
|
|| rowsPerFetch > jdbcFetchSize * 1.05) {
|
||||||
|
Assert
|
||||||
|
.fail(String
|
||||||
|
.format(
|
||||||
|
"The measured JDBC fetch size is not within 5%% of what we "
|
||||||
|
+ "expected. Expected=%s rows/fetch, actual=%s rows/fetch",
|
||||||
|
jdbcFetchSize, rowsPerFetch));
|
||||||
|
}
|
||||||
|
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
Assert.fail(ex.getMessage());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCreateOracleJdbcConnectionBadUserName() {
|
||||||
|
|
||||||
|
try {
|
||||||
|
|
||||||
|
// Prevent createOracleJdbcConnection() from logging a problem with the
|
||||||
|
// bad username we're about to use. Otherwise, the junit output will be
|
||||||
|
// polluted with messages about things that aren't actually a problem...
|
||||||
|
boolean logIsBeingCached =
|
||||||
|
OracleConnectionFactory.LOG.getCacheLogEntries();
|
||||||
|
OracleConnectionFactory.LOG.setCacheLogEntries(true);
|
||||||
|
|
||||||
|
OracleConnectionFactory.createOracleJdbcConnection(
|
||||||
|
OraOopConstants.ORACLE_JDBC_DRIVER_CLASS, OracleUtils.CONNECT_STRING,
|
||||||
|
OracleUtils.ORACLE_INVALID_USER_NAME, OracleUtils.ORACLE_USER_PASS);
|
||||||
|
|
||||||
|
OracleConnectionFactory.LOG.setCacheLogEntries(logIsBeingCached);
|
||||||
|
|
||||||
|
Assert
|
||||||
|
.fail("OracleConnectionFactory should have thrown an exception in "
|
||||||
|
+ "response to a rubbish user name.");
|
||||||
|
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
assertEquals(ex.getErrorCode(), 1017); // <- ORA-01017 invalid
|
||||||
|
// username/password; logon denied.
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCreateOracleJdbcConnectionBadPassword() {
|
||||||
|
|
||||||
|
try {
|
||||||
|
// Prevent createOracleJdbcConnection() from logging a problem with the
|
||||||
|
// bad username we're about to use. Otherwise, the junit output will be
|
||||||
|
// polluted with messages about things that aren't actually a problem...
|
||||||
|
boolean logIsBeingCached =
|
||||||
|
OracleConnectionFactory.LOG.getCacheLogEntries();
|
||||||
|
OracleConnectionFactory.LOG.setCacheLogEntries(true);
|
||||||
|
|
||||||
|
OracleConnectionFactory.createOracleJdbcConnection(
|
||||||
|
OraOopConstants.ORACLE_JDBC_DRIVER_CLASS, OracleUtils.CONNECT_STRING,
|
||||||
|
OracleUtils.ORACLE_USER_NAME, "a" + OracleUtils.ORACLE_USER_PASS);
|
||||||
|
|
||||||
|
OracleConnectionFactory.LOG.setCacheLogEntries(logIsBeingCached);
|
||||||
|
|
||||||
|
Assert
|
||||||
|
.fail("OracleConnectionFactory should have thrown an exception in "
|
||||||
|
+ "response to a rubbish password.");
|
||||||
|
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
assertEquals(ex.getErrorCode(), 1017); // <- ORA-01017 invalid
|
||||||
|
// username/password; logon denied.
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCreateOracleJdbcConnectionOk() {
|
||||||
|
|
||||||
|
try {
|
||||||
|
Connection conn = getConnection();
|
||||||
|
|
||||||
|
assertEquals(
|
||||||
|
"The connection to the Oracle database does not appear to be valid.",
|
||||||
|
true, conn.isValid(15));
|
||||||
|
|
||||||
|
ResultSet resultSet =
|
||||||
|
conn.createStatement().executeQuery(
|
||||||
|
"select instance_name from v$instance");
|
||||||
|
if (!resultSet.next() || resultSet.getString(1).isEmpty()) {
|
||||||
|
Assert.fail("Got blank instance name from v$instance");
|
||||||
|
}
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
Assert.fail(ex.getMessage());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testExecuteOraOopSessionInitializationStatements() {
|
||||||
|
|
||||||
|
// Exposer.LOG = null;
|
||||||
|
// protected static final Log LOG =
|
||||||
|
// LogFactory.getLog(OracleConnectionFactory.class.getName());
|
||||||
|
|
||||||
|
OraOopLogFactory.OraOopLog2 oraoopLog = Exposer.LOG;
|
||||||
|
|
||||||
|
oraoopLog.setCacheLogEntries(true);
|
||||||
|
|
||||||
|
// Check that the default session-initialization statements are reflected in
|
||||||
|
// the log...
|
||||||
|
oraoopLog.clearCache();
|
||||||
|
checkExecuteOraOopSessionInitializationStatements(null);
|
||||||
|
checkLogContainsText(oraoopLog,
|
||||||
|
"Initializing Oracle session with SQL : alter session disable "
|
||||||
|
+ "parallel query");
|
||||||
|
checkLogContainsText(
|
||||||
|
oraoopLog,
|
||||||
|
"Initializing Oracle session with SQL : alter session set "
|
||||||
|
+ "\"_serial_direct_read\"=true");
|
||||||
|
|
||||||
|
// Check that the absence of session-initialization statements is reflected
|
||||||
|
// in the log...
|
||||||
|
oraoopLog.clearCache();
|
||||||
|
checkExecuteOraOopSessionInitializationStatements("");
|
||||||
|
checkLogContainsText(oraoopLog,
|
||||||
|
"No Oracle 'session initialization' statements were found to execute");
|
||||||
|
|
||||||
|
// This should do nothing (i.e. not throw an exception)...
|
||||||
|
checkExecuteOraOopSessionInitializationStatements(";");
|
||||||
|
|
||||||
|
// This should throw an exception, as Oracle won't know what to do with
|
||||||
|
// this...
|
||||||
|
oraoopLog.clearCache();
|
||||||
|
checkExecuteOraOopSessionInitializationStatements("loremipsum");
|
||||||
|
checkLogContainsText(oraoopLog, "loremipsum");
|
||||||
|
checkLogContainsText(oraoopLog, "ORA-00900: invalid SQL statement");
|
||||||
|
|
||||||
|
Connection conn = getConnection();
|
||||||
|
try {
|
||||||
|
|
||||||
|
// Try a session-initialization statement that creates a table...
|
||||||
|
dropTable(conn, OracleUtils.SYSTEMTEST_TABLE_NAME);
|
||||||
|
checkExecuteOraOopSessionInitializationStatements("create table "
|
||||||
|
+ OracleUtils.SYSTEMTEST_TABLE_NAME + " (col1 varchar2(1))");
|
||||||
|
if (!doesTableExist(conn, OracleUtils.SYSTEMTEST_TABLE_NAME)) {
|
||||||
|
Assert.fail("The session-initialization statement to create the table "
|
||||||
|
+ OracleUtils.SYSTEMTEST_TABLE_NAME + " did not work.");
|
||||||
|
}
|
||||||
|
|
||||||
|
// Try a sequence of a few statements...
|
||||||
|
dropTable(conn, OracleUtils.SYSTEMTEST_TABLE_NAME);
|
||||||
|
checkExecuteOraOopSessionInitializationStatements("create table "
|
||||||
|
+ OracleUtils.SYSTEMTEST_TABLE_NAME + " (col1 number);insert into "
|
||||||
|
+ OracleUtils.SYSTEMTEST_TABLE_NAME + " values (1) ; --update "
|
||||||
|
+ OracleUtils.SYSTEMTEST_TABLE_NAME + " set col1 = col1 + 1; update "
|
||||||
|
+ OracleUtils.SYSTEMTEST_TABLE_NAME
|
||||||
|
+ " set col1 = col1 + 1; commit ;;");
|
||||||
|
|
||||||
|
ResultSet resultSet =
|
||||||
|
conn.createStatement().executeQuery(
|
||||||
|
"select col1 from " + OracleUtils.SYSTEMTEST_TABLE_NAME);
|
||||||
|
resultSet.next();
|
||||||
|
int actualValue = resultSet.getInt("col1");
|
||||||
|
if (actualValue != 2) {
|
||||||
|
Assert.fail("The table " + OracleUtils.SYSTEMTEST_TABLE_NAME
|
||||||
|
+ " does not contain the data we expected.");
|
||||||
|
}
|
||||||
|
|
||||||
|
dropTable(conn, OracleUtils.SYSTEMTEST_TABLE_NAME);
|
||||||
|
|
||||||
|
} catch (Exception ex) {
|
||||||
|
Assert.fail(ex.getMessage());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testParseOraOopSessionInitializationStatements() {
|
||||||
|
|
||||||
|
List<String> statements = null;
|
||||||
|
|
||||||
|
try {
|
||||||
|
statements =
|
||||||
|
OracleConnectionFactory
|
||||||
|
.parseOraOopSessionInitializationStatements(null);
|
||||||
|
Assert.fail("An IllegalArgumentException should have been thrown.");
|
||||||
|
} catch (IllegalArgumentException ex) {
|
||||||
|
// This is what we wanted.
|
||||||
|
}
|
||||||
|
|
||||||
|
org.apache.hadoop.conf.Configuration conf = new Configuration();
|
||||||
|
|
||||||
|
statements =
|
||||||
|
OracleConnectionFactory
|
||||||
|
.parseOraOopSessionInitializationStatements(conf);
|
||||||
|
Assert.assertTrue(statements.size() > 0);
|
||||||
|
|
||||||
|
conf.set(OraOopConstants.ORAOOP_SESSION_INITIALIZATION_STATEMENTS, "");
|
||||||
|
statements =
|
||||||
|
OracleConnectionFactory
|
||||||
|
.parseOraOopSessionInitializationStatements(conf);
|
||||||
|
Assert.assertEquals(0, statements.size());
|
||||||
|
|
||||||
|
conf.set(OraOopConstants.ORAOOP_SESSION_INITIALIZATION_STATEMENTS, ";");
|
||||||
|
statements =
|
||||||
|
OracleConnectionFactory
|
||||||
|
.parseOraOopSessionInitializationStatements(conf);
|
||||||
|
Assert.assertEquals(0, statements.size());
|
||||||
|
|
||||||
|
conf.set(OraOopConstants.ORAOOP_SESSION_INITIALIZATION_STATEMENTS,
|
||||||
|
";--;\t--");
|
||||||
|
statements =
|
||||||
|
OracleConnectionFactory
|
||||||
|
.parseOraOopSessionInitializationStatements(conf);
|
||||||
|
Assert.assertEquals(0, statements.size());
|
||||||
|
|
||||||
|
conf.set(OraOopConstants.ORAOOP_SESSION_INITIALIZATION_STATEMENTS
|
||||||
|
, "\ta");
|
||||||
|
statements =
|
||||||
|
OracleConnectionFactory
|
||||||
|
.parseOraOopSessionInitializationStatements(conf);
|
||||||
|
Assert.assertEquals(1, statements.size());
|
||||||
|
if (!statements.get(0).equalsIgnoreCase("a")) {
|
||||||
|
Assert.fail("Expected a session initialization statement of \"a\"");
|
||||||
|
}
|
||||||
|
|
||||||
|
conf.set(OraOopConstants.ORAOOP_SESSION_INITIALIZATION_STATEMENTS,
|
||||||
|
"a;b;--c;d;");
|
||||||
|
statements =
|
||||||
|
OracleConnectionFactory
|
||||||
|
.parseOraOopSessionInitializationStatements(conf);
|
||||||
|
Assert.assertEquals(3, statements.size());
|
||||||
|
if (!statements.get(0).equalsIgnoreCase("a")) {
|
||||||
|
Assert.fail("Expected a session initialization statement of \"a\"");
|
||||||
|
}
|
||||||
|
if (!statements.get(1).equalsIgnoreCase("b")) {
|
||||||
|
Assert.fail("Expected a session initialization statement of \"b\"");
|
||||||
|
}
|
||||||
|
if (!statements.get(2).equalsIgnoreCase("d")) {
|
||||||
|
Assert.fail("Expected a session initialization statement of \"d\"");
|
||||||
|
}
|
||||||
|
|
||||||
|
// Expressions without default values...
|
||||||
|
conf.set(OraOopConstants.ORAOOP_SESSION_INITIALIZATION_STATEMENTS,
|
||||||
|
"set a={expr1};b={expr2}/{expr3};");
|
||||||
|
conf.set("expr1", "1");
|
||||||
|
conf.set("expr2", "2");
|
||||||
|
conf.set("expr3", "3");
|
||||||
|
statements =
|
||||||
|
OracleConnectionFactory
|
||||||
|
.parseOraOopSessionInitializationStatements(conf);
|
||||||
|
Assert.assertEquals(2, statements.size());
|
||||||
|
String actual = statements.get(0);
|
||||||
|
String expected = "set a=1";
|
||||||
|
if (!actual.equalsIgnoreCase(expected)) {
|
||||||
|
Assert.fail(String.format(
|
||||||
|
"Expected a session initialization statement of \"%s\", but got \"%s\"."
|
||||||
|
, expected, actual));
|
||||||
|
}
|
||||||
|
actual = statements.get(1);
|
||||||
|
expected = "b=2/3";
|
||||||
|
if (!actual.equalsIgnoreCase(expected)) {
|
||||||
|
Assert.fail(String.format(
|
||||||
|
"Expected a session initialization statement of \"%s\", but got \"%s\"."
|
||||||
|
, expected, actual));
|
||||||
|
}
|
||||||
|
|
||||||
|
// Expressions with default values...
|
||||||
|
conf.set(OraOopConstants.ORAOOP_SESSION_INITIALIZATION_STATEMENTS,
|
||||||
|
"set c={expr3|66};d={expr4|15}/{expr5|90};");
|
||||||
|
conf.set("expr3", "20");
|
||||||
|
// conf.set("expr4", "21");
|
||||||
|
// conf.set("expr5", "23");
|
||||||
|
statements =
|
||||||
|
OracleConnectionFactory
|
||||||
|
.parseOraOopSessionInitializationStatements(conf);
|
||||||
|
Assert.assertEquals(2, statements.size());
|
||||||
|
actual = statements.get(0);
|
||||||
|
expected = "set c=20";
|
||||||
|
if (!actual.equalsIgnoreCase(expected)) {
|
||||||
|
Assert.fail(String.format(
|
||||||
|
"Expected a session initialization statement of \"%s\", but got \"%s\"."
|
||||||
|
, expected, actual));
|
||||||
|
}
|
||||||
|
actual = statements.get(1);
|
||||||
|
expected = "d=15/90";
|
||||||
|
if (!actual.equalsIgnoreCase(expected)) {
|
||||||
|
Assert.fail(String.format(
|
||||||
|
"Expected a session initialization statement of \"%s\", but got \"%s\"."
|
||||||
|
, expected, actual));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
private void dropTable(Connection conn, String tableName) {
|
||||||
|
|
||||||
|
try {
|
||||||
|
conn.createStatement().executeQuery("drop table " + tableName);
|
||||||
|
|
||||||
|
if (doesTableExist(conn, tableName)) {
|
||||||
|
Assert.fail("Unable to drop the table " + tableName);
|
||||||
|
}
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
if (ex.getErrorCode() != 942) { // <- Table or view does not exist
|
||||||
|
Assert.fail(ex.getMessage());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean doesTableExist(Connection conn, String tableName) {
|
||||||
|
|
||||||
|
boolean result = false;
|
||||||
|
try {
|
||||||
|
List<OracleTable> tables = OraOopOracleQueries.getTables(conn);
|
||||||
|
|
||||||
|
for (int idx = 0; idx < tables.size(); idx++) {
|
||||||
|
if (tables.get(idx).getName().equalsIgnoreCase(tableName)) {
|
||||||
|
result = true;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
Assert.fail(ex.getMessage());
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void checkLogContainsText(OraOopLogFactory.OraOopLog2 oraoopLog,
|
||||||
|
String text) {
|
||||||
|
|
||||||
|
if (!oraoopLog.getLogEntries().toLowerCase().contains(text.toLowerCase())) {
|
||||||
|
Assert.fail(
|
||||||
|
"The LOG does not contain the following text (when it should):\n\t"
|
||||||
|
+ text);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void checkExecuteOraOopSessionInitializationStatements(
|
||||||
|
String statements) {
|
||||||
|
|
||||||
|
Connection conn = getConnection();
|
||||||
|
|
||||||
|
org.apache.hadoop.conf.Configuration conf = new Configuration();
|
||||||
|
if (statements != null) {
|
||||||
|
conf.set(OraOopConstants.ORAOOP_SESSION_INITIALIZATION_STATEMENTS,
|
||||||
|
statements);
|
||||||
|
}
|
||||||
|
|
||||||
|
Exposer.executeOraOopSessionInitializationStatements(conn, conf);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSetSessionClientInfo() {
|
||||||
|
|
||||||
|
Connection conn = getConnection();
|
||||||
|
|
||||||
|
org.apache.hadoop.conf.Configuration conf = new Configuration();
|
||||||
|
|
||||||
|
String moduleName = OraOopConstants.ORACLE_SESSION_MODULE_NAME;
|
||||||
|
String actionName =
|
||||||
|
(new SimpleDateFormat("yyyyMMddHHmmsszzz")).format(new Date());
|
||||||
|
|
||||||
|
conf.set(OraOopConstants.ORACLE_SESSION_ACTION_NAME, actionName);
|
||||||
|
|
||||||
|
try {
|
||||||
|
PreparedStatement statement =
|
||||||
|
conn.prepareStatement("select process, module, action "
|
||||||
|
+ "from v$session " + "where module = ? and action = ?");
|
||||||
|
statement.setString(1, moduleName);
|
||||||
|
statement.setString(2, actionName);
|
||||||
|
|
||||||
|
// Check no session have this action name - because we haven't applied to
|
||||||
|
// our session yet...
|
||||||
|
ResultSet resultSet = statement.executeQuery();
|
||||||
|
if (resultSet.next()) {
|
||||||
|
Assert
|
||||||
|
.fail("There should be no Oracle sessions with an action name of "
|
||||||
|
+ actionName);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Apply this action name to our session...
|
||||||
|
OracleConnectionFactory.setSessionClientInfo(conn, conf);
|
||||||
|
|
||||||
|
// Now check there is a session with our action name...
|
||||||
|
int sessionFoundCount = 0;
|
||||||
|
resultSet = statement.executeQuery();
|
||||||
|
while (resultSet.next()) {
|
||||||
|
sessionFoundCount++;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (sessionFoundCount < 1) {
|
||||||
|
Assert
|
||||||
|
.fail("Unable to locate an Oracle session with the expected module "
|
||||||
|
+ "and action.");
|
||||||
|
}
|
||||||
|
|
||||||
|
if (sessionFoundCount > 1) {
|
||||||
|
Assert
|
||||||
|
.fail("Multiple sessions were found with the expected module and "
|
||||||
|
+ "action - we only expected to find one.");
|
||||||
|
}
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
Assert.fail(ex.getMessage());
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
private Connection getConnection() {
|
||||||
|
|
||||||
|
try {
|
||||||
|
return OracleConnectionFactory.createOracleJdbcConnection(
|
||||||
|
OraOopConstants.ORACLE_JDBC_DRIVER_CLASS, OracleUtils.CONNECT_STRING,
|
||||||
|
OracleUtils.ORACLE_USER_NAME, OracleUtils.ORACLE_USER_PASS);
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
Assert.fail(ex.getMessage());
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
315
src/test/org/apache/sqoop/manager/oracle/SystemImportTest.java
Normal file
315
src/test/org/apache/sqoop/manager/oracle/SystemImportTest.java
Normal file
@ -0,0 +1,315 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle;
|
||||||
|
|
||||||
|
import static org.junit.Assert.*;
|
||||||
|
|
||||||
|
import java.lang.reflect.Method;
|
||||||
|
import java.math.BigDecimal;
|
||||||
|
import java.net.URL;
|
||||||
|
import java.net.URLClassLoader;
|
||||||
|
import java.sql.Blob;
|
||||||
|
import java.sql.Clob;
|
||||||
|
import java.sql.Connection;
|
||||||
|
import java.sql.PreparedStatement;
|
||||||
|
import java.sql.ResultSet;
|
||||||
|
import java.sql.Statement;
|
||||||
|
import java.sql.Struct;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.io.BytesWritable;
|
||||||
|
import org.apache.hadoop.io.LongWritable;
|
||||||
|
import org.apache.hadoop.io.SequenceFile;
|
||||||
|
import org.apache.sqoop.manager.oracle.util.*;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import com.cloudera.sqoop.lib.BlobRef;
|
||||||
|
import com.cloudera.sqoop.lib.ClobRef;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.manager.OracleUtils;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* OraOop system tests of importing data from oracle to hadoop.
|
||||||
|
*/
|
||||||
|
public class SystemImportTest extends OraOopTestCase {
|
||||||
|
|
||||||
|
private static Class<?> preparedStatementClass;
|
||||||
|
private static Method methSetBinaryDouble;
|
||||||
|
private static Method methSetBinaryFloat;
|
||||||
|
|
||||||
|
static {
|
||||||
|
try {
|
||||||
|
preparedStatementClass =
|
||||||
|
Class.forName("oracle.jdbc.OraclePreparedStatement");
|
||||||
|
methSetBinaryDouble =
|
||||||
|
preparedStatementClass.getMethod("setBinaryDouble", int.class,
|
||||||
|
double.class);
|
||||||
|
methSetBinaryFloat =
|
||||||
|
preparedStatementClass.getMethod("setBinaryFloat", int.class,
|
||||||
|
float.class);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new RuntimeException(
|
||||||
|
"Problem getting Oracle JDBC methods via reflection.", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generates pseudo-random test data across all supported data types in an
|
||||||
|
* Oracle database. Imports the data into Hadoop and compares with the data in
|
||||||
|
* Oracle.
|
||||||
|
*
|
||||||
|
* @throws Exception
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void importTest() throws Exception {
|
||||||
|
// Generate test data in oracle
|
||||||
|
setSqoopTargetDirectory(getSqoopTargetDirectory()
|
||||||
|
+ OracleUtils.SYSTEMTEST_TABLE_NAME);
|
||||||
|
int numRows = OracleUtils.SYSTEMTEST_NUM_ROWS;
|
||||||
|
Connection conn = getTestEnvConnection();
|
||||||
|
OraOopOracleQueries.setConnectionTimeZone(conn, "GMT");
|
||||||
|
try {
|
||||||
|
Statement s = conn.createStatement();
|
||||||
|
try {
|
||||||
|
s.executeUpdate("CREATE TABLE "
|
||||||
|
+ OracleUtils.SYSTEMTEST_TABLE_NAME
|
||||||
|
+ " (id NUMBER(10) PRIMARY KEY, bd BINARY_DOUBLE, bf BINARY_FLOAT, "
|
||||||
|
+ "b BLOB, c CHAR(12), cl CLOB, d DATE, "
|
||||||
|
+ "f FLOAT(126), l LONG, nc NCHAR(30), ncl NCLOB, n NUMBER(9,2), "
|
||||||
|
+ "nvc NVARCHAR2(30), r ROWID, u URITYPE, iym INTERVAL YEAR(2) TO "
|
||||||
|
+ "MONTH, ids INTERVAL DAY(2) TO SECOND(6), "
|
||||||
|
+ "t TIMESTAMP(6), tz TIMESTAMP(6) WITH TIME ZONE, "
|
||||||
|
+ "tltz TIMESTAMP(6) WITH LOCAL TIME ZONE, rawcol RAW(21))");
|
||||||
|
BinaryDoubleGenerator bdg = new BinaryDoubleGenerator();
|
||||||
|
BinaryFloatGenerator bfg = new BinaryFloatGenerator();
|
||||||
|
BlobGenerator bg = new BlobGenerator(conn, 2 * 1024, 8 * 1024);
|
||||||
|
CharGenerator cg = new CharGenerator(12, 12);
|
||||||
|
CharGenerator clobg = new CharGenerator(2 * 1024, 8 * 1024);
|
||||||
|
TimestampGenerator dateg = new TimestampGenerator(0);
|
||||||
|
FloatGenerator fg = new FloatGenerator(126);
|
||||||
|
CharGenerator lg = new CharGenerator(2 * 1024, 8 * 1024);
|
||||||
|
NCharGenerator ncg = new NCharGenerator(30, 30);
|
||||||
|
NCharGenerator nclobg = new NCharGenerator(2 * 1024, 8 * 1024);
|
||||||
|
BigDecimalGenerator ng = new BigDecimalGenerator(9, 2);
|
||||||
|
NCharGenerator nvcg = new NCharGenerator(1, 30);
|
||||||
|
RowIdGenerator rg = new RowIdGenerator();
|
||||||
|
URIGenerator ug = new URIGenerator();
|
||||||
|
IntervalYearMonthGenerator iymg = new IntervalYearMonthGenerator(2);
|
||||||
|
IntervalDaySecondGenerator idsg = new IntervalDaySecondGenerator(2, 6);
|
||||||
|
TimestampGenerator tg = new TimestampGenerator(6);
|
||||||
|
TimestampGenerator tzg = new TimestampGenerator(6);
|
||||||
|
TimestampGenerator tltzg = new TimestampGenerator(6);
|
||||||
|
BytesGenerator rawg = new BytesGenerator(21, 21);
|
||||||
|
PreparedStatement ps =
|
||||||
|
conn.prepareStatement("INSERT INTO "
|
||||||
|
+ OracleUtils.SYSTEMTEST_TABLE_NAME
|
||||||
|
+ " ( id, bd, bf, b, c, cl, d, f, nc, ncl, n, nvc, r, u, iym, "
|
||||||
|
+ "ids, t, tz, tltz, rawcol ) VALUES ( ?, ?, ?, ?, ?, ?, ?, ?, "
|
||||||
|
+ "?, ?, ?, ?, ?, sys.UriFactory.getUri(?), ?, ?, ?, ?, ?, ? )");
|
||||||
|
try {
|
||||||
|
for (int i = 0; i < numRows; i++) {
|
||||||
|
ps.setInt(1, i);
|
||||||
|
methSetBinaryDouble.invoke(ps, 2, bdg.next());
|
||||||
|
methSetBinaryFloat.invoke(ps, 3, bfg.next());
|
||||||
|
ps.setBlob(4, bg.next());
|
||||||
|
ps.setString(5, cg.next());
|
||||||
|
ps.setString(6, clobg.next());
|
||||||
|
ps.setTimestamp(7, dateg.next());
|
||||||
|
ps.setBigDecimal(8, fg.next());
|
||||||
|
ps.setString(9, ncg.next());
|
||||||
|
ps.setString(10, nclobg.next());
|
||||||
|
ps.setBigDecimal(11, ng.next());
|
||||||
|
ps.setString(12, nvcg.next());
|
||||||
|
ps.setRowId(13, rg.next());
|
||||||
|
ps.setString(14, ug.next());
|
||||||
|
ps.setString(15, iymg.next());
|
||||||
|
ps.setString(16, idsg.next());
|
||||||
|
ps.setTimestamp(17, tg.next());
|
||||||
|
ps.setTimestamp(18, tzg.next());
|
||||||
|
ps.setTimestamp(19, tltzg.next());
|
||||||
|
ps.setBytes(20, rawg.next());
|
||||||
|
ps.executeUpdate();
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
ps.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
// Can't bind > 4000 bytes of data to LONG and LOB columns in the same
|
||||||
|
// statement, so do LONG by itself
|
||||||
|
ps =
|
||||||
|
conn.prepareStatement("UPDATE " + OracleUtils.SYSTEMTEST_TABLE_NAME
|
||||||
|
+ " SET l = ? WHERE id = ?");
|
||||||
|
try {
|
||||||
|
for (int i = 0; i < numRows; i++) {
|
||||||
|
ps.setString(1, lg.next());
|
||||||
|
ps.setInt(2, i);
|
||||||
|
ps.executeUpdate();
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
ps.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
// Import test data into hadoop
|
||||||
|
|
||||||
|
int retCode =
|
||||||
|
runImport(OracleUtils.SYSTEMTEST_TABLE_NAME, getSqoopConf(), true);
|
||||||
|
assertEquals("Return code should be 0", 0, retCode);
|
||||||
|
|
||||||
|
// Add sqoop generated code to the classpath
|
||||||
|
String sqoopGenJarPath =
|
||||||
|
"file://" + getSqoopGenLibDirectory() + "/"
|
||||||
|
+ getSqoopGenClassName() + ".jar";
|
||||||
|
URLClassLoader loader =
|
||||||
|
new URLClassLoader(new URL[] { new URL(sqoopGenJarPath) },
|
||||||
|
getClass().getClassLoader());
|
||||||
|
Thread.currentThread().setContextClassLoader(loader);
|
||||||
|
|
||||||
|
// Read test data from hadoop
|
||||||
|
Configuration hadoopConf = getSqoopConf();
|
||||||
|
FileSystem hdfs = FileSystem.get(hadoopConf);
|
||||||
|
Path path = new Path(getSqoopTargetDirectory());
|
||||||
|
FileStatus[] statuses = hdfs.listStatus(path);
|
||||||
|
int hadoopRecordCount = 0;
|
||||||
|
for (FileStatus status : statuses) {
|
||||||
|
if (status.getPath().getName().startsWith("part-m-")) {
|
||||||
|
|
||||||
|
SequenceFile.Reader reader =
|
||||||
|
new SequenceFile.Reader(hdfs, status.getPath(), hadoopConf);
|
||||||
|
LongWritable key = new LongWritable();
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
SqoopRecord value =
|
||||||
|
((Class<SqoopRecord>) reader.getValueClass())
|
||||||
|
.getConstructor().newInstance();
|
||||||
|
ps =
|
||||||
|
conn.prepareStatement("SELECT bd, bf, b, c, cl, d, f, l, nc, "
|
||||||
|
+ "ncl, nvc, r, u, iym, ids, t, tz, tltz, rawcol FROM "
|
||||||
|
+ OracleUtils.SYSTEMTEST_TABLE_NAME + " WHERE id = ?");
|
||||||
|
while (reader.next(key, value)) {
|
||||||
|
// Compare test data from hadoop with data in oracle
|
||||||
|
Map<String, Object> fields = value.getFieldMap();
|
||||||
|
BigDecimal id = (BigDecimal) fields.get("ID");
|
||||||
|
ps.setBigDecimal(1, id);
|
||||||
|
ResultSet rs = ps.executeQuery();
|
||||||
|
assertTrue("Did not find row with id " + id + " in oracle", rs
|
||||||
|
.next());
|
||||||
|
assertEquals("BinaryDouble did not match for row " + id, fields
|
||||||
|
.get("BD"), rs.getDouble(1));
|
||||||
|
assertEquals("BinaryFloat did not match for row " + id, fields
|
||||||
|
.get("BF"), rs.getFloat(2));
|
||||||
|
// LONG column needs to be read before BLOB column
|
||||||
|
assertEquals("Long did not match for row " + id, fields
|
||||||
|
.get("L"), rs.getString(8));
|
||||||
|
BlobRef hadoopBlob = (BlobRef) fields.get("B");
|
||||||
|
Blob oraBlob = rs.getBlob(3);
|
||||||
|
assertTrue("Blob did not match for row " + id, Arrays.equals(
|
||||||
|
hadoopBlob.getData(), oraBlob.getBytes(1L, (int) oraBlob
|
||||||
|
.length())));
|
||||||
|
assertEquals("Char did not match for row " + id, fields
|
||||||
|
.get("C"), rs.getString(4));
|
||||||
|
ClobRef hadoopClob = (ClobRef) fields.get("CL");
|
||||||
|
Clob oraClob = rs.getClob(5);
|
||||||
|
assertEquals("Clob did not match for row " + id, hadoopClob
|
||||||
|
.getData(), oraClob.getSubString(1, (int) oraClob.length()));
|
||||||
|
assertEquals("Date did not match for row " + id, fields
|
||||||
|
.get("D"), rs.getString(6));
|
||||||
|
BigDecimal hadoopFloat = (BigDecimal) fields.get("F");
|
||||||
|
BigDecimal oraFloat = rs.getBigDecimal(7);
|
||||||
|
assertEquals("Float did not match for row " + id, hadoopFloat,
|
||||||
|
oraFloat);
|
||||||
|
assertEquals("NChar did not match for row " + id, fields
|
||||||
|
.get("NC"), rs.getString(9));
|
||||||
|
assertEquals("NClob did not match for row " + id, fields
|
||||||
|
.get("NCL"), rs.getString(10));
|
||||||
|
assertEquals("NVarChar did not match for row " + id, fields
|
||||||
|
.get("NVC"), rs.getString(11));
|
||||||
|
assertEquals("RowId did not match for row " + id, fields
|
||||||
|
.get("R"), new String(rs.getRowId(12).getBytes()));
|
||||||
|
Struct url = (Struct) rs.getObject(13); // TODO: Find a fix for
|
||||||
|
// this workaround
|
||||||
|
String urlString = (String) url.getAttributes()[0];
|
||||||
|
if (url.getSQLTypeName().equals("SYS.HTTPURITYPE")) {
|
||||||
|
urlString = "http://" + urlString;
|
||||||
|
} else if (url.getSQLTypeName().equals("SYS.DBURITYPE")) {
|
||||||
|
urlString = "/ORADB" + urlString;
|
||||||
|
}
|
||||||
|
assertEquals("UriType did not match for row " + id, fields
|
||||||
|
.get("U"), urlString);
|
||||||
|
assertEquals("Interval Year to Month did not match for row "
|
||||||
|
+ id, fields.get("IYM"), rs.getString(14));
|
||||||
|
String ids = (String) fields.get("IDS"); // Strip trailing zeros
|
||||||
|
// to match oracle
|
||||||
|
// format
|
||||||
|
int lastNonZero = ids.length() - 1;
|
||||||
|
while (ids.charAt(lastNonZero) == '0') {
|
||||||
|
lastNonZero--;
|
||||||
|
}
|
||||||
|
ids = ids.substring(0, lastNonZero + 1);
|
||||||
|
assertEquals("Interval Day to Second did not match for row "
|
||||||
|
+ id, ids, rs.getString(15));
|
||||||
|
assertEquals("Timestamp did not match for row " + id, fields
|
||||||
|
.get("T"), rs.getString(16));
|
||||||
|
assertEquals("Timestamp with Time Zone did not match for row "
|
||||||
|
+ id, fields.get("TZ"), rs.getString(17));
|
||||||
|
assertEquals(
|
||||||
|
"Timestamp with Local Time Zone did not match for row "
|
||||||
|
+ id, fields.get("TLTZ"), rs.getString(18));
|
||||||
|
BytesWritable rawCol = (BytesWritable) fields.get("RAWCOL");
|
||||||
|
byte[] rawColData =
|
||||||
|
Arrays.copyOf(rawCol.getBytes(), rawCol.getLength());
|
||||||
|
assertTrue("RAW did not match for row " + id, Arrays.equals(
|
||||||
|
rawColData, rs.getBytes(19)));
|
||||||
|
|
||||||
|
assertFalse("Found multiple rows with id " + id + " in oracle",
|
||||||
|
rs.next());
|
||||||
|
hadoopRecordCount++;
|
||||||
|
}
|
||||||
|
reader.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
ResultSet rs =
|
||||||
|
s.executeQuery("SELECT COUNT(*) FROM "
|
||||||
|
+ OracleUtils.SYSTEMTEST_TABLE_NAME);
|
||||||
|
rs.next();
|
||||||
|
int oracleRecordCount = rs.getInt(1);
|
||||||
|
assertEquals(
|
||||||
|
"Number of records in Hadoop does not match number of "
|
||||||
|
+ "records in oracle",
|
||||||
|
hadoopRecordCount, oracleRecordCount);
|
||||||
|
rs.close();
|
||||||
|
} finally {
|
||||||
|
// Delete test data from hadoop
|
||||||
|
cleanupFolders();
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
// Delete test data from oracle
|
||||||
|
s.executeUpdate("DROP TABLE " + OracleUtils.SYSTEMTEST_TABLE_NAME);
|
||||||
|
s.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
} finally {
|
||||||
|
closeTestEnvConnection();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,131 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import org.apache.hadoop.mapreduce.InputSplit;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
|
||||||
|
import org.apache.sqoop.manager.oracle.OraOopConstants.
|
||||||
|
OraOopOracleBlockToSplitAllocationMethod;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Unit tests for OraOopDataDrivenDBInputFormat.
|
||||||
|
*/
|
||||||
|
public class TestOraOopDataDrivenDBInputFormat extends OraOopTestCase {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* We're just exposing a protected method so that it can be called by this
|
||||||
|
* unit test...
|
||||||
|
*/
|
||||||
|
public class Exposer<T extends SqoopRecord> extends
|
||||||
|
OraOopDataDrivenDBInputFormat<T> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public
|
||||||
|
List<InputSplit>
|
||||||
|
groupTableDataChunksIntoSplits(
|
||||||
|
List<? extends OraOopOracleDataChunk> dataChunks,
|
||||||
|
int desiredNumberOfSplits,
|
||||||
|
OraOopConstants.OraOopOracleBlockToSplitAllocationMethod
|
||||||
|
blockAllocationMethod) {
|
||||||
|
|
||||||
|
return super.groupTableDataChunksIntoSplits(dataChunks,
|
||||||
|
desiredNumberOfSplits, blockAllocationMethod);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
@Test
|
||||||
|
public void testgroupTableDataChunksIntoSplits() {
|
||||||
|
|
||||||
|
List<OraOopOracleDataChunk> dataChunks =
|
||||||
|
new ArrayList<OraOopOracleDataChunk>();
|
||||||
|
|
||||||
|
int startBlockNumber = 1;
|
||||||
|
for (int idx = 0; idx < 241; idx++) {
|
||||||
|
OraOopOracleDataChunk dataChunk =
|
||||||
|
new OraOopOracleDataChunkExtent("23480", 666, 1, startBlockNumber,
|
||||||
|
startBlockNumber + 8);
|
||||||
|
startBlockNumber += 8;
|
||||||
|
dataChunks.add(dataChunk);
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("rawtypes")
|
||||||
|
Exposer e = new Exposer();
|
||||||
|
|
||||||
|
// Prevent setJdbcFetchSize() from logging information about the fetch-size
|
||||||
|
// changing. Otherwise, the junit output will be polluted with messages
|
||||||
|
// about
|
||||||
|
// things that aren't actually a problem...
|
||||||
|
boolean logIsBeingCached = Exposer.LOG.getCacheLogEntries();
|
||||||
|
Exposer.LOG.setCacheLogEntries(true);
|
||||||
|
|
||||||
|
List<InputSplit> splits =
|
||||||
|
e.groupTableDataChunksIntoSplits(dataChunks, 32,
|
||||||
|
OraOopOracleBlockToSplitAllocationMethod.SEQUENTIAL);
|
||||||
|
|
||||||
|
Exposer.LOG.setCacheLogEntries(logIsBeingCached);
|
||||||
|
|
||||||
|
int highestNumberOfDataChunksAllocatedToASplit = 0;
|
||||||
|
int lowestNumberOfDataChunksAllocatedToASplit = Integer.MAX_VALUE;
|
||||||
|
|
||||||
|
// Check that all splits have data-chunks assigned to them...
|
||||||
|
for (InputSplit split : splits) {
|
||||||
|
int dataChunksAllocatedToThisSplit =
|
||||||
|
((OraOopDBInputSplit) split).getNumberOfDataChunks();
|
||||||
|
highestNumberOfDataChunksAllocatedToASplit =
|
||||||
|
Math.max(highestNumberOfDataChunksAllocatedToASplit,
|
||||||
|
dataChunksAllocatedToThisSplit);
|
||||||
|
lowestNumberOfDataChunksAllocatedToASplit =
|
||||||
|
Math.min(lowestNumberOfDataChunksAllocatedToASplit,
|
||||||
|
dataChunksAllocatedToThisSplit);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (lowestNumberOfDataChunksAllocatedToASplit == 0) {
|
||||||
|
Assert
|
||||||
|
.fail("There is a split that has not had any "
|
||||||
|
+ "data-chunks allocated to it.");
|
||||||
|
}
|
||||||
|
|
||||||
|
// Check that the split with the least data-chunks has at least
|
||||||
|
// 75% of the number of data-chunks of the split with the most
|
||||||
|
// data-chunks...
|
||||||
|
double minExpectedWorkloadRatio = 0.75;
|
||||||
|
double actualWorkloadRatio =
|
||||||
|
(double) lowestNumberOfDataChunksAllocatedToASplit
|
||||||
|
/ highestNumberOfDataChunksAllocatedToASplit;
|
||||||
|
if (actualWorkloadRatio < minExpectedWorkloadRatio) {
|
||||||
|
Assert.fail(String.format(
|
||||||
|
"There is too much difference in the amount of work assigned "
|
||||||
|
+ "to the 'smallest' split and the 'largest' split. "
|
||||||
|
+ "The split with the least work should have at least %s "
|
||||||
|
+ "of the workload of the 'largest' split, but it actually "
|
||||||
|
+ "only has %s of the workload of the 'largest' split.",
|
||||||
|
minExpectedWorkloadRatio, actualWorkloadRatio));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
276
src/test/org/apache/sqoop/manager/oracle/TestOraOopJdbcUrl.java
Normal file
276
src/test/org/apache/sqoop/manager/oracle/TestOraOopJdbcUrl.java
Normal file
@ -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);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,619 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import junit.framework.Assert;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Unit tests for OraOopUtilities.
|
||||||
|
*/
|
||||||
|
public class TestOraOopUtilities extends OraOopTestCase {
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testdecodeOracleTableName() {
|
||||||
|
|
||||||
|
OracleTable context = null;
|
||||||
|
|
||||||
|
// These are the possibilities for double-quote location...
|
||||||
|
// table
|
||||||
|
// "table"
|
||||||
|
// schema.table
|
||||||
|
// schema."table"
|
||||||
|
// "schema".table
|
||||||
|
// "schema"."table"
|
||||||
|
|
||||||
|
// table
|
||||||
|
context = OraOopUtilities.decodeOracleTableName("oraoop", "junk", null);
|
||||||
|
Assert.assertEquals(context.getSchema(), "ORAOOP");
|
||||||
|
Assert.assertEquals(context.getName(), "JUNK");
|
||||||
|
|
||||||
|
// "table"
|
||||||
|
context = OraOopUtilities.decodeOracleTableName("oraoop", "\"Junk\"", null);
|
||||||
|
Assert.assertEquals(context.getSchema(), "ORAOOP");
|
||||||
|
Assert.assertEquals(context.getName(), "Junk");
|
||||||
|
|
||||||
|
// schema.table
|
||||||
|
context =
|
||||||
|
OraOopUtilities.decodeOracleTableName("oraoop", "targusr.junk", null);
|
||||||
|
Assert.assertEquals(context.getSchema(), "TARGUSR");
|
||||||
|
Assert.assertEquals(context.getName(), "JUNK");
|
||||||
|
|
||||||
|
// schema."table"
|
||||||
|
context =
|
||||||
|
OraOopUtilities.decodeOracleTableName("oraoop", "targusr.\"Junk\"",
|
||||||
|
null);
|
||||||
|
Assert.assertEquals(context.getSchema(), "TARGUSR");
|
||||||
|
Assert.assertEquals(context.getName(), "Junk");
|
||||||
|
|
||||||
|
// "schema".table
|
||||||
|
context =
|
||||||
|
OraOopUtilities.decodeOracleTableName("oraoop", "\"Targusr\".junk",
|
||||||
|
null);
|
||||||
|
Assert.assertEquals(context.getSchema(), "Targusr");
|
||||||
|
Assert.assertEquals(context.getName(), "JUNK");
|
||||||
|
|
||||||
|
// "schema"."table"
|
||||||
|
String inputStr = "\"Targusr\".\"Junk\"";
|
||||||
|
context = OraOopUtilities.decodeOracleTableName("oraoop", inputStr, null);
|
||||||
|
Assert.assertEquals(context.getSchema(), "Targusr");
|
||||||
|
Assert.assertEquals(context.getName(), "Junk");
|
||||||
|
|
||||||
|
// Test for "." within schema...
|
||||||
|
context =
|
||||||
|
OraOopUtilities.decodeOracleTableName("oraoop", "\"targ.usr\".junk",
|
||||||
|
null);
|
||||||
|
Assert.assertEquals(context.getSchema(), "targ.usr");
|
||||||
|
Assert.assertEquals(context.getName(), "JUNK");
|
||||||
|
|
||||||
|
// Test for "." within table...
|
||||||
|
context =
|
||||||
|
OraOopUtilities.decodeOracleTableName("oraoop",
|
||||||
|
"targusr.\"junk.tab.with.dots\"", null);
|
||||||
|
Assert.assertEquals(context.getSchema(), "TARGUSR");
|
||||||
|
Assert.assertEquals(context.getName(), "junk.tab.with.dots");
|
||||||
|
|
||||||
|
// Test for "." within schema and within table...
|
||||||
|
context =
|
||||||
|
OraOopUtilities.decodeOracleTableName("oraoop",
|
||||||
|
"\"targ.usr\".\"junk.tab.with.dots\"", null);
|
||||||
|
Assert.assertEquals(context.getSchema(), "targ.usr");
|
||||||
|
Assert.assertEquals(context.getName(), "junk.tab.with.dots");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testgetCurrentMethodName() {
|
||||||
|
|
||||||
|
String actual = OraOopUtilities.getCurrentMethodName();
|
||||||
|
String expected = "testgetCurrentMethodName()";
|
||||||
|
|
||||||
|
Assert.assertEquals(expected, actual);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testgenerateDataChunkId() {
|
||||||
|
|
||||||
|
String expected;
|
||||||
|
String actual;
|
||||||
|
|
||||||
|
expected = "1_1";
|
||||||
|
actual = OraOopUtilities.generateDataChunkId(1, 1);
|
||||||
|
Assert.assertEquals(expected, actual);
|
||||||
|
|
||||||
|
expected = "1234_99";
|
||||||
|
actual = OraOopUtilities.generateDataChunkId(1234, 99);
|
||||||
|
Assert.assertEquals(expected, actual);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testgetDuplicatedStringArrayValues() {
|
||||||
|
|
||||||
|
try {
|
||||||
|
OraOopUtilities.getDuplicatedStringArrayValues(null, false);
|
||||||
|
Assert.fail("An IllegalArgumentException should be been thrown.");
|
||||||
|
} catch (IllegalArgumentException ex) {
|
||||||
|
// This is what we want to happen.
|
||||||
|
}
|
||||||
|
|
||||||
|
String[] duplicates = null;
|
||||||
|
|
||||||
|
duplicates =
|
||||||
|
OraOopUtilities.getDuplicatedStringArrayValues(new String[] {}, false);
|
||||||
|
Assert.assertEquals(0, duplicates.length);
|
||||||
|
|
||||||
|
duplicates =
|
||||||
|
OraOopUtilities.getDuplicatedStringArrayValues(new String[] { "a", "b",
|
||||||
|
"c", }, false);
|
||||||
|
Assert.assertEquals(0, duplicates.length);
|
||||||
|
|
||||||
|
duplicates =
|
||||||
|
OraOopUtilities.getDuplicatedStringArrayValues(new String[] { "a", "A",
|
||||||
|
"b", }, false);
|
||||||
|
Assert.assertEquals(0, duplicates.length);
|
||||||
|
|
||||||
|
duplicates =
|
||||||
|
OraOopUtilities.getDuplicatedStringArrayValues(new String[] { "a", "A",
|
||||||
|
"b", }, true);
|
||||||
|
Assert.assertEquals(1, duplicates.length);
|
||||||
|
Assert.assertEquals("A", duplicates[0]);
|
||||||
|
|
||||||
|
duplicates =
|
||||||
|
OraOopUtilities.getDuplicatedStringArrayValues(new String[] { "A", "a",
|
||||||
|
"b", }, true);
|
||||||
|
Assert.assertEquals(1, duplicates.length);
|
||||||
|
Assert.assertEquals("a", duplicates[0]);
|
||||||
|
|
||||||
|
duplicates =
|
||||||
|
OraOopUtilities.getDuplicatedStringArrayValues(new String[] { "A", "a",
|
||||||
|
"b", "A", }, false);
|
||||||
|
Assert.assertEquals(1, duplicates.length);
|
||||||
|
Assert.assertEquals("A", duplicates[0]);
|
||||||
|
|
||||||
|
duplicates =
|
||||||
|
OraOopUtilities.getDuplicatedStringArrayValues(new String[] { "A", "a",
|
||||||
|
"b", "A", }, true);
|
||||||
|
Assert.assertEquals(2, duplicates.length);
|
||||||
|
Assert.assertEquals("a", duplicates[0]);
|
||||||
|
Assert.assertEquals("A", duplicates[1]);
|
||||||
|
|
||||||
|
duplicates =
|
||||||
|
OraOopUtilities.getDuplicatedStringArrayValues(new String[] { "A", "a",
|
||||||
|
"b", "A", "A", }, true);
|
||||||
|
Assert.assertEquals(2, duplicates.length);
|
||||||
|
Assert.assertEquals("a", duplicates[0]);
|
||||||
|
Assert.assertEquals("A", duplicates[1]);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testgetFullExceptionMessage() {
|
||||||
|
|
||||||
|
try {
|
||||||
|
|
||||||
|
try {
|
||||||
|
try {
|
||||||
|
throw new IOException("lorem ipsum!");
|
||||||
|
} catch (IOException ex) {
|
||||||
|
throw new SQLException("dolor sit amet", ex);
|
||||||
|
}
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
throw new RuntimeException("consectetur adipisicing elit", ex);
|
||||||
|
}
|
||||||
|
|
||||||
|
} catch (Exception ex) {
|
||||||
|
String msg = OraOopUtilities.getFullExceptionMessage(ex);
|
||||||
|
if (!msg.contains("IOException") || !msg.contains("lorem ipsum!")) {
|
||||||
|
Assert
|
||||||
|
.fail("Inner exception text has not been included in the message");
|
||||||
|
}
|
||||||
|
if (!msg.contains("SQLException") || !msg.contains("dolor sit amet")) {
|
||||||
|
Assert
|
||||||
|
.fail("Inner exception text has not been included in the message");
|
||||||
|
}
|
||||||
|
if (!msg.contains("RuntimeException")
|
||||||
|
|| !msg.contains("consectetur adipisicing elit")) {
|
||||||
|
Assert
|
||||||
|
.fail("Outer exception text has not been included in the message");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetOraOopOracleDataChunkMethod() {
|
||||||
|
try {
|
||||||
|
OraOopUtilities.getOraOopOracleDataChunkMethod(null);
|
||||||
|
Assert.fail("An IllegalArgumentException should be been thrown.");
|
||||||
|
} catch (IllegalArgumentException ex) {
|
||||||
|
// This is what we want to happen.
|
||||||
|
}
|
||||||
|
|
||||||
|
OraOopConstants.OraOopOracleDataChunkMethod dataChunkMethod;
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
|
||||||
|
// Check the default is ROWID
|
||||||
|
dataChunkMethod = OraOopUtilities.getOraOopOracleDataChunkMethod(conf);
|
||||||
|
Assert.assertEquals(OraOopConstants.OraOopOracleDataChunkMethod.ROWID,
|
||||||
|
dataChunkMethod);
|
||||||
|
|
||||||
|
// Invalid value specified
|
||||||
|
OraOopUtilities.LOG.setCacheLogEntries(true);
|
||||||
|
OraOopUtilities.LOG.clearCache();
|
||||||
|
conf.set(OraOopConstants.ORAOOP_ORACLE_DATA_CHUNK_METHOD, "loremipsum");
|
||||||
|
dataChunkMethod = OraOopUtilities.getOraOopOracleDataChunkMethod(conf);
|
||||||
|
String logText = OraOopUtilities.LOG.getLogEntries();
|
||||||
|
OraOopUtilities.LOG.setCacheLogEntries(false);
|
||||||
|
if (!logText.toLowerCase().contains("loremipsum")) {
|
||||||
|
Assert
|
||||||
|
.fail("The LOG should inform the user they've selected an invalid "
|
||||||
|
+ "data chunk method - and what that was.");
|
||||||
|
}
|
||||||
|
Assert.assertEquals("Should have used the default value",
|
||||||
|
OraOopConstants.ORAOOP_ORACLE_DATA_CHUNK_METHOD_DEFAULT,
|
||||||
|
dataChunkMethod);
|
||||||
|
|
||||||
|
// Valid value specified
|
||||||
|
conf.set(OraOopConstants.ORAOOP_ORACLE_DATA_CHUNK_METHOD, "partition");
|
||||||
|
dataChunkMethod = OraOopUtilities.getOraOopOracleDataChunkMethod(conf);
|
||||||
|
Assert.assertEquals(OraOopConstants.OraOopOracleDataChunkMethod.PARTITION,
|
||||||
|
dataChunkMethod);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testgetOraOopOracleBlockToSplitAllocationMethod() {
|
||||||
|
|
||||||
|
// Invalid arguments test...
|
||||||
|
try {
|
||||||
|
OraOopUtilities.getOraOopOracleBlockToSplitAllocationMethod(null,
|
||||||
|
OraOopConstants.OraOopOracleBlockToSplitAllocationMethod.RANDOM);
|
||||||
|
Assert.fail("An IllegalArgumentException should be been thrown.");
|
||||||
|
} catch (IllegalArgumentException ex) {
|
||||||
|
// This is what we want to happen.
|
||||||
|
}
|
||||||
|
|
||||||
|
OraOopConstants.OraOopOracleBlockToSplitAllocationMethod allocationMethod;
|
||||||
|
org.apache.hadoop.conf.Configuration conf = new Configuration();
|
||||||
|
|
||||||
|
// No configuration property - and RANDOM used by default...
|
||||||
|
allocationMethod =
|
||||||
|
OraOopUtilities.getOraOopOracleBlockToSplitAllocationMethod(conf,
|
||||||
|
OraOopConstants.OraOopOracleBlockToSplitAllocationMethod.RANDOM);
|
||||||
|
Assert.assertEquals(
|
||||||
|
OraOopConstants.OraOopOracleBlockToSplitAllocationMethod.RANDOM,
|
||||||
|
allocationMethod);
|
||||||
|
|
||||||
|
// No configuration property - and SEQUENTIAL used by default...
|
||||||
|
allocationMethod =
|
||||||
|
OraOopUtilities.getOraOopOracleBlockToSplitAllocationMethod(
|
||||||
|
conf,
|
||||||
|
OraOopConstants.OraOopOracleBlockToSplitAllocationMethod.SEQUENTIAL);
|
||||||
|
Assert.assertEquals(
|
||||||
|
OraOopConstants.OraOopOracleBlockToSplitAllocationMethod.SEQUENTIAL,
|
||||||
|
allocationMethod);
|
||||||
|
|
||||||
|
// An invalid property value specified...
|
||||||
|
OraOopUtilities.LOG.setCacheLogEntries(true);
|
||||||
|
OraOopUtilities.LOG.clearCache();
|
||||||
|
conf.set(OraOopConstants.ORAOOP_ORACLE_BLOCK_TO_SPLIT_ALLOCATION_METHOD,
|
||||||
|
"loremipsum");
|
||||||
|
allocationMethod =
|
||||||
|
OraOopUtilities.getOraOopOracleBlockToSplitAllocationMethod(
|
||||||
|
conf,
|
||||||
|
OraOopConstants.OraOopOracleBlockToSplitAllocationMethod.SEQUENTIAL);
|
||||||
|
String logText = OraOopUtilities.LOG.getLogEntries();
|
||||||
|
OraOopUtilities.LOG.setCacheLogEntries(false);
|
||||||
|
if (!logText.toLowerCase().contains("loremipsum")) {
|
||||||
|
Assert
|
||||||
|
.fail("The LOG should inform the user they've selected an invalid "
|
||||||
|
+ "allocation method - and what that was.");
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!logText.contains("ROUNDROBIN or SEQUENTIAL or RANDOM")) {
|
||||||
|
Assert.fail("The LOG should inform the user what the valid choices are.");
|
||||||
|
}
|
||||||
|
|
||||||
|
// An valid property value specified...
|
||||||
|
conf.set(OraOopConstants.ORAOOP_ORACLE_BLOCK_TO_SPLIT_ALLOCATION_METHOD,
|
||||||
|
"sequential");
|
||||||
|
allocationMethod =
|
||||||
|
OraOopUtilities.getOraOopOracleBlockToSplitAllocationMethod(
|
||||||
|
conf,
|
||||||
|
OraOopConstants.OraOopOracleBlockToSplitAllocationMethod.SEQUENTIAL);
|
||||||
|
Assert.assertEquals(
|
||||||
|
OraOopConstants.OraOopOracleBlockToSplitAllocationMethod.SEQUENTIAL,
|
||||||
|
allocationMethod);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testgetOraOopTableImportWhereClauseLocation() {
|
||||||
|
|
||||||
|
// Invalid arguments test...
|
||||||
|
try {
|
||||||
|
OraOopUtilities.getOraOopTableImportWhereClauseLocation(null,
|
||||||
|
OraOopConstants.OraOopTableImportWhereClauseLocation.SPLIT);
|
||||||
|
Assert.fail("An IllegalArgumentException should be been thrown.");
|
||||||
|
} catch (IllegalArgumentException ex) {
|
||||||
|
// This is what we want to happen.
|
||||||
|
}
|
||||||
|
|
||||||
|
OraOopConstants.OraOopTableImportWhereClauseLocation location;
|
||||||
|
org.apache.hadoop.conf.Configuration conf = new Configuration();
|
||||||
|
|
||||||
|
// No configuration property - and SPLIT used by default...
|
||||||
|
location =
|
||||||
|
OraOopUtilities.getOraOopTableImportWhereClauseLocation(conf,
|
||||||
|
OraOopConstants.OraOopTableImportWhereClauseLocation.SPLIT);
|
||||||
|
Assert.assertEquals(
|
||||||
|
OraOopConstants.OraOopTableImportWhereClauseLocation.SPLIT, location);
|
||||||
|
|
||||||
|
// An invalid property value specified...
|
||||||
|
OraOopUtilities.LOG.setCacheLogEntries(true);
|
||||||
|
OraOopUtilities.LOG.clearCache();
|
||||||
|
conf.set(OraOopConstants.ORAOOP_TABLE_IMPORT_WHERE_CLAUSE_LOCATION,
|
||||||
|
"loremipsum");
|
||||||
|
location =
|
||||||
|
OraOopUtilities.getOraOopTableImportWhereClauseLocation(conf,
|
||||||
|
OraOopConstants.OraOopTableImportWhereClauseLocation.SPLIT);
|
||||||
|
String logText = OraOopUtilities.LOG.getLogEntries();
|
||||||
|
OraOopUtilities.LOG.setCacheLogEntries(false);
|
||||||
|
if (!logText.toLowerCase().contains("loremipsum")) {
|
||||||
|
Assert
|
||||||
|
.fail("The LOG should inform the user they've selected an invalid "
|
||||||
|
+ "where-clause-location - and what that was.");
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!logText.contains("SUBSPLIT or SPLIT")) {
|
||||||
|
Assert.fail("The LOG should inform the user what the valid choices are.");
|
||||||
|
}
|
||||||
|
|
||||||
|
// An valid property value specified...
|
||||||
|
conf.set(OraOopConstants.ORAOOP_TABLE_IMPORT_WHERE_CLAUSE_LOCATION,
|
||||||
|
"split");
|
||||||
|
location =
|
||||||
|
OraOopUtilities.getOraOopTableImportWhereClauseLocation(conf,
|
||||||
|
OraOopConstants.OraOopTableImportWhereClauseLocation.SUBSPLIT);
|
||||||
|
Assert.assertEquals(
|
||||||
|
OraOopConstants.OraOopTableImportWhereClauseLocation.SPLIT, location);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testpadLeft() {
|
||||||
|
|
||||||
|
String expected = " a";
|
||||||
|
String actual = OraOopUtilities.padLeft("a", 4);
|
||||||
|
Assert.assertEquals(expected, actual);
|
||||||
|
|
||||||
|
expected = "abcd";
|
||||||
|
actual = OraOopUtilities.padLeft("abcd", 3);
|
||||||
|
Assert.assertEquals(expected, actual);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testpadRight() {
|
||||||
|
|
||||||
|
String expected = "a ";
|
||||||
|
String actual = OraOopUtilities.padRight("a", 4);
|
||||||
|
Assert.assertEquals(expected, actual);
|
||||||
|
|
||||||
|
expected = "abcd";
|
||||||
|
actual = OraOopUtilities.padRight("abcd", 3);
|
||||||
|
Assert.assertEquals(expected, actual);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testReplaceConfigurationExpression() {
|
||||||
|
|
||||||
|
org.apache.hadoop.conf.Configuration conf = new Configuration();
|
||||||
|
|
||||||
|
// Default value used...
|
||||||
|
String actual =
|
||||||
|
OraOopUtilities.replaceConfigurationExpression(
|
||||||
|
"alter session set timezone = '{oracle.sessionTimeZone|GMT}';",
|
||||||
|
conf);
|
||||||
|
String expected = "alter session set timezone = 'GMT';";
|
||||||
|
Assert.assertEquals("OraOop configuration expression failure.", expected,
|
||||||
|
actual);
|
||||||
|
|
||||||
|
// Configuration property value exists...
|
||||||
|
conf.set("oracle.sessionTimeZone", "Africa/Algiers");
|
||||||
|
actual =
|
||||||
|
OraOopUtilities.replaceConfigurationExpression(
|
||||||
|
"alter session set timezone = '{oracle.sessionTimeZone|GMT}';",
|
||||||
|
conf);
|
||||||
|
expected = "alter session set timezone = 'Africa/Algiers';";
|
||||||
|
Assert.assertEquals("OraOop configuration expression failure.", expected,
|
||||||
|
actual);
|
||||||
|
|
||||||
|
// Multiple properties in one expression...
|
||||||
|
conf.set("expr1", "1");
|
||||||
|
conf.set("expr2", "2");
|
||||||
|
conf.set("expr3", "3");
|
||||||
|
conf.set("expr4", "4");
|
||||||
|
actual =
|
||||||
|
OraOopUtilities.replaceConfigurationExpression("set {expr1}={expr2};",
|
||||||
|
conf);
|
||||||
|
expected = "set 1=2;";
|
||||||
|
Assert.assertEquals("OraOop configuration expression failure.", expected,
|
||||||
|
actual);
|
||||||
|
|
||||||
|
actual =
|
||||||
|
OraOopUtilities.replaceConfigurationExpression(
|
||||||
|
"set {expr4|0}={expr5|5};", conf);
|
||||||
|
expected = "set 4=5;";
|
||||||
|
Assert.assertEquals("OraOop configuration expression failure.", expected,
|
||||||
|
actual);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testStackContainsClass() {
|
||||||
|
|
||||||
|
if (OraOopUtilities.stackContainsClass("lorem.ipsum.dolor")) {
|
||||||
|
Assert.fail("There's no way the stack actually contains this!");
|
||||||
|
}
|
||||||
|
|
||||||
|
String expected = "org.apache.sqoop.manager.oracle.TestOraOopUtilities";
|
||||||
|
if (!OraOopUtilities.stackContainsClass(expected)) {
|
||||||
|
Assert.fail("The stack should contain the class:" + expected);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetImportHint() {
|
||||||
|
org.apache.hadoop.conf.Configuration conf = new Configuration();
|
||||||
|
|
||||||
|
String hint = OraOopUtilities.getImportHint(conf);
|
||||||
|
Assert.assertEquals("Default import hint", "/*+ NO_INDEX(t) */ ", hint);
|
||||||
|
|
||||||
|
conf.set("oraoop.import.hint", "NO_INDEX(t) SCN_ASCENDING");
|
||||||
|
hint = OraOopUtilities.getImportHint(conf);
|
||||||
|
Assert.assertEquals("Changed import hint",
|
||||||
|
"/*+ NO_INDEX(t) SCN_ASCENDING */ ", hint);
|
||||||
|
|
||||||
|
conf.set("oraoop.import.hint", " ");
|
||||||
|
hint = OraOopUtilities.getImportHint(conf);
|
||||||
|
Assert.assertEquals("Whitespace import hint", "", hint);
|
||||||
|
|
||||||
|
conf.set("oraoop.import.hint", "");
|
||||||
|
hint = OraOopUtilities.getImportHint(conf);
|
||||||
|
Assert.assertEquals("Blank import hint", "", hint);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSplitStringList() {
|
||||||
|
List<String> result = null;
|
||||||
|
List<String> expected = null;
|
||||||
|
|
||||||
|
expected = new ArrayList<String>();
|
||||||
|
expected.add("abcde");
|
||||||
|
expected.add("ghijklm");
|
||||||
|
result = OraOopUtilities.splitStringList("abcde,ghijklm");
|
||||||
|
Assert.assertEquals(expected, result);
|
||||||
|
|
||||||
|
expected = new ArrayList<String>();
|
||||||
|
expected.add("\"abcde\"");
|
||||||
|
expected.add("\"ghijklm\"");
|
||||||
|
result = OraOopUtilities.splitStringList("\"abcde\",\"ghijklm\"");
|
||||||
|
Assert.assertEquals(expected, result);
|
||||||
|
|
||||||
|
expected = new ArrayList<String>();
|
||||||
|
expected.add("abcde");
|
||||||
|
expected.add("\"ghijklm\"");
|
||||||
|
result = OraOopUtilities.splitStringList("abcde,\"ghijklm\"");
|
||||||
|
Assert.assertEquals(expected, result);
|
||||||
|
|
||||||
|
expected = new ArrayList<String>();
|
||||||
|
expected.add("\"abcde\"");
|
||||||
|
expected.add("ghijklm");
|
||||||
|
result = OraOopUtilities.splitStringList("\"abcde\",ghijklm");
|
||||||
|
Assert.assertEquals(expected, result);
|
||||||
|
|
||||||
|
expected = new ArrayList<String>();
|
||||||
|
expected.add("\"ab,cde\"");
|
||||||
|
expected.add("ghijklm");
|
||||||
|
result = OraOopUtilities.splitStringList("\"ab,cde\",ghijklm");
|
||||||
|
Assert.assertEquals(expected, result);
|
||||||
|
|
||||||
|
expected = new ArrayList<String>();
|
||||||
|
expected.add("abcde");
|
||||||
|
expected.add("\"ghi,jklm\"");
|
||||||
|
result = OraOopUtilities.splitStringList("abcde,\"ghi,jklm\"");
|
||||||
|
Assert.assertEquals(expected, result);
|
||||||
|
|
||||||
|
expected = new ArrayList<String>();
|
||||||
|
expected.add("\"ab,cde\"");
|
||||||
|
expected.add("\"ghi,jklm\"");
|
||||||
|
result = OraOopUtilities.splitStringList("\"ab,cde\",\"ghi,jklm\"");
|
||||||
|
Assert.assertEquals(expected, result);
|
||||||
|
|
||||||
|
expected = new ArrayList<String>();
|
||||||
|
expected.add("\"ab,cde\"");
|
||||||
|
expected.add("\"ghi,jklm\"");
|
||||||
|
expected.add("\",Lorem\"");
|
||||||
|
expected.add("\"ip!~sum\"");
|
||||||
|
expected.add("\"do,lo,,r\"");
|
||||||
|
expected.add("\"s#it\"");
|
||||||
|
expected.add("\"am$e$t\"");
|
||||||
|
result =
|
||||||
|
OraOopUtilities
|
||||||
|
.splitStringList("\"ab,cde\",\"ghi,jklm\",\",Lorem\",\"ip!~sum\","
|
||||||
|
+ "\"do,lo,,r\",\"s#it\",\"am$e$t\"");
|
||||||
|
Assert.assertEquals(expected, result);
|
||||||
|
|
||||||
|
expected = new ArrayList<String>();
|
||||||
|
expected.add("LOREM");
|
||||||
|
expected.add("IPSUM");
|
||||||
|
expected.add("DOLOR");
|
||||||
|
expected.add("SIT");
|
||||||
|
expected.add("AMET");
|
||||||
|
result = OraOopUtilities.splitStringList("LOREM,IPSUM,DOLOR,SIT,AMET");
|
||||||
|
Assert.assertEquals(expected, result);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSplitOracleStringList() {
|
||||||
|
List<String> result = null;
|
||||||
|
List<String> expected = null;
|
||||||
|
|
||||||
|
expected = new ArrayList<String>();
|
||||||
|
expected.add("LOREM");
|
||||||
|
expected.add("IPSUM");
|
||||||
|
expected.add("DOLOR");
|
||||||
|
expected.add("SIT");
|
||||||
|
expected.add("AMET");
|
||||||
|
result =
|
||||||
|
OraOopUtilities.splitOracleStringList("lorem,ipsum,dolor,sit,amet");
|
||||||
|
Assert.assertEquals(expected, result);
|
||||||
|
|
||||||
|
expected = new ArrayList<String>();
|
||||||
|
expected.add("LOREM");
|
||||||
|
expected.add("ipsum");
|
||||||
|
expected.add("dolor");
|
||||||
|
expected.add("SIT");
|
||||||
|
expected.add("amet");
|
||||||
|
result =
|
||||||
|
OraOopUtilities
|
||||||
|
.splitOracleStringList("lorem,\"ipsum\",\"dolor\",sit,\"amet\"");
|
||||||
|
Assert.assertEquals(expected, result);
|
||||||
|
|
||||||
|
expected = new ArrayList<String>();
|
||||||
|
expected.add("LOREM");
|
||||||
|
expected.add("ip,sum");
|
||||||
|
expected.add("dol$or");
|
||||||
|
expected.add("SIT");
|
||||||
|
expected.add("am!~#et");
|
||||||
|
result =
|
||||||
|
OraOopUtilities
|
||||||
|
.splitOracleStringList("lorem,\"ip,sum\",\"dol$or\",sit,\"am!~#et\"");
|
||||||
|
Assert.assertEquals(expected, result);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAppendJavaSecurityEgd() {
|
||||||
|
String confProperty = "mapred.child.java.opts";
|
||||||
|
String confValue = "-Djava.security.egd=file:///dev/urandom";
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
|
||||||
|
String expected = confValue;
|
||||||
|
String actual = null;
|
||||||
|
conf.set(confProperty, "");
|
||||||
|
OraOopUtilities.appendJavaSecurityEgd(conf);
|
||||||
|
actual = conf.get(confProperty);
|
||||||
|
Assert.assertEquals("Append to empty string", expected, actual);
|
||||||
|
|
||||||
|
expected = "-Djava.security.egd=file:/dev/random";
|
||||||
|
conf.set(confProperty, expected);
|
||||||
|
OraOopUtilities.appendJavaSecurityEgd(conf);
|
||||||
|
actual = conf.get(confProperty);
|
||||||
|
Assert.assertEquals("Append to empty string", expected, actual);
|
||||||
|
|
||||||
|
expected = confValue + " -Xmx201m";
|
||||||
|
conf.set(confProperty, "-Xmx201m");
|
||||||
|
OraOopUtilities.appendJavaSecurityEgd(conf);
|
||||||
|
actual = conf.get(confProperty);
|
||||||
|
Assert.assertEquals("Append to empty string", expected, actual);
|
||||||
|
}
|
||||||
|
}
|
@ -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());
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -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();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,57 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle.util;
|
||||||
|
|
||||||
|
import java.math.BigDecimal;
|
||||||
|
import java.math.BigInteger;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generates BigDecimal test data.
|
||||||
|
*/
|
||||||
|
public class BigDecimalGenerator extends OraOopTestDataGenerator<BigDecimal> {
|
||||||
|
private final int precision;
|
||||||
|
private final int scale;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a BigDecimalGenerator suitable for populating an Oracle
|
||||||
|
* NUMBER(precision,scale) field.
|
||||||
|
*
|
||||||
|
* @param precision
|
||||||
|
* Maximum number of decimal digits in generated BigDecimals
|
||||||
|
* @param scale
|
||||||
|
* Number of decimal digits to the right of the decimal point in
|
||||||
|
* generated BigDecimals
|
||||||
|
*/
|
||||||
|
public BigDecimalGenerator(int precision, int scale) {
|
||||||
|
super();
|
||||||
|
this.precision = precision;
|
||||||
|
this.scale = scale;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public BigDecimal next() {
|
||||||
|
BigInteger unscaled =
|
||||||
|
BigInteger.valueOf(rng.nextInt((int) Math.pow(10, precision)));
|
||||||
|
BigDecimal value = new BigDecimal(unscaled, scale);
|
||||||
|
if (rng.nextBoolean()) {
|
||||||
|
value = value.negate();
|
||||||
|
}
|
||||||
|
return value;
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,32 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle.util;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generates Double test data. Test data is distributed over the entire range of
|
||||||
|
* possible doubles, including NaN, positive and negative infinity and positive
|
||||||
|
* and negative zero.
|
||||||
|
*/
|
||||||
|
public class BinaryDoubleGenerator extends OraOopTestDataGenerator<Double> {
|
||||||
|
@Override
|
||||||
|
public Double next() {
|
||||||
|
return Double.longBitsToDouble(rng.nextLong());
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,32 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle.util;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generates Float test data. Test data is distributed over the entire range of
|
||||||
|
* possible floats, including NaN, positive and negative infinity and positive
|
||||||
|
* and negative zero.
|
||||||
|
*/
|
||||||
|
public class BinaryFloatGenerator extends OraOopTestDataGenerator<Float> {
|
||||||
|
@Override
|
||||||
|
public Float next() {
|
||||||
|
return Float.intBitsToFloat(rng.nextInt());
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
103
src/test/org/apache/sqoop/manager/oracle/util/BlobGenerator.java
Normal file
103
src/test/org/apache/sqoop/manager/oracle/util/BlobGenerator.java
Normal file
@ -0,0 +1,103 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle.util;
|
||||||
|
|
||||||
|
import java.io.ByteArrayInputStream;
|
||||||
|
import java.io.InputStream;
|
||||||
|
import java.io.OutputStream;
|
||||||
|
import java.lang.reflect.Method;
|
||||||
|
import java.sql.Blob;
|
||||||
|
import java.sql.Connection;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generates Blob test data.
|
||||||
|
*/
|
||||||
|
public class BlobGenerator extends OraOopTestDataGenerator<Blob> {
|
||||||
|
private static Class<?> blobClass;
|
||||||
|
private static Method methCreateTemporary;
|
||||||
|
private static Method methGetBufferSize;
|
||||||
|
private static int durationSession;
|
||||||
|
|
||||||
|
static {
|
||||||
|
try {
|
||||||
|
blobClass = Class.forName("oracle.sql.BLOB");
|
||||||
|
methCreateTemporary =
|
||||||
|
blobClass.getMethod("createTemporary", Connection.class,
|
||||||
|
boolean.class, int.class);
|
||||||
|
methGetBufferSize = blobClass.getMethod("getBufferSize");
|
||||||
|
durationSession = blobClass.getField("DURATION_SESSION").getInt(null);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new RuntimeException(
|
||||||
|
"Problem getting Oracle JDBC methods via reflection.", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private Connection conn;
|
||||||
|
private int minBytes;
|
||||||
|
private int maxBytes;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a generator that will generate BLOBs with length varying between
|
||||||
|
* minBytes and maxBytes.
|
||||||
|
*
|
||||||
|
* @param conn
|
||||||
|
* Oracle connection to use when creating BLOBs
|
||||||
|
* @param minBytes
|
||||||
|
* Minimum number of bytes in generated BLOBs
|
||||||
|
* @param maxBytes
|
||||||
|
* Maximum number of bytes in generated BLOBs
|
||||||
|
*/
|
||||||
|
public BlobGenerator(Connection conn, int minBytes, int maxBytes) {
|
||||||
|
super();
|
||||||
|
this.conn = conn;
|
||||||
|
this.minBytes = minBytes;
|
||||||
|
this.maxBytes = maxBytes;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Blob next() {
|
||||||
|
try {
|
||||||
|
Blob blob =
|
||||||
|
(Blob) methCreateTemporary.invoke(null, conn, false, durationSession);
|
||||||
|
|
||||||
|
int blobSize =
|
||||||
|
(int) (rng.nextDouble() * (maxBytes - minBytes) + minBytes);
|
||||||
|
byte[] blobData = new byte[blobSize];
|
||||||
|
rng.nextBytes(blobData);
|
||||||
|
|
||||||
|
// blob.setBytes(blobData);
|
||||||
|
|
||||||
|
OutputStream os = blob.setBinaryStream(1);
|
||||||
|
InputStream is = new ByteArrayInputStream(blobData);
|
||||||
|
int bufferSize = (Integer) methGetBufferSize.invoke(blob);
|
||||||
|
byte[] buffer = new byte[bufferSize];
|
||||||
|
int bytesRead = 0;
|
||||||
|
while ((bytesRead = is.read(buffer)) != -1) {
|
||||||
|
os.write(buffer, 0, bytesRead);
|
||||||
|
}
|
||||||
|
os.close();
|
||||||
|
is.close();
|
||||||
|
|
||||||
|
return blob;
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,52 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle.util;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generates Blob test data.
|
||||||
|
*/
|
||||||
|
public class BytesGenerator extends OraOopTestDataGenerator<byte[]> {
|
||||||
|
private int minBytes;
|
||||||
|
private int maxBytes;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a generator that will generate arrays of bytes with length varying
|
||||||
|
* between minBytes and maxBytes.
|
||||||
|
*
|
||||||
|
* @param minBytes
|
||||||
|
* Minimum number of bytes in generated RAWs
|
||||||
|
* @param maxBytes
|
||||||
|
* Maximum number of bytes in generated RAWs
|
||||||
|
*/
|
||||||
|
public BytesGenerator(int minBytes, int maxBytes) {
|
||||||
|
super();
|
||||||
|
this.minBytes = minBytes;
|
||||||
|
this.maxBytes = maxBytes;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] next() {
|
||||||
|
int rawSize = (int) (rng.nextDouble() * (maxBytes - minBytes) + minBytes);
|
||||||
|
byte[] rawData = new byte[rawSize];
|
||||||
|
rng.nextBytes(rawData);
|
||||||
|
|
||||||
|
return rawData;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,54 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle.util;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generates String test data. All generated characters will be encodable in
|
||||||
|
* US-ASCII.
|
||||||
|
*/
|
||||||
|
public class CharGenerator extends OraOopTestDataGenerator<String> {
|
||||||
|
private int minLength;
|
||||||
|
private int maxLength;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a CharGenerator that will generate Strings between minLength and
|
||||||
|
* maxLength in length.
|
||||||
|
*
|
||||||
|
* @param minLength
|
||||||
|
* Minimum length for generated strings
|
||||||
|
* @param maxLength
|
||||||
|
* Maximum length for generated strings
|
||||||
|
*/
|
||||||
|
public CharGenerator(int minLength, int maxLength) {
|
||||||
|
super();
|
||||||
|
this.minLength = minLength;
|
||||||
|
this.maxLength = maxLength;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String next() {
|
||||||
|
int length = minLength + rng.nextInt(maxLength - minLength + 1);
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
while (sb.length() < length) {
|
||||||
|
sb.append(Character.toChars(rng.nextInt(128)));
|
||||||
|
}
|
||||||
|
return sb.toString().substring(0, length);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,57 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle.util;
|
||||||
|
|
||||||
|
import java.math.BigDecimal;
|
||||||
|
import java.math.BigInteger;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generates test data for Oracle FLOAT columns.
|
||||||
|
*/
|
||||||
|
public class FloatGenerator extends OraOopTestDataGenerator<BigDecimal> {
|
||||||
|
private static final int MIN_SCALE = -125;
|
||||||
|
private static final int MAX_SCALE = 125;
|
||||||
|
private final int precision;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a float generator with the specified binary precision.
|
||||||
|
*
|
||||||
|
* @param precision
|
||||||
|
* The number of bits in the value of generated numbers
|
||||||
|
*/
|
||||||
|
public FloatGenerator(int precision) {
|
||||||
|
super();
|
||||||
|
this.precision = precision;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public BigDecimal next() {
|
||||||
|
BigInteger unscaled = new BigInteger(precision, rng);
|
||||||
|
BigDecimal unscaledBD = new BigDecimal(unscaled);
|
||||||
|
int scale =
|
||||||
|
rng.nextInt(MAX_SCALE - MIN_SCALE + 1) + MIN_SCALE
|
||||||
|
- unscaledBD.precision();
|
||||||
|
BigDecimal result = new BigDecimal(unscaled, -scale);
|
||||||
|
if (rng.nextBoolean()) {
|
||||||
|
result = result.negate();
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -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);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,64 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle.util;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generates test data for Oracle INTERVAL DAY TO SECOND columns.
|
||||||
|
*/
|
||||||
|
public class IntervalDaySecondGenerator extends
|
||||||
|
OraOopTestDataGenerator<String> {
|
||||||
|
private final int daysPrecision;
|
||||||
|
private final int minDays;
|
||||||
|
private final int maxDays;
|
||||||
|
private final int secondsPrecision;
|
||||||
|
private final int maxFractionalSeconds;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a generator that will generate intervals with the specified
|
||||||
|
* precision for days and seconds.
|
||||||
|
*
|
||||||
|
* @param daysPrecision
|
||||||
|
* Number of decimal digits in the days part of each interval
|
||||||
|
* @param secondsPrecision
|
||||||
|
* Number of decimal digits after the decimal point in seconds part
|
||||||
|
* of each interval.
|
||||||
|
*/
|
||||||
|
public IntervalDaySecondGenerator(int daysPrecision, int secondsPrecision) {
|
||||||
|
super();
|
||||||
|
this.daysPrecision = daysPrecision;
|
||||||
|
this.minDays = -(int) Math.pow(10, daysPrecision) + 1;
|
||||||
|
this.maxDays = (int) Math.pow(10, daysPrecision) - 1;
|
||||||
|
this.secondsPrecision = secondsPrecision;
|
||||||
|
this.maxFractionalSeconds = (int) Math.pow(10, secondsPrecision);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String next() {
|
||||||
|
int days = minDays + rng.nextInt(maxDays - minDays + 1);
|
||||||
|
int hours = rng.nextInt(24);
|
||||||
|
int minutes = rng.nextInt(60);
|
||||||
|
int seconds = rng.nextInt(60);
|
||||||
|
int fractionalSeconds = rng.nextInt(maxFractionalSeconds);
|
||||||
|
String val =
|
||||||
|
String.format("%+0" + daysPrecision + "d %02d:%02d:%02d.%0"
|
||||||
|
+ secondsPrecision + "d", days, hours, minutes, seconds,
|
||||||
|
fractionalSeconds);
|
||||||
|
return val;
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,50 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle.util;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generates test data for Oracle INTERVAL YEAR TO MONTH columns.
|
||||||
|
*/
|
||||||
|
public class IntervalYearMonthGenerator extends
|
||||||
|
OraOopTestDataGenerator<String> {
|
||||||
|
private final int precision;
|
||||||
|
private final int minYear;
|
||||||
|
private final int maxYear;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a generator that will generate intervals with the specified
|
||||||
|
* precision for years.
|
||||||
|
*
|
||||||
|
* @param precision
|
||||||
|
* Number of decimal digits in the years part of each interval
|
||||||
|
*/
|
||||||
|
public IntervalYearMonthGenerator(int precision) {
|
||||||
|
super();
|
||||||
|
this.precision = precision;
|
||||||
|
this.minYear = -(int) Math.pow(10, precision) + 1;
|
||||||
|
this.maxYear = (int) Math.pow(10, precision) - 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String next() {
|
||||||
|
int years = minYear + rng.nextInt(maxYear - minYear + 1);
|
||||||
|
int months = rng.nextInt(12);
|
||||||
|
return String.format("%+0" + precision + "d-%02d", years, months);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,54 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle.util;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generates String test data. All generated characters will be encodable in
|
||||||
|
* UTF-8.
|
||||||
|
*/
|
||||||
|
public class NCharGenerator extends OraOopTestDataGenerator<String> {
|
||||||
|
private int minLength;
|
||||||
|
private int maxLength;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create an NCharGenerator that will generate Strings between minLength and
|
||||||
|
* maxLength in length.
|
||||||
|
*
|
||||||
|
* @param minLength
|
||||||
|
* Minimum length for generated strings
|
||||||
|
* @param maxLength
|
||||||
|
* Maximum length for generated strings
|
||||||
|
*/
|
||||||
|
public NCharGenerator(int minLength, int maxLength) {
|
||||||
|
super();
|
||||||
|
this.minLength = minLength;
|
||||||
|
this.maxLength = maxLength;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String next() {
|
||||||
|
int length = minLength + rng.nextInt(maxLength - minLength + 1);
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
while (sb.length() < length) {
|
||||||
|
sb.append(Character.toChars(rng.nextInt(0x10FFFF)));
|
||||||
|
}
|
||||||
|
return sb.toString().substring(0, length);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,67 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle.util;
|
||||||
|
|
||||||
|
import java.util.Random;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Abstract framework class for generating test data.
|
||||||
|
*
|
||||||
|
* @param <T>
|
||||||
|
* The type that will be generated
|
||||||
|
*/
|
||||||
|
public abstract class OraOopTestDataGenerator<T> {
|
||||||
|
protected Random rng;
|
||||||
|
private long seed;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Initialise with a default seed for the random number generator.
|
||||||
|
*/
|
||||||
|
public OraOopTestDataGenerator() {
|
||||||
|
this(0);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Initialise with a given seed for the random number generator.
|
||||||
|
*
|
||||||
|
* @param seed
|
||||||
|
* The seed to initialise the rng with.
|
||||||
|
*/
|
||||||
|
public OraOopTestDataGenerator(long seed) {
|
||||||
|
this.seed = seed;
|
||||||
|
if (seed == 0) {
|
||||||
|
rng = new Random();
|
||||||
|
} else {
|
||||||
|
rng = new Random(seed);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Reset the rng to its initial state.
|
||||||
|
*/
|
||||||
|
public void reset() {
|
||||||
|
rng = new Random(seed);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The next item of test data. The same sequence will be re-generated
|
||||||
|
* after a call to reset.
|
||||||
|
*/
|
||||||
|
public abstract T next();
|
||||||
|
}
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
192
src/test/org/apache/sqoop/manager/oracle/util/OracleData.java
Normal file
192
src/test/org/apache/sqoop/manager/oracle/util/OracleData.java
Normal file
@ -0,0 +1,192 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle.util;
|
||||||
|
|
||||||
|
import java.net.URL;
|
||||||
|
import java.sql.CallableStatement;
|
||||||
|
import java.sql.Connection;
|
||||||
|
import java.sql.PreparedStatement;
|
||||||
|
import java.sql.ResultSet;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import org.apache.commons.io.IOUtils;
|
||||||
|
import org.apache.sqoop.manager.oracle.OraOopTestCase;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Class to load an Oracle table with data based on configuration file.
|
||||||
|
*/
|
||||||
|
public final class OracleData {
|
||||||
|
private OracleData() {
|
||||||
|
}
|
||||||
|
|
||||||
|
enum KeyType {
|
||||||
|
PRIMARY, UNIQUE
|
||||||
|
}
|
||||||
|
|
||||||
|
private static ClassLoader classLoader;
|
||||||
|
static {
|
||||||
|
classLoader = Thread.currentThread().getContextClassLoader();
|
||||||
|
if (classLoader == null) {
|
||||||
|
classLoader = OraOopTestCase.class.getClassLoader();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static String getColumnList(List<OracleDataDefinition> columnList) {
|
||||||
|
StringBuilder result = new StringBuilder();
|
||||||
|
String delim = "";
|
||||||
|
for (OracleDataDefinition column : columnList) {
|
||||||
|
result.append(delim).append(column.getColumnName()).append(" ").append(
|
||||||
|
column.getDataType());
|
||||||
|
delim = ",\n";
|
||||||
|
}
|
||||||
|
return result.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static String
|
||||||
|
getDataExpression(List<OracleDataDefinition> columnList) {
|
||||||
|
StringBuilder result = new StringBuilder();
|
||||||
|
for (OracleDataDefinition column : columnList) {
|
||||||
|
result.append("l_ret_rec.").append(column.getColumnName()).append(" := ")
|
||||||
|
.append(column.getDataExpression()).append(";\n");
|
||||||
|
}
|
||||||
|
return result.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void createPackageSpec(Connection conn,
|
||||||
|
OracleTableDefinition tableDefinition) throws Exception {
|
||||||
|
String pkgSql =
|
||||||
|
IOUtils.toString(classLoader.getResource(
|
||||||
|
"oraoop/pkg_tst_product_gen.psk").openStream());
|
||||||
|
pkgSql =
|
||||||
|
pkgSql.replaceAll("\\$COLUMN_LIST", getColumnList(tableDefinition
|
||||||
|
.getColumnList()));
|
||||||
|
pkgSql = pkgSql.replaceAll("\\$TABLE_NAME", tableDefinition.getTableName());
|
||||||
|
PreparedStatement stmt = conn.prepareStatement(pkgSql);
|
||||||
|
stmt.execute();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void createPackageBody(Connection conn,
|
||||||
|
OracleTableDefinition tableDefinition) throws Exception {
|
||||||
|
String pkgSql =
|
||||||
|
IOUtils.toString(classLoader.getResource(
|
||||||
|
"oraoop/pkg_tst_product_gen.pbk").openStream());
|
||||||
|
pkgSql =
|
||||||
|
pkgSql.replaceAll("\\$COLUMN_LIST", getColumnList(tableDefinition
|
||||||
|
.getColumnList()));
|
||||||
|
pkgSql = pkgSql.replaceAll("\\$TABLE_NAME", tableDefinition.getTableName());
|
||||||
|
pkgSql =
|
||||||
|
pkgSql.replaceAll("\\$DATA_EXPRESSION_LIST",
|
||||||
|
getDataExpression(tableDefinition.getColumnList()));
|
||||||
|
pkgSql =
|
||||||
|
pkgSql.replaceAll("\\$PARTITION_CLAUSE", tableDefinition
|
||||||
|
.getPartitionClause());
|
||||||
|
PreparedStatement stmt = conn.prepareStatement(pkgSql);
|
||||||
|
stmt.execute();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void createKey(Connection conn, KeyType keyType,
|
||||||
|
OracleTableDefinition tableDefinition) throws Exception {
|
||||||
|
List<String> columns = null;
|
||||||
|
switch (keyType) {
|
||||||
|
case PRIMARY:
|
||||||
|
columns = tableDefinition.getPrimaryKeyColumns();
|
||||||
|
break;
|
||||||
|
case UNIQUE:
|
||||||
|
columns = tableDefinition.getUniqueKeyColumns();
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw new RuntimeException("Missing key type.");
|
||||||
|
}
|
||||||
|
if (columns != null && columns.size() > 0) {
|
||||||
|
StringBuilder keyColumnList = new StringBuilder();
|
||||||
|
String delim = "";
|
||||||
|
for (String column : columns) {
|
||||||
|
keyColumnList.append(delim).append(column);
|
||||||
|
delim = ",";
|
||||||
|
}
|
||||||
|
String keySql =
|
||||||
|
"alter table \"$TABLE_NAME\" add constraint \"$TABLE_NAME_"
|
||||||
|
+ ((keyType == KeyType.PRIMARY) ? "PK\" primary key"
|
||||||
|
: "UK\" unique") + "($PK_COLUMN_LIST) "
|
||||||
|
+ "using index (create unique index \"$TABLE_NAME_"
|
||||||
|
+ ((keyType == KeyType.PRIMARY) ? "PK\"" : "UK\"")
|
||||||
|
+ " on \"$TABLE_NAME\"($PK_COLUMN_LIST) " + "parallel nologging)";
|
||||||
|
keySql =
|
||||||
|
keySql.replaceAll("\\$TABLE_NAME", tableDefinition.getTableName());
|
||||||
|
keySql = keySql.replaceAll("\\$PK_COLUMN_LIST", keyColumnList.toString());
|
||||||
|
PreparedStatement stmt = conn.prepareStatement(keySql);
|
||||||
|
stmt.execute();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static int getParallelProcesses(Connection conn) throws Exception {
|
||||||
|
PreparedStatement stmt =
|
||||||
|
conn.prepareStatement("SELECT cc.value value"
|
||||||
|
+ "\n"
|
||||||
|
+ "FROM"
|
||||||
|
+ "\n"
|
||||||
|
+ " (SELECT to_number(value) value"
|
||||||
|
+ "\n"
|
||||||
|
+ " FROM v$parameter"
|
||||||
|
+ "\n"
|
||||||
|
+ " WHERE name='parallel_max_servers'"
|
||||||
|
+ "\n"
|
||||||
|
+ " ) pms,"
|
||||||
|
+ "\n"
|
||||||
|
+ " (SELECT to_number(value) value"
|
||||||
|
+ "\n"
|
||||||
|
+ " FROM v$parameter"
|
||||||
|
+ "\n"
|
||||||
|
+ " WHERE name='parallel_threads_per_cpu'"
|
||||||
|
+ "\n"
|
||||||
|
+ " ) ptpc,"
|
||||||
|
+ "\n"
|
||||||
|
+ " (SELECT to_number(value) value FROM v$parameter "
|
||||||
|
+ " WHERE name='cpu_count'"
|
||||||
|
+ "\n" + " ) cc");
|
||||||
|
ResultSet res = stmt.executeQuery();
|
||||||
|
res.next();
|
||||||
|
return res.getInt(1);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void createTable(Connection conn,
|
||||||
|
OracleTableDefinition tableDefinition, int parallelDegree,
|
||||||
|
int rowsPerSlave) throws Exception {
|
||||||
|
createPackageSpec(conn, tableDefinition);
|
||||||
|
createPackageBody(conn, tableDefinition);
|
||||||
|
|
||||||
|
CallableStatement procStmt =
|
||||||
|
conn.prepareCall("begin \"PKG_ODG_" + tableDefinition.getTableName()
|
||||||
|
+ "\".prc_load_table(?,?); end;");
|
||||||
|
procStmt.setInt(1, parallelDegree);
|
||||||
|
procStmt.setInt(2, rowsPerSlave);
|
||||||
|
procStmt.execute();
|
||||||
|
|
||||||
|
createKey(conn, KeyType.PRIMARY, tableDefinition);
|
||||||
|
createKey(conn, KeyType.UNIQUE, tableDefinition);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void createTable(Connection conn, String fileName,
|
||||||
|
int parallelDegree, int rowsPerSlave) throws Exception {
|
||||||
|
URL file = classLoader.getResource("oraoop/" + fileName);
|
||||||
|
OracleTableDefinition tableDefinition = new OracleTableDefinition(file);
|
||||||
|
createTable(conn, tableDefinition, parallelDegree, rowsPerSlave);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -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();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,150 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle.util;
|
||||||
|
|
||||||
|
import java.io.File;
|
||||||
|
import java.net.URL;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import javax.xml.parsers.DocumentBuilder;
|
||||||
|
import javax.xml.parsers.DocumentBuilderFactory;
|
||||||
|
import org.w3c.dom.Document;
|
||||||
|
import org.w3c.dom.Element;
|
||||||
|
import org.w3c.dom.Node;
|
||||||
|
import org.w3c.dom.NodeList;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Holds table definition for generated Oracle table.
|
||||||
|
*/
|
||||||
|
public class OracleTableDefinition {
|
||||||
|
|
||||||
|
private String tableName;
|
||||||
|
private List<OracleDataDefinition> columnList =
|
||||||
|
new ArrayList<OracleDataDefinition>();
|
||||||
|
private List<String> primaryKeyColumns = new ArrayList<String>();
|
||||||
|
private List<String> uniqueKeyColumns = new ArrayList<String>();
|
||||||
|
private String partitionClause;
|
||||||
|
|
||||||
|
public List<String> getUniqueKeyColumns() {
|
||||||
|
return uniqueKeyColumns;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setUniqueKeyColumns(List<String> newUniqueKeyColumns) {
|
||||||
|
this.uniqueKeyColumns = newUniqueKeyColumns;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<String> getPrimaryKeyColumns() {
|
||||||
|
return primaryKeyColumns;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setPrimaryKeyColumns(List<String> newPrimaryKeyColumns) {
|
||||||
|
this.primaryKeyColumns = newPrimaryKeyColumns;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<OracleDataDefinition> getColumnList() {
|
||||||
|
return columnList;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setColumnList(List<OracleDataDefinition> newColumnList) {
|
||||||
|
this.columnList = newColumnList;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getTableName() {
|
||||||
|
return tableName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setTableName(String newTableName) {
|
||||||
|
this.tableName = newTableName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getPartitionClause() {
|
||||||
|
return partitionClause == null ? "" : partitionClause;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setPartitionClause(String newPartitionClause) {
|
||||||
|
this.partitionClause = newPartitionClause;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OracleTableDefinition() {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
public OracleTableDefinition(URL url) {
|
||||||
|
try {
|
||||||
|
DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
|
||||||
|
DocumentBuilder builder = factory.newDocumentBuilder();
|
||||||
|
Document doc = builder.parse(new File(url.toURI()));
|
||||||
|
|
||||||
|
Element table = doc.getDocumentElement();
|
||||||
|
this.tableName =
|
||||||
|
table.getElementsByTagName("name").item(0).getChildNodes().item(0)
|
||||||
|
.getNodeValue();
|
||||||
|
NodeList columns = table.getElementsByTagName("column");
|
||||||
|
for (int i = 0; i < columns.getLength(); i++) {
|
||||||
|
Node columnNode = columns.item(i);
|
||||||
|
if (columnNode.getNodeType() == Node.ELEMENT_NODE) {
|
||||||
|
Element columnElement = (Element) columnNode;
|
||||||
|
String name =
|
||||||
|
columnElement.getElementsByTagName("name").item(0)
|
||||||
|
.getChildNodes().item(0).getNodeValue();
|
||||||
|
String dataType =
|
||||||
|
columnElement.getElementsByTagName("dataType").item(0)
|
||||||
|
.getChildNodes().item(0).getNodeValue();
|
||||||
|
String dataExpression =
|
||||||
|
columnElement.getElementsByTagName("dataExpression").item(0)
|
||||||
|
.getChildNodes().item(0).getNodeValue();
|
||||||
|
this.columnList.add(new OracleDataDefinition(name, dataType,
|
||||||
|
dataExpression));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
NodeList primaryKeyColumnsNodeList =
|
||||||
|
table.getElementsByTagName("primaryKeyColumn");
|
||||||
|
for (int i = 0; i < primaryKeyColumnsNodeList.getLength(); i++) {
|
||||||
|
Node primaryKeyColumnNode = primaryKeyColumnsNodeList.item(i);
|
||||||
|
if (primaryKeyColumnNode.getNodeType() == Node.ELEMENT_NODE) {
|
||||||
|
Element primaryKeyColumnElement = (Element) primaryKeyColumnNode;
|
||||||
|
this.primaryKeyColumns.add(primaryKeyColumnElement.getChildNodes()
|
||||||
|
.item(0).getNodeValue());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
NodeList uniqueKeyColumnsNodeList =
|
||||||
|
table.getElementsByTagName("uniqueKeyColumn");
|
||||||
|
for (int i = 0; i < uniqueKeyColumnsNodeList.getLength(); i++) {
|
||||||
|
Node uniqueKeyColumnNode = uniqueKeyColumnsNodeList.item(i);
|
||||||
|
if (uniqueKeyColumnNode.getNodeType() == Node.ELEMENT_NODE) {
|
||||||
|
Element uniqueKeyColumnElement = (Element) uniqueKeyColumnNode;
|
||||||
|
this.uniqueKeyColumns.add(uniqueKeyColumnElement.getChildNodes()
|
||||||
|
.item(0).getNodeValue());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
Node partitionClauseNode =
|
||||||
|
table.getElementsByTagName("partitionClause").item(0);
|
||||||
|
if (partitionClauseNode != null) {
|
||||||
|
this.partitionClause =
|
||||||
|
partitionClauseNode.getChildNodes().item(0).getNodeValue();
|
||||||
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new RuntimeException("Could not load table configuration", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,64 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle.util;
|
||||||
|
|
||||||
|
import java.lang.reflect.Constructor;
|
||||||
|
import java.nio.charset.Charset;
|
||||||
|
import java.sql.RowId;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generates ROWID test data. ROWIDs are represented by 18 ASCII encoded
|
||||||
|
* characters from the set A-Za-z0-9/+
|
||||||
|
*
|
||||||
|
* Generated ROWIDs are unlikely to represent actual rows in any Oracle
|
||||||
|
* database, so should be used for import/export tests only, and not used to
|
||||||
|
* reference data.
|
||||||
|
*/
|
||||||
|
public class RowIdGenerator extends OraOopTestDataGenerator<RowId> {
|
||||||
|
private static final String VALID_CHARS =
|
||||||
|
"ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789/+";
|
||||||
|
private static final int LENGTH = 18;
|
||||||
|
private static Class<?> rowIdClass;
|
||||||
|
private static Constructor<?> rowIdConstructor;
|
||||||
|
|
||||||
|
static {
|
||||||
|
try {
|
||||||
|
rowIdClass = Class.forName("oracle.sql.ROWID");
|
||||||
|
rowIdConstructor = rowIdClass.getConstructor(byte[].class);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new RuntimeException(
|
||||||
|
"Problem getting Oracle JDBC methods via reflection.", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RowId next() {
|
||||||
|
try {
|
||||||
|
StringBuffer sb = new StringBuffer();
|
||||||
|
while (sb.length() < LENGTH) {
|
||||||
|
sb.append(VALID_CHARS.charAt(rng.nextInt(VALID_CHARS.length())));
|
||||||
|
}
|
||||||
|
return (RowId) rowIdConstructor.newInstance(sb.toString().getBytes(
|
||||||
|
Charset.forName("US-ASCII")));
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,71 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle.util;
|
||||||
|
|
||||||
|
import java.sql.Timestamp;
|
||||||
|
import java.util.Calendar;
|
||||||
|
import java.util.GregorianCalendar;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generates test data for Oracle DATE columns. Generated Timestamps are between
|
||||||
|
* 4711BC and 9999AD.
|
||||||
|
*/
|
||||||
|
public class TimestampGenerator extends OraOopTestDataGenerator<Timestamp> {
|
||||||
|
private static final int NANOS_DIGITS = 9;
|
||||||
|
private static final int MIN_YEAR = -4711;
|
||||||
|
private static final int MAX_YEAR = 9999;
|
||||||
|
private final int precision;
|
||||||
|
private final Calendar cal = Calendar.getInstance();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a TimestampGenerator that will generate Timestamps with a given
|
||||||
|
* precision.
|
||||||
|
*
|
||||||
|
* @param precision
|
||||||
|
* Number of decimal digits after the decimal point in the seconds of
|
||||||
|
* generated Timestamps.
|
||||||
|
*/
|
||||||
|
public TimestampGenerator(int precision) {
|
||||||
|
this.precision = precision;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Timestamp next() {
|
||||||
|
cal.clear();
|
||||||
|
cal.set(Calendar.YEAR, MIN_YEAR + rng.nextInt(MAX_YEAR - MIN_YEAR + 1));
|
||||||
|
cal.set(Calendar.DAY_OF_YEAR, 1 + rng.nextInt(cal
|
||||||
|
.getActualMaximum(Calendar.DAY_OF_YEAR)));
|
||||||
|
cal.set(Calendar.HOUR_OF_DAY, rng.nextInt(24));
|
||||||
|
cal.set(Calendar.MINUTE, rng.nextInt(60));
|
||||||
|
cal.set(Calendar.SECOND, rng.nextInt(
|
||||||
|
cal.getActualMaximum(Calendar.SECOND)));
|
||||||
|
// Workaround for oracle jdbc bugs related to BC leap years
|
||||||
|
if (cal.get(Calendar.ERA) == GregorianCalendar.BC
|
||||||
|
&& cal.get(Calendar.MONTH) == 1 && cal.get(Calendar.DAY_OF_MONTH) >= 28) {
|
||||||
|
return next();
|
||||||
|
}
|
||||||
|
Timestamp timestamp = new Timestamp(cal.getTimeInMillis());
|
||||||
|
if (precision > 0) {
|
||||||
|
int nanos = rng.nextInt((int) Math.pow(10, precision));
|
||||||
|
timestamp.setNanos(nanos * (int) Math.pow(10, NANOS_DIGITS - precision));
|
||||||
|
}
|
||||||
|
return timestamp;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,57 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.sqoop.manager.oracle.util;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generates test data for Oracle UriType columns. Generated Strings can be cast
|
||||||
|
* to URITypes with sys.UriFactory.getUri(value). Generated strings may be
|
||||||
|
* detect as any of HTTPURIType, DBURIType or XDBURIType.
|
||||||
|
*
|
||||||
|
* Generated URIs are unlikely to resolve successfully, so should be used for
|
||||||
|
* import/export tests only, and not used to reference data.
|
||||||
|
*/
|
||||||
|
public class URIGenerator extends OraOopTestDataGenerator<String> {
|
||||||
|
private static final int MIN_LENGTH = 15;
|
||||||
|
private static final int MAX_LENGTH = 30;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String next() {
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
switch (rng.nextInt(3)) {
|
||||||
|
case 0: // Generate a String that will detect as an HTTPURIType
|
||||||
|
sb.append("http://");
|
||||||
|
break;
|
||||||
|
case 1: // Generate a String that will detect as an DBURIType
|
||||||
|
sb.append("/oradb/");
|
||||||
|
break;
|
||||||
|
case 2: // Generate a String that will detect as an XDBURIType
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw new RuntimeException("Invalid number generated.");
|
||||||
|
}
|
||||||
|
|
||||||
|
int length =
|
||||||
|
sb.length() + MIN_LENGTH + rng.nextInt(MAX_LENGTH - MIN_LENGTH + 1);
|
||||||
|
while (sb.length() < length) {
|
||||||
|
sb.append(Character.toChars(rng.nextInt(128)));
|
||||||
|
}
|
||||||
|
return sb.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
Loading…
Reference in New Issue
Block a user