mirror of
https://github.com/apache/sqoop.git
synced 2025-05-03 19:02:36 +08:00
SQOOP-376. Migrate mapreduce package to new namespace.
(Bilung Lee via Arvind Prabhakar) git-svn-id: https://svn.apache.org/repos/asf/incubator/sqoop/trunk@1190489 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
37c9642e7d
commit
37831c6125
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,289 +18,65 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.sql.Connection;
|
import java.sql.Connection;
|
||||||
import java.sql.PreparedStatement;
|
import java.sql.PreparedStatement;
|
||||||
import java.sql.SQLException;
|
|
||||||
import java.util.concurrent.SynchronousQueue;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.mapreduce.JobContext;
|
|
||||||
import org.apache.hadoop.mapreduce.OutputCommitter;
|
|
||||||
import org.apache.hadoop.mapreduce.OutputFormat;
|
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
||||||
import org.apache.hadoop.util.StringUtils;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Abstract OutputFormat class that allows the RecordWriter to buffer
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* up SQL commands which should be executed in a separate thread after
|
|
||||||
* enough commands are created.
|
|
||||||
*
|
|
||||||
* This supports a configurable "spill threshold" at which
|
|
||||||
* point intermediate transactions are committed.
|
|
||||||
*
|
|
||||||
* Uses DBOutputFormat/DBConfiguration for configuring the output.
|
|
||||||
* This is used in conjunction with the abstract AsyncSqlRecordWriter
|
|
||||||
* class.
|
|
||||||
*
|
|
||||||
* Clients of this OutputFormat must implement getRecordWriter(); the
|
|
||||||
* returned RecordWriter is intended to subclass AsyncSqlRecordWriter.
|
|
||||||
*/
|
*/
|
||||||
public abstract class AsyncSqlOutputFormat<K extends SqoopRecord, V>
|
public abstract class AsyncSqlOutputFormat<K extends SqoopRecord, V>
|
||||||
extends OutputFormat<K, V> {
|
extends org.apache.sqoop.mapreduce.AsyncSqlOutputFormat<K, V> {
|
||||||
|
|
||||||
/** conf key: number of rows to export per INSERT statement. */
|
|
||||||
public static final String RECORDS_PER_STATEMENT_KEY =
|
public static final String RECORDS_PER_STATEMENT_KEY =
|
||||||
"sqoop.export.records.per.statement";
|
org.apache.sqoop.mapreduce.AsyncSqlOutputFormat.
|
||||||
|
RECORDS_PER_STATEMENT_KEY;
|
||||||
|
|
||||||
/** conf key: number of INSERT statements to bundle per tx.
|
|
||||||
* If this is set to -1, then a single transaction will be used
|
|
||||||
* per task. Note that each statement may encompass multiple
|
|
||||||
* rows, depending on the value of sqoop.export.records.per.statement.
|
|
||||||
*/
|
|
||||||
public static final String STATEMENTS_PER_TRANSACTION_KEY =
|
public static final String STATEMENTS_PER_TRANSACTION_KEY =
|
||||||
"sqoop.export.statements.per.transaction";
|
org.apache.sqoop.mapreduce.AsyncSqlOutputFormat.
|
||||||
|
STATEMENTS_PER_TRANSACTION_KEY;
|
||||||
|
|
||||||
|
public static final int DEFAULT_RECORDS_PER_STATEMENT =
|
||||||
|
org.apache.sqoop.mapreduce.AsyncSqlOutputFormat.
|
||||||
|
DEFAULT_RECORDS_PER_STATEMENT;
|
||||||
|
|
||||||
|
public static final int DEFAULT_STATEMENTS_PER_TRANSACTION =
|
||||||
|
org.apache.sqoop.mapreduce.AsyncSqlOutputFormat.
|
||||||
|
DEFAULT_STATEMENTS_PER_TRANSACTION;
|
||||||
|
|
||||||
|
public static final int UNLIMITED_STATEMENTS_PER_TRANSACTION =
|
||||||
|
org.apache.sqoop.mapreduce.AsyncSqlOutputFormat.
|
||||||
|
UNLIMITED_STATEMENTS_PER_TRANSACTION;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Default number of records to put in an INSERT statement or
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* other batched update statement.
|
|
||||||
*/
|
*/
|
||||||
public static final int DEFAULT_RECORDS_PER_STATEMENT = 100;
|
public static class AsyncDBOperation
|
||||||
|
extends org.apache.sqoop.mapreduce.AsyncSqlOutputFormat.
|
||||||
|
AsyncDBOperation {
|
||||||
|
|
||||||
/**
|
|
||||||
* Default number of statements to execute before committing the
|
|
||||||
* current transaction.
|
|
||||||
*/
|
|
||||||
public static final int DEFAULT_STATEMENTS_PER_TRANSACTION = 100;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Value for STATEMENTS_PER_TRANSACTION_KEY signifying that we should
|
|
||||||
* not commit until the RecordWriter is being closed, regardless of
|
|
||||||
* the number of statements we execute.
|
|
||||||
*/
|
|
||||||
public static final int UNLIMITED_STATEMENTS_PER_TRANSACTION = -1;
|
|
||||||
|
|
||||||
private static final Log LOG = LogFactory.getLog(AsyncSqlOutputFormat.class);
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public void checkOutputSpecs(JobContext context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public OutputCommitter getOutputCommitter(TaskAttemptContext context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
return new NullOutputCommitter();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Represents a database update operation that should be performed
|
|
||||||
* by an asynchronous background thread.
|
|
||||||
* AsyncDBOperation objects are immutable.
|
|
||||||
* They MAY contain a statement which should be executed. The
|
|
||||||
* statement may also be null.
|
|
||||||
*
|
|
||||||
* They may also set 'commitAndClose' to true. If true, then the
|
|
||||||
* executor of this operation should commit the current
|
|
||||||
* transaction, even if stmt is null, and then stop the executor
|
|
||||||
* thread.
|
|
||||||
*/
|
|
||||||
public static class AsyncDBOperation {
|
|
||||||
private final PreparedStatement stmt;
|
|
||||||
private final boolean isBatch;
|
|
||||||
private final boolean commit;
|
|
||||||
private final boolean stopThread;
|
|
||||||
|
|
||||||
@Deprecated
|
|
||||||
/** Do not use AsyncDBOperation(PreparedStatement s, boolean
|
|
||||||
* commitAndClose, boolean batch). Use AsyncDBOperation(PreparedStatement
|
|
||||||
* s, boolean batch, boolean commit, boolean stopThread) instead.
|
|
||||||
*/
|
|
||||||
public AsyncDBOperation(PreparedStatement s, boolean commitAndClose,
|
public AsyncDBOperation(PreparedStatement s, boolean commitAndClose,
|
||||||
boolean batch) {
|
boolean batch) {
|
||||||
this(s, batch, commitAndClose, commitAndClose);
|
super(s, commitAndClose, batch);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Create an asynchronous database operation.
|
|
||||||
* @param s the statement, if any, to execute.
|
|
||||||
* @param batch is true if this is a batch PreparedStatement, or false
|
|
||||||
* if it's a normal singleton statement.
|
|
||||||
* @param commit is true if this statement should be committed to the
|
|
||||||
* database.
|
|
||||||
* @param stopThread if true, the executor thread should stop after this
|
|
||||||
* operation.
|
|
||||||
*/
|
|
||||||
public AsyncDBOperation(PreparedStatement s, boolean batch,
|
public AsyncDBOperation(PreparedStatement s, boolean batch,
|
||||||
boolean commit, boolean stopThread) {
|
boolean commit, boolean stopThread) {
|
||||||
this.stmt = s;
|
super(s, batch, commit, stopThread);
|
||||||
this.isBatch = batch;
|
|
||||||
this.commit = commit;
|
|
||||||
this.stopThread = stopThread;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @return a statement to run as an update.
|
|
||||||
*/
|
|
||||||
public PreparedStatement getStatement() {
|
|
||||||
return stmt;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return true if the executor should commit the current transaction.
|
|
||||||
* If getStatement() is non-null, the statement is run first.
|
|
||||||
*/
|
|
||||||
public boolean requiresCommit() {
|
|
||||||
return this.commit;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return true if the executor should stop after this command.
|
|
||||||
*/
|
|
||||||
public boolean stop() {
|
|
||||||
return this.stopThread;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return true if this is a batch SQL statement.
|
|
||||||
*/
|
|
||||||
public boolean execAsBatch() {
|
|
||||||
return this.isBatch;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A thread that runs the database interactions asynchronously
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* from the OutputCollector.
|
|
||||||
*/
|
*/
|
||||||
public static class AsyncSqlExecThread extends Thread {
|
public static class AsyncSqlExecThread
|
||||||
|
extends org.apache.sqoop.mapreduce.AsyncSqlOutputFormat.
|
||||||
|
AsyncSqlExecThread{
|
||||||
|
|
||||||
private final Connection conn; // The connection to the database.
|
|
||||||
private SQLException err; // Error from a previously-run statement.
|
|
||||||
|
|
||||||
// How we receive database operations from the RecordWriter.
|
|
||||||
private SynchronousQueue<AsyncDBOperation> opsQueue;
|
|
||||||
|
|
||||||
protected int curNumStatements; // statements executed thus far in the tx.
|
|
||||||
protected final int stmtsPerTx; // statements per transaction.
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Create a new update thread that interacts with the database.
|
|
||||||
* @param conn the connection to use. This must only be used by this
|
|
||||||
* thread.
|
|
||||||
* @param stmtsPerTx the number of statements to execute before committing
|
|
||||||
* the current transaction.
|
|
||||||
*/
|
|
||||||
public AsyncSqlExecThread(Connection conn, int stmtsPerTx) {
|
public AsyncSqlExecThread(Connection conn, int stmtsPerTx) {
|
||||||
this.conn = conn;
|
super(conn, stmtsPerTx);
|
||||||
this.err = null;
|
|
||||||
this.opsQueue = new SynchronousQueue<AsyncDBOperation>();
|
|
||||||
this.stmtsPerTx = stmtsPerTx;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void run() {
|
|
||||||
while (true) {
|
|
||||||
AsyncDBOperation op = null;
|
|
||||||
try {
|
|
||||||
op = opsQueue.take();
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
LOG.warn("Interrupted retrieving from operation queue: "
|
|
||||||
+ StringUtils.stringifyException(ie));
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (null == op) {
|
|
||||||
// This shouldn't be allowed to happen.
|
|
||||||
LOG.warn("Null operation in queue; illegal state.");
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
|
|
||||||
PreparedStatement stmt = op.getStatement();
|
|
||||||
// Synchronize on the connection to ensure it does not conflict
|
|
||||||
// with the prepareStatement() call in the main thread.
|
|
||||||
synchronized (conn) {
|
|
||||||
try {
|
|
||||||
if (null != stmt) {
|
|
||||||
if (op.execAsBatch()) {
|
|
||||||
stmt.executeBatch();
|
|
||||||
} else {
|
|
||||||
stmt.execute();
|
|
||||||
}
|
|
||||||
stmt.close();
|
|
||||||
stmt = null;
|
|
||||||
this.curNumStatements++;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (op.requiresCommit() || (curNumStatements >= stmtsPerTx
|
|
||||||
&& stmtsPerTx != UNLIMITED_STATEMENTS_PER_TRANSACTION)) {
|
|
||||||
LOG.debug("Committing transaction of " + curNumStatements
|
|
||||||
+ " statements");
|
|
||||||
this.conn.commit();
|
|
||||||
this.curNumStatements = 0;
|
|
||||||
}
|
|
||||||
} catch (SQLException sqlE) {
|
|
||||||
setLastError(sqlE);
|
|
||||||
} finally {
|
|
||||||
// Close the statement on our way out if that didn't happen
|
|
||||||
// via the normal execution path.
|
|
||||||
if (null != stmt) {
|
|
||||||
try {
|
|
||||||
stmt.close();
|
|
||||||
} catch (SQLException sqlE) {
|
|
||||||
setLastError(sqlE);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Always check whether we should end the loop, regardless
|
|
||||||
// of the presence of an exception.
|
|
||||||
if (op.stop()) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
} // try .. catch .. finally.
|
|
||||||
} // synchronized (conn)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Allows a user to enqueue the next database operation to run.
|
|
||||||
* Since the connection can only execute a single operation at a time,
|
|
||||||
* the put() method may block if another operation is already underway.
|
|
||||||
* @param op the database operation to perform.
|
|
||||||
*/
|
|
||||||
public void put(AsyncDBOperation op) throws InterruptedException {
|
|
||||||
opsQueue.put(op);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* If a previously-executed statement resulted in an error, post it here.
|
|
||||||
* If the error slot was already filled, then subsequent errors are
|
|
||||||
* squashed until the user calls this method (which clears the error
|
|
||||||
* slot).
|
|
||||||
* @return any SQLException that occurred due to a previously-run
|
|
||||||
* statement.
|
|
||||||
*/
|
|
||||||
public synchronized SQLException getLastError() {
|
|
||||||
SQLException e = this.err;
|
|
||||||
this.err = null;
|
|
||||||
return e;
|
|
||||||
}
|
|
||||||
|
|
||||||
private synchronized void setLastError(SQLException e) {
|
|
||||||
if (this.err == null) {
|
|
||||||
// Just set it.
|
|
||||||
LOG.error("Got exception in update thread: "
|
|
||||||
+ StringUtils.stringifyException(e));
|
|
||||||
this.err = e;
|
|
||||||
} else {
|
|
||||||
// Slot is full. Log it and discard.
|
|
||||||
LOG.error("SQLException in update thread but error slot full: "
|
|
||||||
+ StringUtils.stringifyException(e));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,207 +18,19 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.sql.Connection;
|
|
||||||
import java.sql.PreparedStatement;
|
|
||||||
import java.sql.SQLException;
|
import java.sql.SQLException;
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.mapreduce.RecordWriter;
|
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
|
||||||
import com.cloudera.sqoop.util.LoggingUtils;
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Abstract RecordWriter base class that buffers SqoopRecords to be injected
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* into JDBC SQL PreparedStatements to be executed by the
|
|
||||||
* AsyncSqlOutputFormat's background thread.
|
|
||||||
*
|
|
||||||
* Record objects are buffered before actually performing the INSERT
|
|
||||||
* statements; this requires that the key implement the SqoopRecord interface.
|
|
||||||
*
|
|
||||||
* Uses DBOutputFormat/DBConfiguration for configuring the output.
|
|
||||||
*/
|
*/
|
||||||
public abstract class AsyncSqlRecordWriter<K extends SqoopRecord, V>
|
public abstract class AsyncSqlRecordWriter<K extends SqoopRecord, V>
|
||||||
extends RecordWriter<K, V> {
|
extends org.apache.sqoop.mapreduce.AsyncSqlRecordWriter<K, V> {
|
||||||
|
|
||||||
private static final Log LOG = LogFactory.getLog(AsyncSqlRecordWriter.class);
|
|
||||||
|
|
||||||
private Connection connection;
|
|
||||||
|
|
||||||
private Configuration conf;
|
|
||||||
|
|
||||||
protected final int rowsPerStmt; // rows to insert per statement.
|
|
||||||
|
|
||||||
// Buffer for records to be put into export SQL statements.
|
|
||||||
private List<SqoopRecord> records;
|
|
||||||
|
|
||||||
// Background thread to actually perform the updates.
|
|
||||||
private AsyncSqlOutputFormat.AsyncSqlExecThread execThread;
|
|
||||||
private boolean startedExecThread;
|
|
||||||
|
|
||||||
public AsyncSqlRecordWriter(TaskAttemptContext context)
|
public AsyncSqlRecordWriter(TaskAttemptContext context)
|
||||||
throws ClassNotFoundException, SQLException {
|
throws ClassNotFoundException, SQLException {
|
||||||
this.conf = context.getConfiguration();
|
super(context);
|
||||||
|
|
||||||
this.rowsPerStmt = conf.getInt(
|
|
||||||
AsyncSqlOutputFormat.RECORDS_PER_STATEMENT_KEY,
|
|
||||||
AsyncSqlOutputFormat.DEFAULT_RECORDS_PER_STATEMENT);
|
|
||||||
int stmtsPerTx = conf.getInt(
|
|
||||||
AsyncSqlOutputFormat.STATEMENTS_PER_TRANSACTION_KEY,
|
|
||||||
AsyncSqlOutputFormat.DEFAULT_STATEMENTS_PER_TRANSACTION);
|
|
||||||
|
|
||||||
DBConfiguration dbConf = new DBConfiguration(conf);
|
|
||||||
this.connection = dbConf.getConnection();
|
|
||||||
this.connection.setAutoCommit(false);
|
|
||||||
|
|
||||||
this.records = new ArrayList<SqoopRecord>(this.rowsPerStmt);
|
|
||||||
|
|
||||||
this.execThread = new AsyncSqlOutputFormat.AsyncSqlExecThread(
|
|
||||||
connection, stmtsPerTx);
|
|
||||||
this.execThread.setDaemon(true);
|
|
||||||
this.startedExecThread = false;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Allow subclasses access to the Connection instance we hold.
|
|
||||||
* This Connection is shared with the asynchronous SQL exec thread.
|
|
||||||
* Any uses of the Connection must be synchronized on it.
|
|
||||||
* @return the Connection object used for this SQL transaction.
|
|
||||||
*/
|
|
||||||
protected final Connection getConnection() {
|
|
||||||
return this.connection;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Allow subclasses access to the Configuration.
|
|
||||||
* @return the Configuration for this MapReduc task.
|
|
||||||
*/
|
|
||||||
protected final Configuration getConf() {
|
|
||||||
return this.conf;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Should return 'true' if the PreparedStatements generated by the
|
|
||||||
* RecordWriter are intended to be executed in "batch" mode, or false
|
|
||||||
* if it's just one big statement.
|
|
||||||
*/
|
|
||||||
protected boolean isBatchExec() {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Generate the PreparedStatement object that will be fed into the execution
|
|
||||||
* thread. All parameterized fields of the PreparedStatement must be set in
|
|
||||||
* this method as well; this is usually based on the records collected from
|
|
||||||
* the user in the userRecords list.
|
|
||||||
*
|
|
||||||
* Note that any uses of the Connection object here must be synchronized on
|
|
||||||
* the Connection.
|
|
||||||
*
|
|
||||||
* @param userRecords a list of records that should be injected into SQL
|
|
||||||
* statements.
|
|
||||||
* @return a PreparedStatement to be populated with rows
|
|
||||||
* from the collected record list.
|
|
||||||
*/
|
|
||||||
protected abstract PreparedStatement getPreparedStatement(
|
|
||||||
List<SqoopRecord> userRecords) throws SQLException;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Takes the current contents of 'records' and formats and executes the
|
|
||||||
* INSERT statement.
|
|
||||||
* @param closeConn if true, commits the transaction and closes the
|
|
||||||
* connection.
|
|
||||||
*/
|
|
||||||
private void execUpdate(boolean commit, boolean stopThread)
|
|
||||||
throws InterruptedException, SQLException {
|
|
||||||
|
|
||||||
if (!startedExecThread) {
|
|
||||||
this.execThread.start();
|
|
||||||
this.startedExecThread = true;
|
|
||||||
}
|
|
||||||
|
|
||||||
PreparedStatement stmt = null;
|
|
||||||
boolean successfulPut = false;
|
|
||||||
try {
|
|
||||||
if (records.size() > 0) {
|
|
||||||
stmt = getPreparedStatement(records);
|
|
||||||
this.records.clear();
|
|
||||||
}
|
|
||||||
|
|
||||||
// Pass this operation off to the update thread. This will block if
|
|
||||||
// the update thread is already performing an update.
|
|
||||||
AsyncSqlOutputFormat.AsyncDBOperation op =
|
|
||||||
new AsyncSqlOutputFormat.AsyncDBOperation(stmt, isBatchExec(),
|
|
||||||
commit, stopThread);
|
|
||||||
execThread.put(op);
|
|
||||||
successfulPut = true; // op has been posted to the other thread.
|
|
||||||
} finally {
|
|
||||||
if (!successfulPut && null != stmt) {
|
|
||||||
// We created a statement but failed to enqueue it. Close it.
|
|
||||||
stmt.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Check for any previous SQLException. If one happened, rethrow it here.
|
|
||||||
SQLException lastException = execThread.getLastError();
|
|
||||||
if (null != lastException) {
|
|
||||||
LoggingUtils.logAll(LOG, lastException);
|
|
||||||
throw lastException;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public void close(TaskAttemptContext context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
try {
|
|
||||||
try {
|
|
||||||
execUpdate(true, true);
|
|
||||||
execThread.join();
|
|
||||||
} catch (SQLException sqle) {
|
|
||||||
throw new IOException(sqle);
|
|
||||||
}
|
|
||||||
|
|
||||||
// If we're not leaving on an error return path already,
|
|
||||||
// now that execThread is definitely stopped, check that the
|
|
||||||
// error slot remains empty.
|
|
||||||
SQLException lastErr = execThread.getLastError();
|
|
||||||
if (null != lastErr) {
|
|
||||||
throw new IOException(lastErr);
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
try {
|
|
||||||
closeConnection(context);
|
|
||||||
} catch (SQLException sqle) {
|
|
||||||
throw new IOException(sqle);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public void closeConnection(TaskAttemptContext context)
|
|
||||||
throws SQLException {
|
|
||||||
this.connection.close();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public void write(K key, V value)
|
|
||||||
throws InterruptedException, IOException {
|
|
||||||
try {
|
|
||||||
records.add((SqoopRecord) key.clone());
|
|
||||||
if (records.size() >= this.rowsPerStmt) {
|
|
||||||
execUpdate(false, false);
|
|
||||||
}
|
|
||||||
} catch (CloneNotSupportedException cnse) {
|
|
||||||
throw new IOException("Could not buffer record", cnse);
|
|
||||||
} catch (SQLException sqlException) {
|
|
||||||
throw new IOException(sqlException);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,186 +18,25 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
|
|
||||||
import org.apache.hadoop.mapreduce.Mapper;
|
|
||||||
import org.apache.hadoop.mapreduce.Mapper.Context;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Identity mapper that continuously reports progress via a background thread.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
*/
|
*/
|
||||||
public class AutoProgressMapper<KEYIN, VALIN, KEYOUT, VALOUT>
|
public class AutoProgressMapper<KEYIN, VALIN, KEYOUT, VALOUT>
|
||||||
extends Mapper<KEYIN, VALIN, KEYOUT, VALOUT> {
|
extends org.apache.sqoop.mapreduce.AutoProgressMapper
|
||||||
|
<KEYIN, VALIN, KEYOUT, VALOUT> {
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
AutoProgressMapper.class.getName());
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Total number of millis for which progress will be reported by the
|
|
||||||
* auto-progress thread. If this is zero, then the auto-progress thread will
|
|
||||||
* never voluntarily exit.
|
|
||||||
*/
|
|
||||||
private int maxProgressPeriod;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Number of milliseconds to sleep for between loop iterations. Must be less
|
|
||||||
* than report interval.
|
|
||||||
*/
|
|
||||||
private int sleepInterval;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Number of milliseconds between calls to Reporter.progress().
|
|
||||||
* Should be a multiple of the sleepInterval.
|
|
||||||
*/
|
|
||||||
private int reportInterval;
|
|
||||||
|
|
||||||
public static final String MAX_PROGRESS_PERIOD_KEY =
|
public static final String MAX_PROGRESS_PERIOD_KEY =
|
||||||
"sqoop.mapred.auto.progress.max";
|
org.apache.sqoop.mapreduce.AutoProgressMapper.MAX_PROGRESS_PERIOD_KEY;
|
||||||
public static final String SLEEP_INTERVAL_KEY =
|
public static final String SLEEP_INTERVAL_KEY =
|
||||||
"sqoop.mapred.auto.progress.sleep";
|
org.apache.sqoop.mapreduce.AutoProgressMapper.SLEEP_INTERVAL_KEY;
|
||||||
public static final String REPORT_INTERVAL_KEY =
|
public static final String REPORT_INTERVAL_KEY =
|
||||||
"sqoop.mapred.auto.progress.report";
|
org.apache.sqoop.mapreduce.AutoProgressMapper.REPORT_INTERVAL_KEY;
|
||||||
|
|
||||||
// Sleep for 10 seconds at a time.
|
public static final int DEFAULT_SLEEP_INTERVAL =
|
||||||
static final int DEFAULT_SLEEP_INTERVAL = 10000;
|
org.apache.sqoop.mapreduce.AutoProgressMapper.DEFAULT_SLEEP_INTERVAL;
|
||||||
|
public static final int DEFAULT_REPORT_INTERVAL =
|
||||||
|
org.apache.sqoop.mapreduce.AutoProgressMapper.DEFAULT_REPORT_INTERVAL;
|
||||||
|
public static final int DEFAULT_MAX_PROGRESS =
|
||||||
|
org.apache.sqoop.mapreduce.AutoProgressMapper.DEFAULT_MAX_PROGRESS;
|
||||||
|
|
||||||
// Report progress every 30 seconds.
|
|
||||||
static final int DEFAULT_REPORT_INTERVAL = 30000;
|
|
||||||
|
|
||||||
// Disable max progress, by default.
|
|
||||||
static final int DEFAULT_MAX_PROGRESS = 0;
|
|
||||||
|
|
||||||
private class ProgressThread extends Thread {
|
|
||||||
|
|
||||||
private volatile boolean keepGoing; // While this is true, thread runs.
|
|
||||||
|
|
||||||
private Context context;
|
|
||||||
private long startTimeMillis;
|
|
||||||
private long lastReportMillis;
|
|
||||||
|
|
||||||
public ProgressThread(final Context ctxt) {
|
|
||||||
this.context = ctxt;
|
|
||||||
this.keepGoing = true;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void signalShutdown() {
|
|
||||||
this.keepGoing = false; // volatile update.
|
|
||||||
this.interrupt();
|
|
||||||
}
|
|
||||||
|
|
||||||
public void run() {
|
|
||||||
this.lastReportMillis = System.currentTimeMillis();
|
|
||||||
this.startTimeMillis = this.lastReportMillis;
|
|
||||||
|
|
||||||
final long MAX_PROGRESS = AutoProgressMapper.this.maxProgressPeriod;
|
|
||||||
final long REPORT_INTERVAL = AutoProgressMapper.this.reportInterval;
|
|
||||||
final long SLEEP_INTERVAL = AutoProgressMapper.this.sleepInterval;
|
|
||||||
|
|
||||||
// In a loop:
|
|
||||||
// * Check that we haven't run for too long (maxProgressPeriod).
|
|
||||||
// * If it's been a report interval since we last made progress,
|
|
||||||
// make more.
|
|
||||||
// * Sleep for a bit.
|
|
||||||
// * If the parent thread has signaled for exit, do so.
|
|
||||||
while (this.keepGoing) {
|
|
||||||
long curTimeMillis = System.currentTimeMillis();
|
|
||||||
|
|
||||||
if (MAX_PROGRESS != 0
|
|
||||||
&& curTimeMillis - this.startTimeMillis > MAX_PROGRESS) {
|
|
||||||
this.keepGoing = false;
|
|
||||||
LOG.info("Auto-progress thread exiting after " + MAX_PROGRESS
|
|
||||||
+ " ms.");
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (curTimeMillis - this.lastReportMillis > REPORT_INTERVAL) {
|
|
||||||
// It's been a full report interval -- claim progress.
|
|
||||||
LOG.debug("Auto-progress thread reporting progress");
|
|
||||||
this.context.progress();
|
|
||||||
this.lastReportMillis = curTimeMillis;
|
|
||||||
}
|
|
||||||
|
|
||||||
// Unless we got an interrupt while we were working,
|
|
||||||
// sleep a bit before doing more work.
|
|
||||||
if (!this.interrupted()) {
|
|
||||||
try {
|
|
||||||
Thread.sleep(SLEEP_INTERVAL);
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
// we were notified on something; not necessarily an error.
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
LOG.info("Auto-progress thread is finished. keepGoing=" + this.keepGoing);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Set configuration parameters for the auto-progress thread.
|
|
||||||
*/
|
|
||||||
private void configureAutoProgress(Configuration job) {
|
|
||||||
this.maxProgressPeriod = job.getInt(MAX_PROGRESS_PERIOD_KEY,
|
|
||||||
DEFAULT_MAX_PROGRESS);
|
|
||||||
this.sleepInterval = job.getInt(SLEEP_INTERVAL_KEY,
|
|
||||||
DEFAULT_SLEEP_INTERVAL);
|
|
||||||
this.reportInterval = job.getInt(REPORT_INTERVAL_KEY,
|
|
||||||
DEFAULT_REPORT_INTERVAL);
|
|
||||||
|
|
||||||
if (this.reportInterval < 1) {
|
|
||||||
LOG.warn("Invalid " + REPORT_INTERVAL_KEY + "; setting to "
|
|
||||||
+ DEFAULT_REPORT_INTERVAL);
|
|
||||||
this.reportInterval = DEFAULT_REPORT_INTERVAL;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (this.sleepInterval > this.reportInterval || this.sleepInterval < 1) {
|
|
||||||
LOG.warn("Invalid " + SLEEP_INTERVAL_KEY + "; setting to "
|
|
||||||
+ DEFAULT_SLEEP_INTERVAL);
|
|
||||||
this.sleepInterval = DEFAULT_SLEEP_INTERVAL;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (this.maxProgressPeriod < 0) {
|
|
||||||
LOG.warn("Invalid " + MAX_PROGRESS_PERIOD_KEY + "; setting to "
|
|
||||||
+ DEFAULT_MAX_PROGRESS);
|
|
||||||
this.maxProgressPeriod = DEFAULT_MAX_PROGRESS;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
// map() method intentionally omitted; Mapper.map() is the identity mapper.
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Run the mapping process for this task, wrapped in an auto-progress system.
|
|
||||||
*/
|
|
||||||
@Override
|
|
||||||
public void run(Context context) throws IOException, InterruptedException {
|
|
||||||
configureAutoProgress(context.getConfiguration());
|
|
||||||
ProgressThread thread = this.new ProgressThread(context);
|
|
||||||
|
|
||||||
try {
|
|
||||||
thread.setDaemon(true);
|
|
||||||
thread.start();
|
|
||||||
|
|
||||||
// use default run() method to actually drive the mapping.
|
|
||||||
super.run(context);
|
|
||||||
} finally {
|
|
||||||
// Tell the progress thread to exit..
|
|
||||||
LOG.debug("Instructing auto-progress thread to quit.");
|
|
||||||
thread.signalShutdown();
|
|
||||||
try {
|
|
||||||
// And wait for that to happen.
|
|
||||||
LOG.debug("Waiting for progress thread shutdown...");
|
|
||||||
thread.join();
|
|
||||||
LOG.debug("Progress thread shutdown detected.");
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
LOG.warn("Interrupted when waiting on auto-progress thread: "
|
|
||||||
+ ie.toString());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,188 +18,13 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
|
||||||
import com.cloudera.sqoop.orm.ClassWriter;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.math.BigDecimal;
|
|
||||||
import java.nio.ByteBuffer;
|
|
||||||
import java.sql.Date;
|
|
||||||
import java.sql.Time;
|
|
||||||
import java.sql.Timestamp;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
import org.apache.avro.Schema;
|
|
||||||
import org.apache.avro.Schema.Field;
|
|
||||||
import org.apache.avro.generic.GenericEnumSymbol;
|
|
||||||
import org.apache.avro.generic.GenericFixed;
|
|
||||||
import org.apache.avro.generic.GenericRecord;
|
|
||||||
import org.apache.avro.mapred.AvroWrapper;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.io.BytesWritable;
|
|
||||||
import org.apache.hadoop.io.DefaultStringifier;
|
|
||||||
import org.apache.hadoop.io.MapWritable;
|
|
||||||
import org.apache.hadoop.io.NullWritable;
|
|
||||||
import org.apache.hadoop.io.Writable;
|
|
||||||
import org.apache.hadoop.util.ReflectionUtils;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Exports records from an Avro data file.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
*/
|
*/
|
||||||
public class AvroExportMapper
|
public class AvroExportMapper
|
||||||
extends AutoProgressMapper<AvroWrapper<GenericRecord>, NullWritable,
|
extends org.apache.sqoop.mapreduce.AvroExportMapper {
|
||||||
SqoopRecord, NullWritable> {
|
|
||||||
|
|
||||||
private static final String TIMESTAMP_TYPE = "java.sql.Timestamp";
|
public static final String AVRO_COLUMN_TYPES_MAP =
|
||||||
|
org.apache.sqoop.mapreduce.AvroExportMapper.AVRO_COLUMN_TYPES_MAP;
|
||||||
private static final String TIME_TYPE = "java.sql.Time";
|
|
||||||
|
|
||||||
private static final String DATE_TYPE = "java.sql.Date";
|
|
||||||
|
|
||||||
private static final String BIG_DECIMAL_TYPE = "java.math.BigDecimal";
|
|
||||||
|
|
||||||
static final String AVRO_COLUMN_TYPES_MAP = "sqoop.avro.column.types.map";
|
|
||||||
|
|
||||||
private MapWritable columnTypes;
|
|
||||||
private SqoopRecord recordImpl;
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void setup(Context context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
|
|
||||||
super.setup(context);
|
|
||||||
|
|
||||||
Configuration conf = context.getConfiguration();
|
|
||||||
|
|
||||||
// Instantiate a copy of the user's class to hold and parse the record.
|
|
||||||
String recordClassName = conf.get(
|
|
||||||
ExportJobBase.SQOOP_EXPORT_TABLE_CLASS_KEY);
|
|
||||||
if (null == recordClassName) {
|
|
||||||
throw new IOException("Export table class name ("
|
|
||||||
+ ExportJobBase.SQOOP_EXPORT_TABLE_CLASS_KEY
|
|
||||||
+ ") is not set!");
|
|
||||||
}
|
|
||||||
|
|
||||||
try {
|
|
||||||
Class cls = Class.forName(recordClassName, true,
|
|
||||||
Thread.currentThread().getContextClassLoader());
|
|
||||||
recordImpl = (SqoopRecord) ReflectionUtils.newInstance(cls, conf);
|
|
||||||
} catch (ClassNotFoundException cnfe) {
|
|
||||||
throw new IOException(cnfe);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (null == recordImpl) {
|
|
||||||
throw new IOException("Could not instantiate object of type "
|
|
||||||
+ recordClassName);
|
|
||||||
}
|
|
||||||
|
|
||||||
columnTypes = DefaultStringifier.load(conf, AVRO_COLUMN_TYPES_MAP,
|
|
||||||
MapWritable.class);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void map(AvroWrapper<GenericRecord> key, NullWritable value,
|
|
||||||
Context context) throws IOException, InterruptedException {
|
|
||||||
context.write(toSqoopRecord(key.datum()), NullWritable.get());
|
|
||||||
}
|
|
||||||
|
|
||||||
private SqoopRecord toSqoopRecord(GenericRecord record) throws IOException {
|
|
||||||
Schema avroSchema = record.getSchema();
|
|
||||||
for (Map.Entry<Writable, Writable> e : columnTypes.entrySet()) {
|
|
||||||
String columnName = e.getKey().toString();
|
|
||||||
String columnType = e.getValue().toString();
|
|
||||||
String cleanedCol = ClassWriter.toIdentifier(columnName);
|
|
||||||
Field field = getField(avroSchema, cleanedCol, record);
|
|
||||||
if (field == null) {
|
|
||||||
throw new IOException("Cannot find field " + cleanedCol
|
|
||||||
+ " in Avro schema " + avroSchema);
|
|
||||||
} else {
|
|
||||||
Object avroObject = record.get(field.name());
|
|
||||||
Object fieldVal = fromAvro(avroObject, field.schema(), columnType);
|
|
||||||
recordImpl.setField(cleanedCol, fieldVal);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return recordImpl;
|
|
||||||
}
|
|
||||||
|
|
||||||
private Field getField(Schema avroSchema, String fieldName,
|
|
||||||
GenericRecord record) {
|
|
||||||
for (Field field : avroSchema.getFields()) {
|
|
||||||
if (field.name().equalsIgnoreCase(fieldName)) {
|
|
||||||
return field;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
private Object fromAvro(Object avroObject, Schema fieldSchema,
|
|
||||||
String columnType) {
|
|
||||||
// map from Avro type to Sqoop's Java representation of the SQL type
|
|
||||||
// see SqlManager#toJavaType
|
|
||||||
|
|
||||||
if (avroObject == null) {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
switch (fieldSchema.getType()) {
|
|
||||||
case NULL:
|
|
||||||
return null;
|
|
||||||
case BOOLEAN:
|
|
||||||
case INT:
|
|
||||||
case FLOAT:
|
|
||||||
case DOUBLE:
|
|
||||||
return avroObject;
|
|
||||||
case LONG:
|
|
||||||
if (columnType.equals(DATE_TYPE)) {
|
|
||||||
return new Date((Long) avroObject);
|
|
||||||
} else if (columnType.equals(TIME_TYPE)) {
|
|
||||||
return new Time((Long) avroObject);
|
|
||||||
} else if (columnType.equals(TIMESTAMP_TYPE)) {
|
|
||||||
return new Timestamp((Long) avroObject);
|
|
||||||
}
|
|
||||||
return avroObject;
|
|
||||||
case BYTES:
|
|
||||||
ByteBuffer bb = (ByteBuffer) avroObject;
|
|
||||||
BytesWritable bw = new BytesWritable();
|
|
||||||
bw.set(bb.array(), bb.arrayOffset() + bb.position(), bb.remaining());
|
|
||||||
return bw;
|
|
||||||
case STRING:
|
|
||||||
if (columnType.equals(BIG_DECIMAL_TYPE)) {
|
|
||||||
return new BigDecimal(avroObject.toString());
|
|
||||||
} else if (columnType.equals(DATE_TYPE)) {
|
|
||||||
return Date.valueOf(avroObject.toString());
|
|
||||||
} else if (columnType.equals(TIME_TYPE)) {
|
|
||||||
return Time.valueOf(avroObject.toString());
|
|
||||||
} else if (columnType.equals(TIMESTAMP_TYPE)) {
|
|
||||||
return Timestamp.valueOf(avroObject.toString());
|
|
||||||
}
|
|
||||||
return avroObject.toString();
|
|
||||||
case ENUM:
|
|
||||||
return ((GenericEnumSymbol) avroObject).toString();
|
|
||||||
case UNION:
|
|
||||||
List<Schema> types = fieldSchema.getTypes();
|
|
||||||
if (types.size() != 2) {
|
|
||||||
throw new IllegalArgumentException("Only support union with null");
|
|
||||||
}
|
|
||||||
Schema s1 = types.get(0);
|
|
||||||
Schema s2 = types.get(1);
|
|
||||||
if (s1.getType() == Schema.Type.NULL) {
|
|
||||||
return fromAvro(avroObject, s2, columnType);
|
|
||||||
} else if (s2.getType() == Schema.Type.NULL) {
|
|
||||||
return fromAvro(avroObject, s1, columnType);
|
|
||||||
} else {
|
|
||||||
throw new IllegalArgumentException("Only support union with null");
|
|
||||||
}
|
|
||||||
case FIXED:
|
|
||||||
return new BytesWritable(((GenericFixed) avroObject).bytes());
|
|
||||||
case RECORD:
|
|
||||||
case ARRAY:
|
|
||||||
case MAP:
|
|
||||||
default:
|
|
||||||
throw new IllegalArgumentException("Cannot convert Avro type "
|
|
||||||
+ fieldSchema.getType());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,85 +18,9 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import com.cloudera.sqoop.lib.BlobRef;
|
|
||||||
import com.cloudera.sqoop.lib.ClobRef;
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.math.BigDecimal;
|
|
||||||
import java.nio.ByteBuffer;
|
|
||||||
import java.sql.Date;
|
|
||||||
import java.sql.Time;
|
|
||||||
import java.sql.Timestamp;
|
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
import org.apache.avro.Schema;
|
|
||||||
import org.apache.avro.generic.GenericData;
|
|
||||||
import org.apache.avro.generic.GenericRecord;
|
|
||||||
import org.apache.avro.mapred.AvroWrapper;
|
|
||||||
import org.apache.hadoop.io.BytesWritable;
|
|
||||||
import org.apache.hadoop.io.LongWritable;
|
|
||||||
import org.apache.hadoop.io.NullWritable;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Imports records by transforming them to Avro records in an Avro data file.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
*/
|
*/
|
||||||
public class AvroImportMapper
|
public class AvroImportMapper
|
||||||
extends AutoProgressMapper<LongWritable, SqoopRecord,
|
extends org.apache.sqoop.mapreduce.AvroImportMapper {
|
||||||
AvroWrapper<GenericRecord>, NullWritable> {
|
|
||||||
|
|
||||||
private final AvroWrapper<GenericRecord> wrapper =
|
|
||||||
new AvroWrapper<GenericRecord>();
|
|
||||||
private Schema schema;
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void setup(Context context) {
|
|
||||||
schema = AvroJob.getMapOutputSchema(context.getConfiguration());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void map(LongWritable key, SqoopRecord val, Context context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
wrapper.datum(toGenericRecord(val));
|
|
||||||
context.write(wrapper, NullWritable.get());
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
private GenericRecord toGenericRecord(SqoopRecord val) {
|
|
||||||
Map<String, Object> fieldMap = val.getFieldMap();
|
|
||||||
GenericRecord record = new GenericData.Record(schema);
|
|
||||||
for (Map.Entry<String, Object> entry : fieldMap.entrySet()) {
|
|
||||||
record.put(entry.getKey(), toAvro(entry.getValue()));
|
|
||||||
}
|
|
||||||
return record;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Convert the Avro representation of a Java type (that has already been
|
|
||||||
* converted from the SQL equivalent).
|
|
||||||
* @param o
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
private Object toAvro(Object o) {
|
|
||||||
if (o instanceof BigDecimal) {
|
|
||||||
return o.toString();
|
|
||||||
} else if (o instanceof Date) {
|
|
||||||
return ((Date) o).getTime();
|
|
||||||
} else if (o instanceof Time) {
|
|
||||||
return ((Time) o).getTime();
|
|
||||||
} else if (o instanceof Timestamp) {
|
|
||||||
return ((Timestamp) o).getTime();
|
|
||||||
} else if (o instanceof BytesWritable) {
|
|
||||||
BytesWritable bw = (BytesWritable) o;
|
|
||||||
return ByteBuffer.wrap(bw.getBytes(), 0, bw.getLength());
|
|
||||||
} else if (o instanceof ClobRef) {
|
|
||||||
throw new UnsupportedOperationException("ClobRef not suported");
|
|
||||||
} else if (o instanceof BlobRef) {
|
|
||||||
throw new UnsupportedOperationException("BlobRef not suported");
|
|
||||||
}
|
|
||||||
// primitive types (Integer, etc) are left unchanged
|
|
||||||
return o;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,42 +18,10 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
/**
|
||||||
import java.util.ArrayList;
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
import java.util.List;
|
*/
|
||||||
|
|
||||||
import org.apache.avro.mapred.AvroWrapper;
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
|
||||||
import org.apache.hadoop.io.NullWritable;
|
|
||||||
import org.apache.hadoop.mapreduce.InputSplit;
|
|
||||||
import org.apache.hadoop.mapreduce.JobContext;
|
|
||||||
import org.apache.hadoop.mapreduce.RecordReader;
|
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
|
|
||||||
|
|
||||||
/** An {@link org.apache.hadoop.mapred.InputFormat} for Avro data files. */
|
|
||||||
public class AvroInputFormat<T>
|
public class AvroInputFormat<T>
|
||||||
extends FileInputFormat<AvroWrapper<T>, NullWritable> {
|
extends org.apache.sqoop.mapreduce.AvroInputFormat<T> {
|
||||||
|
|
||||||
@Override
|
|
||||||
protected List<FileStatus> listStatus(JobContext job) throws IOException {
|
|
||||||
List<FileStatus> result = new ArrayList<FileStatus>();
|
|
||||||
for (FileStatus file : super.listStatus(job)) {
|
|
||||||
if (file.getPath().getName().endsWith(
|
|
||||||
org.apache.avro.mapred.AvroOutputFormat.EXT)) {
|
|
||||||
result.add(file);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public RecordReader<AvroWrapper<T>, NullWritable> createRecordReader(
|
|
||||||
InputSplit split, TaskAttemptContext context) throws IOException,
|
|
||||||
InterruptedException {
|
|
||||||
context.setStatus(split.toString());
|
|
||||||
return new AvroRecordReader<T>();
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -24,20 +22,21 @@
|
|||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Helper class for setting up an Avro MapReduce job.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
*/
|
*/
|
||||||
public final class AvroJob {
|
public final class AvroJob {
|
||||||
public static final String MAP_OUTPUT_SCHEMA = "avro.map.output.schema";
|
|
||||||
|
|
||||||
private AvroJob() {
|
public static final String MAP_OUTPUT_SCHEMA =
|
||||||
}
|
org.apache.sqoop.mapreduce.AvroJob.MAP_OUTPUT_SCHEMA;
|
||||||
|
|
||||||
|
private AvroJob() { }
|
||||||
|
|
||||||
public static void setMapOutputSchema(Configuration job, Schema s) {
|
public static void setMapOutputSchema(Configuration job, Schema s) {
|
||||||
job.set(MAP_OUTPUT_SCHEMA, s.toString());
|
org.apache.sqoop.mapreduce.AvroJob.setMapOutputSchema(job, s);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Return a job's map output key schema. */
|
|
||||||
public static Schema getMapOutputSchema(Configuration job) {
|
public static Schema getMapOutputSchema(Configuration job) {
|
||||||
return Schema.parse(job.get(MAP_OUTPUT_SCHEMA));
|
return org.apache.sqoop.mapreduce.AvroJob.getMapOutputSchema(job);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,48 +18,9 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
/**
|
||||||
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
import org.apache.avro.Schema;
|
*/
|
||||||
import org.apache.avro.file.DataFileWriter;
|
|
||||||
import org.apache.avro.generic.GenericDatumWriter;
|
|
||||||
import org.apache.avro.mapred.AvroWrapper;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.io.NullWritable;
|
|
||||||
import org.apache.hadoop.mapreduce.RecordWriter;
|
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
|
||||||
|
|
||||||
/** An {@link org.apache.hadoop.mapred.OutputFormat} for Avro data files. */
|
|
||||||
public class AvroOutputFormat<T>
|
public class AvroOutputFormat<T>
|
||||||
extends FileOutputFormat<AvroWrapper<T>, NullWritable> {
|
extends org.apache.sqoop.mapreduce.AvroOutputFormat<T> {
|
||||||
|
|
||||||
@Override
|
|
||||||
public RecordWriter<AvroWrapper<T>, NullWritable> getRecordWriter(
|
|
||||||
TaskAttemptContext context) throws IOException, InterruptedException {
|
|
||||||
|
|
||||||
Schema schema = AvroJob.getMapOutputSchema(context.getConfiguration());
|
|
||||||
|
|
||||||
final DataFileWriter<T> WRITER =
|
|
||||||
new DataFileWriter<T>(new GenericDatumWriter<T>());
|
|
||||||
|
|
||||||
Path path = getDefaultWorkFile(context,
|
|
||||||
org.apache.avro.mapred.AvroOutputFormat.EXT);
|
|
||||||
WRITER.create(schema,
|
|
||||||
path.getFileSystem(context.getConfiguration()).create(path));
|
|
||||||
|
|
||||||
return new RecordWriter<AvroWrapper<T>, NullWritable>() {
|
|
||||||
@Override
|
|
||||||
public void write(AvroWrapper<T> wrapper, NullWritable ignore)
|
|
||||||
throws IOException {
|
|
||||||
WRITER.append(wrapper.datum());
|
|
||||||
}
|
|
||||||
@Override
|
|
||||||
public void close(TaskAttemptContext context) throws IOException,
|
|
||||||
InterruptedException {
|
|
||||||
WRITER.close();
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,88 +18,10 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
/**
|
||||||
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
import org.apache.avro.file.DataFileReader;
|
*/
|
||||||
import org.apache.avro.file.FileReader;
|
|
||||||
import org.apache.avro.file.SeekableInput;
|
|
||||||
import org.apache.avro.generic.GenericDatumReader;
|
|
||||||
import org.apache.avro.io.DatumReader;
|
|
||||||
import org.apache.avro.mapred.AvroWrapper;
|
|
||||||
import org.apache.avro.mapred.FsInput;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.io.NullWritable;
|
|
||||||
import org.apache.hadoop.mapreduce.InputSplit;
|
|
||||||
import org.apache.hadoop.mapreduce.RecordReader;
|
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.input.FileSplit;
|
|
||||||
|
|
||||||
/** An {@link RecordReader} for Avro data files. */
|
|
||||||
public class AvroRecordReader<T>
|
public class AvroRecordReader<T>
|
||||||
extends RecordReader<AvroWrapper<T>, NullWritable> {
|
extends org.apache.sqoop.mapreduce.AvroRecordReader<T> {
|
||||||
|
|
||||||
private FileReader<T> reader;
|
|
||||||
private long start;
|
|
||||||
private long end;
|
|
||||||
private AvroWrapper<T> key;
|
|
||||||
private NullWritable value;
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void initialize(InputSplit genericSplit, TaskAttemptContext context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
FileSplit split = (FileSplit) genericSplit;
|
|
||||||
Configuration conf = context.getConfiguration();
|
|
||||||
SeekableInput in = new FsInput(split.getPath(), conf);
|
|
||||||
DatumReader<T> datumReader = new GenericDatumReader<T>();
|
|
||||||
this.reader = DataFileReader.openReader(in, datumReader);
|
|
||||||
reader.sync(split.getStart()); // sync to start
|
|
||||||
this.start = reader.tell();
|
|
||||||
this.end = split.getStart() + split.getLength();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean nextKeyValue() throws IOException, InterruptedException {
|
|
||||||
if (!reader.hasNext() || reader.pastSync(end)) {
|
|
||||||
key = null;
|
|
||||||
value = null;
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
if (key == null) {
|
|
||||||
key = new AvroWrapper<T>();
|
|
||||||
}
|
|
||||||
if (value == null) {
|
|
||||||
value = NullWritable.get();
|
|
||||||
}
|
|
||||||
key.datum(reader.next(key.datum()));
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public AvroWrapper<T> getCurrentKey() throws IOException,
|
|
||||||
InterruptedException {
|
|
||||||
return key;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public NullWritable getCurrentValue()
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
return value;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public float getProgress() throws IOException {
|
|
||||||
if (end == start) {
|
|
||||||
return 0.0f;
|
|
||||||
} else {
|
|
||||||
return Math.min(1.0f, (getPos() - start) / (float)(end - start));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public long getPos() throws IOException {
|
|
||||||
return reader.tell();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void close() throws IOException { reader.close(); }
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -21,115 +19,19 @@
|
|||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.io.LongWritable;
|
|
||||||
import org.apache.hadoop.mapreduce.InputSplit;
|
|
||||||
import org.apache.hadoop.mapreduce.RecordReader;
|
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit;
|
import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit;
|
||||||
import org.apache.hadoop.mapreduce.lib.input.FileSplit;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.input.LineRecordReader;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.input.SequenceFileRecordReader;
|
|
||||||
import org.apache.hadoop.util.ReflectionUtils;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* RecordReader that CombineFileRecordReader can instantiate, which itself
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* translates a CombineFileSplit into a FileSplit.
|
|
||||||
*/
|
*/
|
||||||
public class CombineShimRecordReader
|
public class CombineShimRecordReader
|
||||||
extends RecordReader<LongWritable, Object> {
|
extends org.apache.sqoop.mapreduce.CombineShimRecordReader {
|
||||||
|
|
||||||
public static final Log LOG =
|
|
||||||
LogFactory.getLog(CombineShimRecordReader.class.getName());
|
|
||||||
|
|
||||||
private CombineFileSplit split;
|
|
||||||
private TaskAttemptContext context;
|
|
||||||
private int index;
|
|
||||||
private RecordReader<LongWritable, Object> rr;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Constructor invoked by CombineFileRecordReader that identifies part of a
|
|
||||||
* CombineFileSplit to use.
|
|
||||||
*/
|
|
||||||
public CombineShimRecordReader(CombineFileSplit split,
|
public CombineShimRecordReader(CombineFileSplit split,
|
||||||
TaskAttemptContext context, Integer index)
|
TaskAttemptContext context, Integer index)
|
||||||
throws IOException, InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
this.index = index;
|
super(split, context, index);
|
||||||
this.split = (CombineFileSplit) split;
|
|
||||||
this.context = context;
|
|
||||||
|
|
||||||
createChildReader();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public void initialize(InputSplit curSplit, TaskAttemptContext curContext)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
this.split = (CombineFileSplit) curSplit;
|
|
||||||
this.context = curContext;
|
|
||||||
|
|
||||||
if (null == rr) {
|
|
||||||
createChildReader();
|
|
||||||
}
|
|
||||||
|
|
||||||
FileSplit fileSplit = new FileSplit(this.split.getPath(index),
|
|
||||||
this.split.getOffset(index), this.split.getLength(index),
|
|
||||||
this.split.getLocations());
|
|
||||||
this.rr.initialize(fileSplit, this.context);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public float getProgress() throws IOException, InterruptedException {
|
|
||||||
return rr.getProgress();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void close() throws IOException {
|
|
||||||
if (null != rr) {
|
|
||||||
rr.close();
|
|
||||||
rr = null;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public LongWritable getCurrentKey()
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
return rr.getCurrentKey();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Object getCurrentValue()
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
return rr.getCurrentValue();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean nextKeyValue() throws IOException, InterruptedException {
|
|
||||||
return rr.nextKeyValue();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Actually instantiate the user's chosen RecordReader implementation.
|
|
||||||
*/
|
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
private void createChildReader() throws IOException, InterruptedException {
|
|
||||||
LOG.debug("ChildSplit operates on: " + split.getPath(index));
|
|
||||||
|
|
||||||
Configuration conf = context.getConfiguration();
|
|
||||||
|
|
||||||
// Determine the file format we're reading.
|
|
||||||
Class rrClass;
|
|
||||||
if (ExportJobBase.isSequenceFiles(conf, split.getPath(index))) {
|
|
||||||
rrClass = SequenceFileRecordReader.class;
|
|
||||||
} else {
|
|
||||||
rrClass = LineRecordReader.class;
|
|
||||||
}
|
|
||||||
|
|
||||||
// Create the appropriate record reader.
|
|
||||||
this.rr = (RecordReader<LongWritable, Object>)
|
|
||||||
ReflectionUtils.newInstance(rrClass, conf);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -21,189 +21,24 @@
|
|||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import com.cloudera.sqoop.SqoopOptions;
|
import com.cloudera.sqoop.SqoopOptions;
|
||||||
import com.cloudera.sqoop.config.ConfigurationHelper;
|
|
||||||
import com.cloudera.sqoop.lib.LargeObjectLoader;
|
|
||||||
import com.cloudera.sqoop.manager.ConnManager;
|
|
||||||
import com.cloudera.sqoop.manager.ImportJobContext;
|
import com.cloudera.sqoop.manager.ImportJobContext;
|
||||||
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
|
||||||
import com.cloudera.sqoop.mapreduce.db.DataDrivenDBInputFormat;
|
|
||||||
import com.cloudera.sqoop.orm.AvroSchemaGenerator;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.sql.SQLException;
|
|
||||||
|
|
||||||
import org.apache.avro.Schema;
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.io.NullWritable;
|
|
||||||
import org.apache.hadoop.io.Text;
|
|
||||||
import org.apache.hadoop.mapreduce.InputFormat;
|
import org.apache.hadoop.mapreduce.InputFormat;
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
|
||||||
import org.apache.hadoop.mapreduce.Mapper;
|
|
||||||
import org.apache.hadoop.mapreduce.OutputFormat;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.db.DBWritable;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Actually runs a jdbc import job using the ORM files generated by the
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* sqoop.orm package. Uses DataDrivenDBInputFormat.
|
|
||||||
*/
|
*/
|
||||||
public class DataDrivenImportJob extends ImportJobBase {
|
public class DataDrivenImportJob
|
||||||
|
extends org.apache.sqoop.mapreduce.DataDrivenImportJob {
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
DataDrivenImportJob.class.getName());
|
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
public DataDrivenImportJob(final SqoopOptions opts) {
|
public DataDrivenImportJob(final SqoopOptions opts) {
|
||||||
super(opts, null, DataDrivenDBInputFormat.class, null, null);
|
super(opts);
|
||||||
}
|
}
|
||||||
|
|
||||||
public DataDrivenImportJob(final SqoopOptions opts,
|
public DataDrivenImportJob(final SqoopOptions opts,
|
||||||
final Class<? extends InputFormat> inputFormatClass,
|
final Class<? extends InputFormat> inputFormatClass,
|
||||||
ImportJobContext context) {
|
ImportJobContext context) {
|
||||||
super(opts, null, inputFormatClass, null, context);
|
super(opts, inputFormatClass, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void configureMapper(Job job, String tableName,
|
|
||||||
String tableClassName) throws IOException {
|
|
||||||
if (options.getFileLayout() == SqoopOptions.FileLayout.TextFile) {
|
|
||||||
// For text files, specify these as the output types; for
|
|
||||||
// other types, we just use the defaults.
|
|
||||||
job.setOutputKeyClass(Text.class);
|
|
||||||
job.setOutputValueClass(NullWritable.class);
|
|
||||||
} else if (options.getFileLayout()
|
|
||||||
== SqoopOptions.FileLayout.AvroDataFile) {
|
|
||||||
ConnManager connManager = getContext().getConnManager();
|
|
||||||
AvroSchemaGenerator generator = new AvroSchemaGenerator(options,
|
|
||||||
connManager, tableName);
|
|
||||||
Schema schema = generator.generate();
|
|
||||||
AvroJob.setMapOutputSchema(job.getConfiguration(), schema);
|
|
||||||
}
|
|
||||||
|
|
||||||
job.setMapperClass(getMapperClass());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected Class<? extends Mapper> getMapperClass() {
|
|
||||||
if (options.getFileLayout() == SqoopOptions.FileLayout.TextFile) {
|
|
||||||
return TextImportMapper.class;
|
|
||||||
} else if (options.getFileLayout()
|
|
||||||
== SqoopOptions.FileLayout.SequenceFile) {
|
|
||||||
return SequenceFileImportMapper.class;
|
|
||||||
} else if (options.getFileLayout()
|
|
||||||
== SqoopOptions.FileLayout.AvroDataFile) {
|
|
||||||
return AvroImportMapper.class;
|
|
||||||
}
|
|
||||||
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected Class<? extends OutputFormat> getOutputFormatClass()
|
|
||||||
throws ClassNotFoundException {
|
|
||||||
if (options.getFileLayout() == SqoopOptions.FileLayout.TextFile) {
|
|
||||||
return RawKeyTextOutputFormat.class;
|
|
||||||
} else if (options.getFileLayout()
|
|
||||||
== SqoopOptions.FileLayout.SequenceFile) {
|
|
||||||
return SequenceFileOutputFormat.class;
|
|
||||||
} else if (options.getFileLayout()
|
|
||||||
== SqoopOptions.FileLayout.AvroDataFile) {
|
|
||||||
return AvroOutputFormat.class;
|
|
||||||
}
|
|
||||||
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void configureInputFormat(Job job, String tableName,
|
|
||||||
String tableClassName, String splitByCol) throws IOException {
|
|
||||||
ConnManager mgr = getContext().getConnManager();
|
|
||||||
try {
|
|
||||||
String username = options.getUsername();
|
|
||||||
if (null == username || username.length() == 0) {
|
|
||||||
DBConfiguration.configureDB(job.getConfiguration(),
|
|
||||||
mgr.getDriverClass(), options.getConnectString(),
|
|
||||||
options.getFetchSize());
|
|
||||||
} else {
|
|
||||||
DBConfiguration.configureDB(job.getConfiguration(),
|
|
||||||
mgr.getDriverClass(), options.getConnectString(),
|
|
||||||
username, options.getPassword(), options.getFetchSize());
|
|
||||||
}
|
|
||||||
|
|
||||||
if (null != tableName) {
|
|
||||||
// Import a table.
|
|
||||||
String [] colNames = options.getColumns();
|
|
||||||
if (null == colNames) {
|
|
||||||
colNames = mgr.getColumnNames(tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
String [] sqlColNames = null;
|
|
||||||
if (null != colNames) {
|
|
||||||
sqlColNames = new String[colNames.length];
|
|
||||||
for (int i = 0; i < colNames.length; i++) {
|
|
||||||
sqlColNames[i] = mgr.escapeColName(colNames[i]);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// It's ok if the where clause is null in DBInputFormat.setInput.
|
|
||||||
String whereClause = options.getWhereClause();
|
|
||||||
|
|
||||||
// We can't set the class properly in here, because we may not have the
|
|
||||||
// jar loaded in this JVM. So we start by calling setInput() with
|
|
||||||
// DBWritable and then overriding the string manually.
|
|
||||||
DataDrivenDBInputFormat.setInput(job, DBWritable.class,
|
|
||||||
mgr.escapeTableName(tableName), whereClause,
|
|
||||||
mgr.escapeColName(splitByCol), sqlColNames);
|
|
||||||
|
|
||||||
// If user specified boundary query on the command line propagate it to
|
|
||||||
// the job
|
|
||||||
if(options.getBoundaryQuery() != null) {
|
|
||||||
DataDrivenDBInputFormat.setBoundingQuery(job.getConfiguration(),
|
|
||||||
options.getBoundaryQuery());
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
// Import a free-form query.
|
|
||||||
String inputQuery = options.getSqlQuery();
|
|
||||||
String sanitizedQuery = inputQuery.replace(
|
|
||||||
DataDrivenDBInputFormat.SUBSTITUTE_TOKEN, " (1 = 1) ");
|
|
||||||
|
|
||||||
String inputBoundingQuery = options.getBoundaryQuery();
|
|
||||||
|
|
||||||
if(inputBoundingQuery == null) {
|
|
||||||
inputBoundingQuery =
|
|
||||||
mgr.getInputBoundsQuery(splitByCol, sanitizedQuery);
|
|
||||||
if (inputBoundingQuery == null) {
|
|
||||||
if (splitByCol != null) {
|
|
||||||
inputBoundingQuery = "SELECT MIN(" + splitByCol + "), MAX("
|
|
||||||
+ splitByCol + ") FROM (" + sanitizedQuery + ") AS t1";
|
|
||||||
} else {
|
|
||||||
inputBoundingQuery = "";
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
DataDrivenDBInputFormat.setInput(job, DBWritable.class,
|
|
||||||
inputQuery, inputBoundingQuery);
|
|
||||||
new DBConfiguration(job.getConfiguration()).setInputOrderBy(
|
|
||||||
splitByCol);
|
|
||||||
}
|
|
||||||
|
|
||||||
LOG.debug("Using table class: " + tableClassName);
|
|
||||||
job.getConfiguration().set(ConfigurationHelper.getDbInputClassProperty(),
|
|
||||||
tableClassName);
|
|
||||||
|
|
||||||
job.getConfiguration().setLong(LargeObjectLoader.MAX_INLINE_LOB_LEN_KEY,
|
|
||||||
options.getInlineLobLimit());
|
|
||||||
|
|
||||||
LOG.debug("Using InputFormat: " + inputFormatClass);
|
|
||||||
job.setInputFormatClass(inputFormatClass);
|
|
||||||
} finally {
|
|
||||||
try {
|
|
||||||
mgr.close();
|
|
||||||
} catch (SQLException sqlE) {
|
|
||||||
LOG.warn("Error closing connection: " + sqlE);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,119 +18,15 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.Closeable;
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.mapreduce.JobContext;
|
|
||||||
import org.apache.hadoop.mapreduce.OutputCommitter;
|
|
||||||
import org.apache.hadoop.mapreduce.OutputFormat;
|
|
||||||
import org.apache.hadoop.mapreduce.RecordWriter;
|
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
||||||
import org.apache.hadoop.util.ReflectionUtils;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.lib.FieldMappable;
|
import com.cloudera.sqoop.lib.FieldMappable;
|
||||||
import com.cloudera.sqoop.lib.FieldMapProcessor;
|
|
||||||
import com.cloudera.sqoop.lib.ProcessingException;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* OutputFormat that produces a RecordReader which instantiates
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* a FieldMapProcessor which will process FieldMappable
|
|
||||||
* output keys.
|
|
||||||
*
|
|
||||||
* <p>The output value is ignored.</p>
|
|
||||||
*
|
|
||||||
* <p>The FieldMapProcessor implementation may do any arbitrary
|
|
||||||
* processing on the object. For example, it may write an object
|
|
||||||
* to HBase, etc.</p>
|
|
||||||
*
|
|
||||||
* <p>If the FieldMapProcessor implementation also implements
|
|
||||||
* Closeable, it will be close()'d in the RecordReader's close()
|
|
||||||
* method.</p>
|
|
||||||
*
|
|
||||||
* <p>If the FMP implements Configurable, it will be configured
|
|
||||||
* correctly via ReflectionUtils.</p>
|
|
||||||
*/
|
*/
|
||||||
public class DelegatingOutputFormat<K extends FieldMappable, V>
|
public class DelegatingOutputFormat<K extends FieldMappable, V>
|
||||||
extends OutputFormat<K, V> {
|
extends org.apache.sqoop.mapreduce.DelegatingOutputFormat <K, V> {
|
||||||
|
|
||||||
/** conf key: the FieldMapProcessor class to instantiate. */
|
|
||||||
public static final String DELEGATE_CLASS_KEY =
|
public static final String DELEGATE_CLASS_KEY =
|
||||||
"sqoop.output.delegate.field.map.processor.class";
|
org.apache.sqoop.mapreduce.DelegatingOutputFormat.DELEGATE_CLASS_KEY;
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public void checkOutputSpecs(JobContext context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
Configuration conf = context.getConfiguration();
|
|
||||||
|
|
||||||
if (null == conf.get(DELEGATE_CLASS_KEY)) {
|
|
||||||
throw new IOException("Delegate FieldMapProcessor class is not set.");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public OutputCommitter getOutputCommitter(TaskAttemptContext context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
return new NullOutputCommitter();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
|
||||||
throws IOException {
|
|
||||||
try {
|
|
||||||
return new DelegatingRecordWriter(context);
|
|
||||||
} catch (ClassNotFoundException cnfe) {
|
|
||||||
throw new IOException(cnfe);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* RecordWriter to write the output to a row in a database table.
|
|
||||||
* The actual database updates are executed in a second thread.
|
|
||||||
*/
|
|
||||||
public class DelegatingRecordWriter extends RecordWriter<K, V> {
|
|
||||||
|
|
||||||
private Configuration conf;
|
|
||||||
|
|
||||||
private FieldMapProcessor mapProcessor;
|
|
||||||
|
|
||||||
public DelegatingRecordWriter(TaskAttemptContext context)
|
|
||||||
throws ClassNotFoundException {
|
|
||||||
|
|
||||||
this.conf = context.getConfiguration();
|
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
Class<? extends FieldMapProcessor> procClass =
|
|
||||||
(Class<? extends FieldMapProcessor>)
|
|
||||||
conf.getClass(DELEGATE_CLASS_KEY, null);
|
|
||||||
this.mapProcessor = ReflectionUtils.newInstance(procClass, this.conf);
|
|
||||||
}
|
|
||||||
|
|
||||||
protected Configuration getConf() {
|
|
||||||
return this.conf;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public void close(TaskAttemptContext context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
if (mapProcessor instanceof Closeable) {
|
|
||||||
((Closeable) mapProcessor).close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public void write(K key, V value)
|
|
||||||
throws InterruptedException, IOException {
|
|
||||||
try {
|
|
||||||
mapProcessor.accept(key);
|
|
||||||
} catch (ProcessingException pe) {
|
|
||||||
throw new IOException(pe);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,123 +18,12 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.sql.Connection;
|
|
||||||
import java.sql.PreparedStatement;
|
|
||||||
import java.sql.SQLException;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.mapreduce.RecordWriter;
|
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class uses batch mode to execute underlying statements instead of
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* using a single multirow insert statement as its superclass.
|
|
||||||
*/
|
*/
|
||||||
public class ExportBatchOutputFormat<K extends SqoopRecord, V>
|
public class ExportBatchOutputFormat<K extends SqoopRecord, V>
|
||||||
extends ExportOutputFormat<K, V> {
|
extends org.apache.sqoop.mapreduce.ExportBatchOutputFormat
|
||||||
|
<SqoopRecord,V> {
|
||||||
private static final Log LOG =
|
|
||||||
LogFactory.getLog(ExportBatchOutputFormat.class);
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
|
||||||
throws IOException {
|
|
||||||
try {
|
|
||||||
return new ExportBatchRecordWriter(context);
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new IOException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* RecordWriter to write the output to a row in a database table.
|
|
||||||
* The actual database updates are executed in a second thread.
|
|
||||||
*/
|
|
||||||
public class ExportBatchRecordWriter extends ExportRecordWriter {
|
|
||||||
|
|
||||||
public ExportBatchRecordWriter(TaskAttemptContext context)
|
|
||||||
throws ClassNotFoundException, SQLException {
|
|
||||||
super(context);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
protected boolean isBatchExec() {
|
|
||||||
// We use batches here.
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
protected PreparedStatement getPreparedStatement(
|
|
||||||
List<SqoopRecord> userRecords) throws SQLException {
|
|
||||||
|
|
||||||
PreparedStatement stmt = null;
|
|
||||||
|
|
||||||
// Synchronize on connection to ensure this does not conflict
|
|
||||||
// with the operations in the update thread.
|
|
||||||
Connection conn = getConnection();
|
|
||||||
synchronized (conn) {
|
|
||||||
stmt = conn.prepareStatement(getInsertStatement(userRecords.size()));
|
|
||||||
}
|
|
||||||
|
|
||||||
// Inject the record parameters into the VALUES clauses.
|
|
||||||
for (SqoopRecord record : userRecords) {
|
|
||||||
record.write(stmt, 0);
|
|
||||||
stmt.addBatch();
|
|
||||||
}
|
|
||||||
|
|
||||||
return stmt;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return an INSERT statement.
|
|
||||||
*/
|
|
||||||
protected String getInsertStatement(int numRows) {
|
|
||||||
StringBuilder sb = new StringBuilder();
|
|
||||||
|
|
||||||
sb.append("INSERT INTO " + tableName + " ");
|
|
||||||
|
|
||||||
int numSlots;
|
|
||||||
if (this.columnNames != null) {
|
|
||||||
numSlots = this.columnNames.length;
|
|
||||||
|
|
||||||
sb.append("(");
|
|
||||||
boolean first = true;
|
|
||||||
for (String col : columnNames) {
|
|
||||||
if (!first) {
|
|
||||||
sb.append(", ");
|
|
||||||
}
|
|
||||||
|
|
||||||
sb.append(col);
|
|
||||||
first = false;
|
|
||||||
}
|
|
||||||
|
|
||||||
sb.append(") ");
|
|
||||||
} else {
|
|
||||||
numSlots = this.columnCount; // set if columnNames is null.
|
|
||||||
}
|
|
||||||
|
|
||||||
sb.append("VALUES ");
|
|
||||||
|
|
||||||
// generates the (?, ?, ?...).
|
|
||||||
sb.append("(");
|
|
||||||
for (int i = 0; i < numSlots; i++) {
|
|
||||||
if (i != 0) {
|
|
||||||
sb.append(", ");
|
|
||||||
}
|
|
||||||
|
|
||||||
sb.append("?");
|
|
||||||
}
|
|
||||||
sb.append(")");
|
|
||||||
|
|
||||||
return sb.toString();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,104 +18,23 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
|
||||||
import org.apache.hadoop.io.LongWritable;
|
|
||||||
import org.apache.hadoop.mapreduce.InputSplit;
|
|
||||||
import org.apache.hadoop.mapreduce.JobContext;
|
import org.apache.hadoop.mapreduce.JobContext;
|
||||||
import org.apache.hadoop.mapreduce.RecordReader;
|
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* InputFormat that generates a user-defined number of splits to inject data
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* into the database.
|
|
||||||
*/
|
*/
|
||||||
public class ExportInputFormat
|
public class ExportInputFormat
|
||||||
extends CombineFileInputFormat<LongWritable, Object> {
|
extends org.apache.sqoop.mapreduce.ExportInputFormat {
|
||||||
|
|
||||||
public static final Log LOG =
|
public static final int DEFAULT_NUM_MAP_TASKS =
|
||||||
LogFactory.getLog(ExportInputFormat.class.getName());
|
org.apache.sqoop.mapreduce.ExportInputFormat.DEFAULT_NUM_MAP_TASKS;
|
||||||
|
|
||||||
public static final int DEFAULT_NUM_MAP_TASKS = 4;
|
|
||||||
|
|
||||||
public ExportInputFormat() {
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return the number of bytes across all files in the job.
|
|
||||||
*/
|
|
||||||
private long getJobSize(JobContext job) throws IOException {
|
|
||||||
List<FileStatus> stats = listStatus(job);
|
|
||||||
long count = 0;
|
|
||||||
for (FileStatus stat : stats) {
|
|
||||||
count += stat.getLen();
|
|
||||||
}
|
|
||||||
|
|
||||||
return count;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public List<InputSplit> getSplits(JobContext job) throws IOException {
|
|
||||||
// Set the max split size based on the number of map tasks we want.
|
|
||||||
long numTasks = getNumMapTasks(job);
|
|
||||||
long numFileBytes = getJobSize(job);
|
|
||||||
long maxSplitSize = numFileBytes / numTasks;
|
|
||||||
|
|
||||||
setMaxSplitSize(maxSplitSize);
|
|
||||||
|
|
||||||
LOG.debug("Target numMapTasks=" + numTasks);
|
|
||||||
LOG.debug("Total input bytes=" + numFileBytes);
|
|
||||||
LOG.debug("maxSplitSize=" + maxSplitSize);
|
|
||||||
|
|
||||||
List<InputSplit> splits = super.getSplits(job);
|
|
||||||
|
|
||||||
if (LOG.isDebugEnabled()) {
|
|
||||||
LOG.debug("Generated splits:");
|
|
||||||
for (InputSplit split : splits) {
|
|
||||||
LOG.debug(" " + split);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return splits;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
public RecordReader createRecordReader(
|
|
||||||
InputSplit split, TaskAttemptContext context) throws IOException {
|
|
||||||
|
|
||||||
CombineFileSplit combineSplit = (CombineFileSplit) split;
|
|
||||||
|
|
||||||
// Use CombineFileRecordReader since this can handle CombineFileSplits
|
|
||||||
// and instantiate another RecordReader in a loop; do this with the
|
|
||||||
// CombineShimRecordReader.
|
|
||||||
RecordReader rr = new CombineFileRecordReader(combineSplit, context,
|
|
||||||
CombineShimRecordReader.class);
|
|
||||||
|
|
||||||
return rr;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Allows the user to control the number of map tasks used for this
|
|
||||||
* export job.
|
|
||||||
*/
|
|
||||||
public static void setNumMapTasks(JobContext job, int numTasks) {
|
public static void setNumMapTasks(JobContext job, int numTasks) {
|
||||||
job.getConfiguration().setInt(ExportJobBase.EXPORT_MAP_TASKS_KEY, numTasks);
|
org.apache.sqoop.mapreduce.ExportInputFormat.setNumMapTasks(job, numTasks);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @return the number of map tasks to use in this export job.
|
|
||||||
*/
|
|
||||||
public static int getNumMapTasks(JobContext job) {
|
public static int getNumMapTasks(JobContext job) {
|
||||||
return job.getConfiguration().getInt(ExportJobBase.EXPORT_MAP_TASKS_KEY,
|
return org.apache.sqoop.mapreduce.ExportInputFormat.getNumMapTasks(job);
|
||||||
DEFAULT_NUM_MAP_TASKS);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,394 +18,48 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.FileNotFoundException;
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.sql.SQLException;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FSDataInputStream;
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.io.NullWritable;
|
|
||||||
import org.apache.hadoop.mapreduce.Counters;
|
|
||||||
import org.apache.hadoop.mapreduce.InputFormat;
|
import org.apache.hadoop.mapreduce.InputFormat;
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
|
||||||
import org.apache.hadoop.mapreduce.Mapper;
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
import org.apache.hadoop.mapreduce.OutputFormat;
|
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||||
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.SqoopOptions;
|
|
||||||
import com.cloudera.sqoop.config.ConfigurationHelper;
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
|
||||||
import com.cloudera.sqoop.manager.ConnManager;
|
|
||||||
import com.cloudera.sqoop.manager.ExportJobContext;
|
import com.cloudera.sqoop.manager.ExportJobContext;
|
||||||
import com.cloudera.sqoop.orm.TableClassName;
|
|
||||||
import com.cloudera.sqoop.util.ExportException;
|
|
||||||
import com.cloudera.sqoop.util.PerfCounters;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Base class for running an export MapReduce job.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
*/
|
*/
|
||||||
public class ExportJobBase extends JobBase {
|
public class ExportJobBase
|
||||||
|
extends org.apache.sqoop.mapreduce.ExportJobBase {
|
||||||
|
|
||||||
/**
|
|
||||||
* The (inferred) type of a file or group of files.
|
|
||||||
*/
|
|
||||||
public enum FileType {
|
|
||||||
SEQUENCE_FILE, AVRO_DATA_FILE, UNKNOWN
|
|
||||||
}
|
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
ExportJobBase.class.getName());
|
|
||||||
|
|
||||||
/** What SqoopRecord class to use to read a record for export. */
|
|
||||||
public static final String SQOOP_EXPORT_TABLE_CLASS_KEY =
|
public static final String SQOOP_EXPORT_TABLE_CLASS_KEY =
|
||||||
"sqoop.mapreduce.export.table.class";
|
org.apache.sqoop.mapreduce.ExportJobBase.SQOOP_EXPORT_TABLE_CLASS_KEY;
|
||||||
|
|
||||||
/**
|
|
||||||
* What column of the table to use for the WHERE clause of
|
|
||||||
* an updating export.
|
|
||||||
*/
|
|
||||||
public static final String SQOOP_EXPORT_UPDATE_COL_KEY =
|
public static final String SQOOP_EXPORT_UPDATE_COL_KEY =
|
||||||
"sqoop.mapreduce.export.update.col";
|
org.apache.sqoop.mapreduce.ExportJobBase.SQOOP_EXPORT_UPDATE_COL_KEY;
|
||||||
|
|
||||||
/** Number of map tasks to use for an export. */
|
|
||||||
public static final String EXPORT_MAP_TASKS_KEY =
|
public static final String EXPORT_MAP_TASKS_KEY =
|
||||||
"sqoop.mapreduce.export.map.tasks";
|
org.apache.sqoop.mapreduce.ExportJobBase.EXPORT_MAP_TASKS_KEY;
|
||||||
|
|
||||||
protected ExportJobContext context;
|
|
||||||
|
|
||||||
public ExportJobBase(final ExportJobContext ctxt) {
|
public ExportJobBase(final ExportJobContext ctxt) {
|
||||||
this(ctxt, null, null, null);
|
super(ctxt);
|
||||||
}
|
}
|
||||||
|
|
||||||
public ExportJobBase(final ExportJobContext ctxt,
|
public ExportJobBase(final ExportJobContext ctxt,
|
||||||
final Class<? extends Mapper> mapperClass,
|
final Class<? extends Mapper> mapperClass,
|
||||||
final Class<? extends InputFormat> inputFormatClass,
|
final Class<? extends InputFormat> inputFormatClass,
|
||||||
final Class<? extends OutputFormat> outputFormatClass) {
|
final Class<? extends OutputFormat> outputFormatClass) {
|
||||||
super(ctxt.getOptions(), mapperClass, inputFormatClass, outputFormatClass);
|
super(ctxt, mapperClass, inputFormatClass, outputFormatClass);
|
||||||
this.context = ctxt;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @return true if p is a SequenceFile, or a directory containing
|
|
||||||
* SequenceFiles.
|
|
||||||
*/
|
|
||||||
public static boolean isSequenceFiles(Configuration conf, Path p)
|
public static boolean isSequenceFiles(Configuration conf, Path p)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return getFileType(conf, p) == FileType.SEQUENCE_FILE;
|
return org.apache.sqoop.mapreduce.ExportJobBase.isSequenceFiles(conf, p);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @return the type of the file represented by p (or the files in p, if a
|
|
||||||
* directory)
|
|
||||||
*/
|
|
||||||
public static FileType getFileType(Configuration conf, Path p)
|
public static FileType getFileType(Configuration conf, Path p)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
FileSystem fs = p.getFileSystem(conf);
|
return org.apache.sqoop.mapreduce.ExportJobBase.getFileType(conf, p);
|
||||||
|
|
||||||
try {
|
|
||||||
FileStatus stat = fs.getFileStatus(p);
|
|
||||||
|
|
||||||
if (null == stat) {
|
|
||||||
// Couldn't get the item.
|
|
||||||
LOG.warn("Input path " + p + " does not exist");
|
|
||||||
return FileType.UNKNOWN;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (stat.isDir()) {
|
|
||||||
FileStatus [] subitems = fs.listStatus(p);
|
|
||||||
if (subitems == null || subitems.length == 0) {
|
|
||||||
LOG.warn("Input path " + p + " contains no files");
|
|
||||||
return FileType.UNKNOWN; // empty dir.
|
|
||||||
}
|
|
||||||
|
|
||||||
// Pick a child entry to examine instead.
|
|
||||||
boolean foundChild = false;
|
|
||||||
for (int i = 0; i < subitems.length; i++) {
|
|
||||||
stat = subitems[i];
|
|
||||||
if (!stat.isDir() && !stat.getPath().getName().startsWith("_")) {
|
|
||||||
foundChild = true;
|
|
||||||
break; // This item is a visible file. Check it.
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!foundChild) {
|
|
||||||
stat = null; // Couldn't find a reasonable candidate.
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (null == stat) {
|
|
||||||
LOG.warn("null FileStatus object in isSequenceFiles(); "
|
|
||||||
+ "assuming false.");
|
|
||||||
return FileType.UNKNOWN;
|
|
||||||
}
|
|
||||||
|
|
||||||
Path target = stat.getPath();
|
|
||||||
return fromMagicNumber(target, conf);
|
|
||||||
} catch (FileNotFoundException fnfe) {
|
|
||||||
LOG.warn("Input path " + p + " does not exist");
|
|
||||||
return FileType.UNKNOWN; // doesn't exist!
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @param file a file to test.
|
|
||||||
* @return true if 'file' refers to a SequenceFile.
|
|
||||||
*/
|
|
||||||
private static FileType fromMagicNumber(Path file, Configuration conf) {
|
|
||||||
// Test target's header to see if it contains magic numbers indicating its
|
|
||||||
// file type
|
|
||||||
byte [] header = new byte[3];
|
|
||||||
FSDataInputStream is = null;
|
|
||||||
try {
|
|
||||||
FileSystem fs = file.getFileSystem(conf);
|
|
||||||
is = fs.open(file);
|
|
||||||
is.readFully(header);
|
|
||||||
} catch (IOException ioe) {
|
|
||||||
// Error reading header or EOF; assume unknown
|
|
||||||
LOG.warn("IOException checking input file header: " + ioe);
|
|
||||||
return FileType.UNKNOWN;
|
|
||||||
} finally {
|
|
||||||
try {
|
|
||||||
if (null != is) {
|
|
||||||
is.close();
|
|
||||||
}
|
|
||||||
} catch (IOException ioe) {
|
|
||||||
// ignore; closing.
|
|
||||||
LOG.warn("IOException closing input stream: " + ioe + "; ignoring.");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (header[0] == 'S' && header[1] == 'E' && header[2] == 'Q') {
|
|
||||||
return FileType.SEQUENCE_FILE;
|
|
||||||
}
|
|
||||||
if (header[0] == 'O' && header[1] == 'b' && header[2] == 'j') {
|
|
||||||
return FileType.AVRO_DATA_FILE;
|
|
||||||
}
|
|
||||||
return FileType.UNKNOWN;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return the Path to the files we are going to export to the db.
|
|
||||||
*/
|
|
||||||
protected Path getInputPath() throws IOException {
|
|
||||||
Path inputPath = new Path(context.getOptions().getExportDir());
|
|
||||||
Configuration conf = options.getConf();
|
|
||||||
inputPath = inputPath.makeQualified(FileSystem.get(conf));
|
|
||||||
return inputPath;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void configureInputFormat(Job job, String tableName,
|
|
||||||
String tableClassName, String splitByCol)
|
|
||||||
throws ClassNotFoundException, IOException {
|
|
||||||
|
|
||||||
super.configureInputFormat(job, tableName, tableClassName, splitByCol);
|
|
||||||
FileInputFormat.addInputPath(job, getInputPath());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected Class<? extends InputFormat> getInputFormatClass()
|
|
||||||
throws ClassNotFoundException {
|
|
||||||
Class<? extends InputFormat> configuredIF = super.getInputFormatClass();
|
|
||||||
if (null == configuredIF) {
|
|
||||||
return ExportInputFormat.class;
|
|
||||||
} else {
|
|
||||||
return configuredIF;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected Class<? extends OutputFormat> getOutputFormatClass()
|
|
||||||
throws ClassNotFoundException {
|
|
||||||
Class<? extends OutputFormat> configuredOF = super.getOutputFormatClass();
|
|
||||||
if (null == configuredOF) {
|
|
||||||
if (!options.isBatchMode()) {
|
|
||||||
return ExportOutputFormat.class;
|
|
||||||
} else {
|
|
||||||
return ExportBatchOutputFormat.class;
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
return configuredOF;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void configureMapper(Job job, String tableName,
|
|
||||||
String tableClassName) throws ClassNotFoundException, IOException {
|
|
||||||
|
|
||||||
job.setMapperClass(getMapperClass());
|
|
||||||
|
|
||||||
// Concurrent writes of the same records would be problematic.
|
|
||||||
ConfigurationHelper.setJobMapSpeculativeExecution(job, false);
|
|
||||||
|
|
||||||
job.setMapOutputKeyClass(SqoopRecord.class);
|
|
||||||
job.setMapOutputValueClass(NullWritable.class);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected int configureNumTasks(Job job) throws IOException {
|
|
||||||
int numMaps = super.configureNumTasks(job);
|
|
||||||
job.getConfiguration().setInt(EXPORT_MAP_TASKS_KEY, numMaps);
|
|
||||||
return numMaps;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected boolean runJob(Job job) throws ClassNotFoundException, IOException,
|
|
||||||
InterruptedException {
|
|
||||||
|
|
||||||
PerfCounters perfCounters = new PerfCounters();
|
|
||||||
perfCounters.startClock();
|
|
||||||
|
|
||||||
boolean success = job.waitForCompletion(true);
|
|
||||||
perfCounters.stopClock();
|
|
||||||
|
|
||||||
Counters jobCounters = job.getCounters();
|
|
||||||
// If the job has been retired, these may be unavailable.
|
|
||||||
if (null == jobCounters) {
|
|
||||||
displayRetiredJobNotice(LOG);
|
|
||||||
} else {
|
|
||||||
perfCounters.addBytes(jobCounters.getGroup("FileSystemCounters")
|
|
||||||
.findCounter("HDFS_BYTES_READ").getValue());
|
|
||||||
LOG.info("Transferred " + perfCounters.toString());
|
|
||||||
long numRecords = ConfigurationHelper.getNumMapInputRecords(job);
|
|
||||||
LOG.info("Exported " + numRecords + " records.");
|
|
||||||
}
|
|
||||||
|
|
||||||
return success;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Run an export job to dump a table from HDFS to a database. If a staging
|
|
||||||
* table is specified and the connection manager supports staging of data,
|
|
||||||
* the export will first populate the staging table and then migrate the
|
|
||||||
* data to the target table.
|
|
||||||
* @throws IOException if the export job encounters an IO error
|
|
||||||
* @throws ExportException if the job fails unexpectedly or is misconfigured.
|
|
||||||
*/
|
|
||||||
public void runExport() throws ExportException, IOException {
|
|
||||||
|
|
||||||
ConnManager cmgr = context.getConnManager();
|
|
||||||
SqoopOptions options = context.getOptions();
|
|
||||||
Configuration conf = options.getConf();
|
|
||||||
|
|
||||||
String outputTableName = context.getTableName();
|
|
||||||
String stagingTableName = context.getOptions().getStagingTableName();
|
|
||||||
|
|
||||||
String tableName = outputTableName;
|
|
||||||
boolean stagingEnabled = false;
|
|
||||||
if (stagingTableName != null) { // user has specified the staging table
|
|
||||||
if (cmgr.supportsStagingForExport()) {
|
|
||||||
LOG.info("Data will be staged in the table: " + stagingTableName);
|
|
||||||
tableName = stagingTableName;
|
|
||||||
stagingEnabled = true;
|
|
||||||
} else {
|
|
||||||
throw new ExportException("The active connection manager ("
|
|
||||||
+ cmgr.getClass().getCanonicalName()
|
|
||||||
+ ") does not support staging of data for export. "
|
|
||||||
+ "Please retry without specifying the --staging-table option.");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
String tableClassName =
|
|
||||||
new TableClassName(options).getClassForTable(outputTableName);
|
|
||||||
String ormJarFile = context.getJarFile();
|
|
||||||
|
|
||||||
LOG.info("Beginning export of " + outputTableName);
|
|
||||||
loadJars(conf, ormJarFile, tableClassName);
|
|
||||||
|
|
||||||
if (stagingEnabled) {
|
|
||||||
// Prepare the staging table
|
|
||||||
if (options.doClearStagingTable()) {
|
|
||||||
try {
|
|
||||||
// Delete all records from staging table
|
|
||||||
cmgr.deleteAllRecords(stagingTableName);
|
|
||||||
} catch (SQLException ex) {
|
|
||||||
throw new ExportException(
|
|
||||||
"Failed to empty staging table before export run", ex);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
// User has not explicitly specified the clear staging table option.
|
|
||||||
// Assert that the staging table is empty.
|
|
||||||
try {
|
|
||||||
long rowCount = cmgr.getTableRowCount(stagingTableName);
|
|
||||||
if (rowCount != 0L) {
|
|
||||||
throw new ExportException("The specified staging table ("
|
|
||||||
+ stagingTableName + ") is not empty. To force deletion of "
|
|
||||||
+ "its data, please retry with --clear-staging-table option.");
|
|
||||||
}
|
|
||||||
} catch (SQLException ex) {
|
|
||||||
throw new ExportException(
|
|
||||||
"Failed to count data rows in staging table: "
|
|
||||||
+ stagingTableName, ex);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
try {
|
|
||||||
Job job = new Job(conf);
|
|
||||||
|
|
||||||
// Set the external jar to use for the job.
|
|
||||||
job.getConfiguration().set("mapred.jar", ormJarFile);
|
|
||||||
|
|
||||||
configureInputFormat(job, tableName, tableClassName, null);
|
|
||||||
configureOutputFormat(job, tableName, tableClassName);
|
|
||||||
configureMapper(job, tableName, tableClassName);
|
|
||||||
configureNumTasks(job);
|
|
||||||
cacheJars(job, context.getConnManager());
|
|
||||||
setJob(job);
|
|
||||||
boolean success = runJob(job);
|
|
||||||
if (!success) {
|
|
||||||
throw new ExportException("Export job failed!");
|
|
||||||
}
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
throw new IOException(ie);
|
|
||||||
} catch (ClassNotFoundException cnfe) {
|
|
||||||
throw new IOException(cnfe);
|
|
||||||
} finally {
|
|
||||||
unloadJars();
|
|
||||||
}
|
|
||||||
|
|
||||||
// Unstage the data if needed
|
|
||||||
if (stagingEnabled) {
|
|
||||||
// Migrate data from staging table to the output table
|
|
||||||
try {
|
|
||||||
LOG.info("Starting to migrate data from staging table to destination.");
|
|
||||||
cmgr.migrateData(stagingTableName, outputTableName);
|
|
||||||
} catch (SQLException ex) {
|
|
||||||
LOG.error("Failed to move data from staging table ("
|
|
||||||
+ stagingTableName + ") to target table ("
|
|
||||||
+ outputTableName + ")", ex);
|
|
||||||
throw new ExportException(
|
|
||||||
"Failed to move data from staging table", ex);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return true if the input directory contains SequenceFiles.
|
|
||||||
* @deprecated use {@link #getInputFileType()} instead
|
|
||||||
*/
|
|
||||||
@Deprecated
|
|
||||||
protected boolean inputIsSequenceFiles() {
|
|
||||||
try {
|
|
||||||
return isSequenceFiles(
|
|
||||||
context.getOptions().getConf(), getInputPath());
|
|
||||||
} catch (IOException ioe) {
|
|
||||||
LOG.warn("Could not check file format for export; assuming text");
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
protected FileType getInputFileType() {
|
|
||||||
try {
|
|
||||||
return getFileType(context.getOptions().getConf(), getInputPath());
|
|
||||||
} catch (IOException ioe) {
|
|
||||||
return FileType.UNKNOWN;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,191 +18,11 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.sql.Connection;
|
|
||||||
import java.sql.PreparedStatement;
|
|
||||||
import java.sql.SQLException;
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.mapreduce.JobContext;
|
|
||||||
import org.apache.hadoop.mapreduce.RecordWriter;
|
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
||||||
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Insert the emitted keys as records into a database table.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* This supports a configurable "spill threshold" at which
|
|
||||||
* point intermediate transactions are committed.
|
|
||||||
*
|
|
||||||
* Record objects are buffered before actually performing the INSERT
|
|
||||||
* statements; this requires that the key implement the
|
|
||||||
* SqoopRecord interface.
|
|
||||||
*
|
|
||||||
* Uses DBOutputFormat/DBConfiguration for configuring the output.
|
|
||||||
*/
|
*/
|
||||||
public class ExportOutputFormat<K extends SqoopRecord, V>
|
public class ExportOutputFormat<K extends SqoopRecord, V>
|
||||||
extends AsyncSqlOutputFormat<K, V> {
|
extends org.apache.sqoop.mapreduce.ExportOutputFormat<K, V> {
|
||||||
|
|
||||||
private static final Log LOG = LogFactory.getLog(ExportOutputFormat.class);
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public void checkOutputSpecs(JobContext context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
Configuration conf = context.getConfiguration();
|
|
||||||
DBConfiguration dbConf = new DBConfiguration(conf);
|
|
||||||
|
|
||||||
// Sanity check all the configuration values we need.
|
|
||||||
if (null == conf.get(DBConfiguration.URL_PROPERTY)) {
|
|
||||||
throw new IOException("Database connection URL is not set.");
|
|
||||||
} else if (null == dbConf.getOutputTableName()) {
|
|
||||||
throw new IOException("Table name is not set for export");
|
|
||||||
} else if (null == dbConf.getOutputFieldNames()
|
|
||||||
&& 0 == dbConf.getOutputFieldCount()) {
|
|
||||||
throw new IOException(
|
|
||||||
"Output field names are null and zero output field count set.");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
|
||||||
throws IOException {
|
|
||||||
try {
|
|
||||||
return new ExportRecordWriter(context);
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new IOException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* RecordWriter to write the output to a row in a database table.
|
|
||||||
* The actual database updates are executed in a second thread.
|
|
||||||
*/
|
|
||||||
public class ExportRecordWriter extends AsyncSqlRecordWriter<K, V> {
|
|
||||||
|
|
||||||
protected String tableName;
|
|
||||||
protected String [] columnNames; // The columns to insert into.
|
|
||||||
protected int columnCount; // If columnNames is null, tells ## of cols.
|
|
||||||
|
|
||||||
public ExportRecordWriter(TaskAttemptContext context)
|
|
||||||
throws ClassNotFoundException, SQLException {
|
|
||||||
super(context);
|
|
||||||
|
|
||||||
Configuration conf = getConf();
|
|
||||||
|
|
||||||
DBConfiguration dbConf = new DBConfiguration(conf);
|
|
||||||
tableName = dbConf.getOutputTableName();
|
|
||||||
columnNames = dbConf.getOutputFieldNames();
|
|
||||||
columnCount = dbConf.getOutputFieldCount();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return the name of the table we are inserting into.
|
|
||||||
*/
|
|
||||||
protected final String getTableName() {
|
|
||||||
return tableName;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return the list of columns we are updating.
|
|
||||||
*/
|
|
||||||
protected final String [] getColumnNames() {
|
|
||||||
if (null == columnNames) {
|
|
||||||
return null;
|
|
||||||
} else {
|
|
||||||
return Arrays.copyOf(columnNames, columnNames.length);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return the number of columns we are updating.
|
|
||||||
*/
|
|
||||||
protected final int getColumnCount() {
|
|
||||||
return columnCount;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
protected PreparedStatement getPreparedStatement(
|
|
||||||
List<SqoopRecord> userRecords) throws SQLException {
|
|
||||||
|
|
||||||
PreparedStatement stmt = null;
|
|
||||||
|
|
||||||
// Synchronize on connection to ensure this does not conflict
|
|
||||||
// with the operations in the update thread.
|
|
||||||
Connection conn = getConnection();
|
|
||||||
synchronized (conn) {
|
|
||||||
stmt = conn.prepareStatement(getInsertStatement(userRecords.size()));
|
|
||||||
}
|
|
||||||
|
|
||||||
// Inject the record parameters into the VALUES clauses.
|
|
||||||
int position = 0;
|
|
||||||
for (SqoopRecord record : userRecords) {
|
|
||||||
position += record.write(stmt, position);
|
|
||||||
}
|
|
||||||
|
|
||||||
return stmt;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return an INSERT statement suitable for inserting 'numRows' rows.
|
|
||||||
*/
|
|
||||||
protected String getInsertStatement(int numRows) {
|
|
||||||
StringBuilder sb = new StringBuilder();
|
|
||||||
|
|
||||||
sb.append("INSERT INTO " + tableName + " ");
|
|
||||||
|
|
||||||
int numSlots;
|
|
||||||
if (this.columnNames != null) {
|
|
||||||
numSlots = this.columnNames.length;
|
|
||||||
|
|
||||||
sb.append("(");
|
|
||||||
boolean first = true;
|
|
||||||
for (String col : columnNames) {
|
|
||||||
if (!first) {
|
|
||||||
sb.append(", ");
|
|
||||||
}
|
|
||||||
|
|
||||||
sb.append(col);
|
|
||||||
first = false;
|
|
||||||
}
|
|
||||||
|
|
||||||
sb.append(") ");
|
|
||||||
} else {
|
|
||||||
numSlots = this.columnCount; // set if columnNames is null.
|
|
||||||
}
|
|
||||||
|
|
||||||
sb.append("VALUES ");
|
|
||||||
|
|
||||||
// generates the (?, ?, ?...) used for each row.
|
|
||||||
StringBuilder sbRow = new StringBuilder();
|
|
||||||
sbRow.append("(");
|
|
||||||
for (int i = 0; i < numSlots; i++) {
|
|
||||||
if (i != 0) {
|
|
||||||
sbRow.append(", ");
|
|
||||||
}
|
|
||||||
|
|
||||||
sbRow.append("?");
|
|
||||||
}
|
|
||||||
sbRow.append(")");
|
|
||||||
|
|
||||||
// Now append that numRows times.
|
|
||||||
for (int i = 0; i < numRows; i++) {
|
|
||||||
if (i != 0) {
|
|
||||||
sb.append(", ");
|
|
||||||
}
|
|
||||||
|
|
||||||
sb.append(sbRow);
|
|
||||||
}
|
|
||||||
|
|
||||||
return sb.toString();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,163 +18,24 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
|
||||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
|
||||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
|
||||||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
|
||||||
import org.apache.hadoop.hbase.client.HTable;
|
|
||||||
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
|
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
|
||||||
import org.apache.hadoop.io.NullWritable;
|
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
|
||||||
import org.apache.hadoop.mapreduce.Mapper;
|
|
||||||
import org.apache.hadoop.mapreduce.OutputFormat;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.SqoopOptions;
|
import com.cloudera.sqoop.SqoopOptions;
|
||||||
import com.cloudera.sqoop.hbase.HBasePutProcessor;
|
|
||||||
import com.cloudera.sqoop.lib.FieldMapProcessor;
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
|
||||||
import com.cloudera.sqoop.manager.ConnManager;
|
|
||||||
import com.cloudera.sqoop.manager.ImportJobContext;
|
import com.cloudera.sqoop.manager.ImportJobContext;
|
||||||
import com.cloudera.sqoop.util.ImportException;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Runs an HBase import via DataDrivenDBInputFormat to the HBasePutProcessor
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* in the DelegatingOutputFormat.
|
|
||||||
*/
|
*/
|
||||||
public class HBaseImportJob extends DataDrivenImportJob {
|
public class HBaseImportJob
|
||||||
|
extends org.apache.sqoop.mapreduce.HBaseImportJob {
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
public static final Log LOG = LogFactory.getLog(
|
||||||
HBaseImportJob.class.getName());
|
HBaseImportJob.class.getName());
|
||||||
|
|
||||||
public HBaseImportJob(final SqoopOptions opts,
|
public HBaseImportJob(final SqoopOptions opts,
|
||||||
final ImportJobContext importContext) {
|
final ImportJobContext importContext) {
|
||||||
super(opts, importContext.getInputFormat(), importContext);
|
super(opts, importContext);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void configureMapper(Job job, String tableName,
|
|
||||||
String tableClassName) throws IOException {
|
|
||||||
job.setOutputKeyClass(SqoopRecord.class);
|
|
||||||
job.setOutputValueClass(NullWritable.class);
|
|
||||||
job.setMapperClass(getMapperClass());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected Class<? extends Mapper> getMapperClass() {
|
|
||||||
return HBaseImportMapper.class;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected Class<? extends OutputFormat> getOutputFormatClass()
|
|
||||||
throws ClassNotFoundException {
|
|
||||||
return DelegatingOutputFormat.class;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void configureOutputFormat(Job job, String tableName,
|
|
||||||
String tableClassName) throws ClassNotFoundException, IOException {
|
|
||||||
|
|
||||||
// Use the DelegatingOutputFormat with the HBasePutProcessor.
|
|
||||||
job.setOutputFormatClass(getOutputFormatClass());
|
|
||||||
|
|
||||||
Configuration conf = job.getConfiguration();
|
|
||||||
conf.setClass("sqoop.output.delegate.field.map.processor.class",
|
|
||||||
HBasePutProcessor.class,
|
|
||||||
FieldMapProcessor.class);
|
|
||||||
|
|
||||||
// Set the HBase parameters (table, column family, row key):
|
|
||||||
conf.set(HBasePutProcessor.TABLE_NAME_KEY, options.getHBaseTable());
|
|
||||||
conf.set(HBasePutProcessor.COL_FAMILY_KEY, options.getHBaseColFamily());
|
|
||||||
|
|
||||||
// What column of the input becomes the row key?
|
|
||||||
String rowKeyCol = options.getHBaseRowKeyColumn();
|
|
||||||
if (null == rowKeyCol) {
|
|
||||||
// User didn't explicitly set one. If there's a split-by column set,
|
|
||||||
// use that.
|
|
||||||
rowKeyCol = options.getSplitByCol();
|
|
||||||
}
|
|
||||||
|
|
||||||
if (null == rowKeyCol) {
|
|
||||||
// No split-by column is explicitly set.
|
|
||||||
// If the table has a primary key, use that.
|
|
||||||
ConnManager manager = getContext().getConnManager();
|
|
||||||
rowKeyCol = manager.getPrimaryKey(tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (null == rowKeyCol) {
|
|
||||||
// Give up here if this is still unset.
|
|
||||||
throw new IOException("Could not determine the row-key column. "
|
|
||||||
+ "Use --hbase-row-key to specify the input column that "
|
|
||||||
+ "names each row.");
|
|
||||||
}
|
|
||||||
|
|
||||||
conf.set(HBasePutProcessor.ROW_KEY_COLUMN_KEY, rowKeyCol);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** Create the target HBase table before running the job. */
|
|
||||||
protected void jobSetup(Job job) throws IOException, ImportException {
|
|
||||||
Configuration conf = job.getConfiguration();
|
|
||||||
String tableName = conf.get(HBasePutProcessor.TABLE_NAME_KEY);
|
|
||||||
String familyName = conf.get(HBasePutProcessor.COL_FAMILY_KEY);
|
|
||||||
|
|
||||||
if (null == tableName) {
|
|
||||||
throw new ImportException(
|
|
||||||
"Import to HBase error: Table name not specified");
|
|
||||||
}
|
|
||||||
|
|
||||||
if (null == familyName) {
|
|
||||||
throw new ImportException(
|
|
||||||
"Import to HBase error: Column family not specified");
|
|
||||||
}
|
|
||||||
|
|
||||||
// Add HBase configuration files to this conf object.
|
|
||||||
HBaseConfiguration.addHbaseResources(conf);
|
|
||||||
|
|
||||||
HBaseAdmin admin = new HBaseAdmin(conf);
|
|
||||||
|
|
||||||
// Check to see if the table exists.
|
|
||||||
HTableDescriptor tableDesc = new HTableDescriptor(tableName);
|
|
||||||
byte [] familyBytes = Bytes.toBytes(familyName);
|
|
||||||
HColumnDescriptor colDesc = new HColumnDescriptor(familyBytes);
|
|
||||||
if (!admin.tableExists(tableName)) {
|
|
||||||
if (options.getCreateHBaseTable()) {
|
|
||||||
// Create the table.
|
|
||||||
LOG.info("Creating missing HBase table " + tableName);
|
|
||||||
tableDesc.addFamily(colDesc);
|
|
||||||
admin.createTable(tableDesc);
|
|
||||||
} else {
|
|
||||||
LOG.warn("Could not find HBase table " + tableName);
|
|
||||||
LOG.warn("This job may fail. Either explicitly create the table,");
|
|
||||||
LOG.warn("or re-run with --hbase-create-table.");
|
|
||||||
}
|
|
||||||
} else if (!tableDesc.hasFamily(familyBytes)) {
|
|
||||||
if (options.getCreateHBaseTable()) {
|
|
||||||
// Create the column family.
|
|
||||||
LOG.info("Creating missing column family " + familyName);
|
|
||||||
admin.disableTable(tableName);
|
|
||||||
admin.addColumn(tableName, colDesc);
|
|
||||||
admin.enableTable(tableName);
|
|
||||||
} else {
|
|
||||||
LOG.warn("Could not find column family " + familyName + " in table "
|
|
||||||
+ tableName);
|
|
||||||
LOG.warn("This job may fail. Either create the column family,");
|
|
||||||
LOG.warn("or re-run with --hbase-create-table.");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Make sure HBase libraries are shipped as part of the job.
|
|
||||||
TableMapReduceUtil.addDependencyJars(job);
|
|
||||||
TableMapReduceUtil.addDependencyJars(conf, HTable.class);
|
|
||||||
|
|
||||||
super.jobSetup(job);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,24 +18,10 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.io.LongWritable;
|
|
||||||
import org.apache.hadoop.io.NullWritable;
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Imports records by writing them to HBase via the DelegatingOutputFormat
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* and the HBasePutProcessor.
|
|
||||||
*/
|
*/
|
||||||
public class HBaseImportMapper
|
public class HBaseImportMapper
|
||||||
extends AutoProgressMapper<LongWritable, SqoopRecord, SqoopRecord,
|
extends org.apache.sqoop.mapreduce.HBaseImportMapper {
|
||||||
NullWritable> {
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void map(LongWritable key, SqoopRecord val, Context context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
context.write(val, NullWritable.get());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,49 +18,24 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.io.SequenceFile.CompressionType;
|
|
||||||
import org.apache.hadoop.io.compress.CompressionCodec;
|
|
||||||
import org.apache.hadoop.io.compress.GzipCodec;
|
|
||||||
import org.apache.hadoop.mapreduce.Counters;
|
|
||||||
import org.apache.hadoop.mapreduce.InputFormat;
|
import org.apache.hadoop.mapreduce.InputFormat;
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
|
||||||
import org.apache.hadoop.mapreduce.Mapper;
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
import org.apache.hadoop.mapreduce.OutputFormat;
|
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||||
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.SqoopOptions;
|
import com.cloudera.sqoop.SqoopOptions;
|
||||||
import com.cloudera.sqoop.orm.TableClassName;
|
|
||||||
import com.cloudera.sqoop.util.ImportException;
|
|
||||||
import com.cloudera.sqoop.util.PerfCounters;
|
|
||||||
import com.cloudera.sqoop.config.ConfigurationHelper;
|
|
||||||
import com.cloudera.sqoop.io.CodecMap;
|
|
||||||
import com.cloudera.sqoop.manager.ImportJobContext;
|
import com.cloudera.sqoop.manager.ImportJobContext;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Base class for running an import MapReduce job.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* Allows dependency injection, etc, for easy customization of import job types.
|
|
||||||
*/
|
*/
|
||||||
public class ImportJobBase extends JobBase {
|
public class ImportJobBase
|
||||||
|
extends org.apache.sqoop.mapreduce.ImportJobBase {
|
||||||
private ImportJobContext context;
|
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
ImportJobBase.class.getName());
|
|
||||||
|
|
||||||
public ImportJobBase() {
|
public ImportJobBase() {
|
||||||
this(null);
|
super();
|
||||||
}
|
}
|
||||||
|
|
||||||
public ImportJobBase(final SqoopOptions opts) {
|
public ImportJobBase(final SqoopOptions opts) {
|
||||||
this(opts, null, null, null, null);
|
super(opts);
|
||||||
}
|
}
|
||||||
|
|
||||||
public ImportJobBase(final SqoopOptions opts,
|
public ImportJobBase(final SqoopOptions opts,
|
||||||
@ -70,138 +43,7 @@ public ImportJobBase(final SqoopOptions opts,
|
|||||||
final Class<? extends InputFormat> inputFormatClass,
|
final Class<? extends InputFormat> inputFormatClass,
|
||||||
final Class<? extends OutputFormat> outputFormatClass,
|
final Class<? extends OutputFormat> outputFormatClass,
|
||||||
final ImportJobContext context) {
|
final ImportJobContext context) {
|
||||||
super(opts, mapperClass, inputFormatClass, outputFormatClass);
|
super(opts, mapperClass, inputFormatClass, outputFormatClass, context);
|
||||||
this.context = context;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Configure the output format to use for the job.
|
|
||||||
*/
|
|
||||||
@Override
|
|
||||||
protected void configureOutputFormat(Job job, String tableName,
|
|
||||||
String tableClassName) throws ClassNotFoundException, IOException {
|
|
||||||
|
|
||||||
job.setOutputFormatClass(getOutputFormatClass());
|
|
||||||
|
|
||||||
if (options.getFileLayout() == SqoopOptions.FileLayout.SequenceFile) {
|
|
||||||
job.getConfiguration().set("mapred.output.value.class", tableClassName);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (options.shouldUseCompression()) {
|
|
||||||
FileOutputFormat.setCompressOutput(job, true);
|
|
||||||
|
|
||||||
String codecName = options.getCompressionCodec();
|
|
||||||
Class<? extends CompressionCodec> codecClass;
|
|
||||||
if (codecName == null) {
|
|
||||||
codecClass = GzipCodec.class;
|
|
||||||
} else {
|
|
||||||
Configuration conf = job.getConfiguration();
|
|
||||||
codecClass = CodecMap.getCodec(codecName, conf).getClass();
|
|
||||||
}
|
|
||||||
FileOutputFormat.setOutputCompressorClass(job, codecClass);
|
|
||||||
|
|
||||||
if (options.getFileLayout() == SqoopOptions.FileLayout.SequenceFile) {
|
|
||||||
SequenceFileOutputFormat.setOutputCompressionType(job,
|
|
||||||
CompressionType.BLOCK);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
Path outputPath = context.getDestination();
|
|
||||||
FileOutputFormat.setOutputPath(job, outputPath);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Actually run the MapReduce job.
|
|
||||||
*/
|
|
||||||
@Override
|
|
||||||
protected boolean runJob(Job job) throws ClassNotFoundException, IOException,
|
|
||||||
InterruptedException {
|
|
||||||
|
|
||||||
PerfCounters perfCounters = new PerfCounters();
|
|
||||||
perfCounters.startClock();
|
|
||||||
|
|
||||||
boolean success = job.waitForCompletion(true);
|
|
||||||
perfCounters.stopClock();
|
|
||||||
|
|
||||||
Counters jobCounters = job.getCounters();
|
|
||||||
// If the job has been retired, these may be unavailable.
|
|
||||||
if (null == jobCounters) {
|
|
||||||
displayRetiredJobNotice(LOG);
|
|
||||||
} else {
|
|
||||||
perfCounters.addBytes(jobCounters.getGroup("FileSystemCounters")
|
|
||||||
.findCounter("HDFS_BYTES_WRITTEN").getValue());
|
|
||||||
LOG.info("Transferred " + perfCounters.toString());
|
|
||||||
long numRecords = ConfigurationHelper.getNumMapOutputRecords(job);
|
|
||||||
LOG.info("Retrieved " + numRecords + " records.");
|
|
||||||
}
|
|
||||||
return success;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Run an import job to read a table in to HDFS.
|
|
||||||
*
|
|
||||||
* @param tableName the database table to read; may be null if a free-form
|
|
||||||
* query is specified in the SqoopOptions, and the ImportJobBase subclass
|
|
||||||
* supports free-form queries.
|
|
||||||
* @param ormJarFile the Jar file to insert into the dcache classpath.
|
|
||||||
* (may be null)
|
|
||||||
* @param splitByCol the column of the database table to use to split
|
|
||||||
* the import
|
|
||||||
* @param conf A fresh Hadoop Configuration to use to build an MR job.
|
|
||||||
* @throws IOException if the job encountered an IO problem
|
|
||||||
* @throws ImportException if the job failed unexpectedly or was
|
|
||||||
* misconfigured.
|
|
||||||
*/
|
|
||||||
public void runImport(String tableName, String ormJarFile, String splitByCol,
|
|
||||||
Configuration conf) throws IOException, ImportException {
|
|
||||||
|
|
||||||
if (null != tableName) {
|
|
||||||
LOG.info("Beginning import of " + tableName);
|
|
||||||
} else {
|
|
||||||
LOG.info("Beginning query import.");
|
|
||||||
}
|
|
||||||
|
|
||||||
String tableClassName =
|
|
||||||
new TableClassName(options).getClassForTable(tableName);
|
|
||||||
loadJars(conf, ormJarFile, tableClassName);
|
|
||||||
|
|
||||||
try {
|
|
||||||
Job job = new Job(conf);
|
|
||||||
|
|
||||||
// Set the external jar to use for the job.
|
|
||||||
job.getConfiguration().set("mapred.jar", ormJarFile);
|
|
||||||
|
|
||||||
configureInputFormat(job, tableName, tableClassName, splitByCol);
|
|
||||||
configureOutputFormat(job, tableName, tableClassName);
|
|
||||||
configureMapper(job, tableName, tableClassName);
|
|
||||||
configureNumTasks(job);
|
|
||||||
cacheJars(job, getContext().getConnManager());
|
|
||||||
|
|
||||||
jobSetup(job);
|
|
||||||
setJob(job);
|
|
||||||
boolean success = runJob(job);
|
|
||||||
if (!success) {
|
|
||||||
throw new ImportException("Import job failed!");
|
|
||||||
}
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
throw new IOException(ie);
|
|
||||||
} catch (ClassNotFoundException cnfe) {
|
|
||||||
throw new IOException(cnfe);
|
|
||||||
} finally {
|
|
||||||
unloadJars();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Open-ended "setup" routine that is called after the job is configured
|
|
||||||
* but just before it is submitted to MapReduce. Subclasses may override
|
|
||||||
* if necessary.
|
|
||||||
*/
|
|
||||||
protected void jobSetup(Job job) throws IOException, ImportException {
|
|
||||||
}
|
|
||||||
|
|
||||||
protected ImportJobContext getContext() {
|
|
||||||
return context;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,33 +18,16 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.io.DefaultStringifier;
|
|
||||||
import org.apache.hadoop.io.MapWritable;
|
|
||||||
import org.apache.hadoop.io.Text;
|
|
||||||
import org.apache.hadoop.mapreduce.InputFormat;
|
import org.apache.hadoop.mapreduce.InputFormat;
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
|
||||||
import org.apache.hadoop.mapreduce.Mapper;
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
import org.apache.hadoop.mapreduce.OutputFormat;
|
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||||
|
|
||||||
import com.cloudera.sqoop.manager.ConnManager;
|
|
||||||
import com.cloudera.sqoop.manager.ExportJobContext;
|
import com.cloudera.sqoop.manager.ExportJobContext;
|
||||||
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
|
||||||
import com.cloudera.sqoop.mapreduce.db.DBOutputFormat;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Run an export using JDBC (JDBC-based ExportOutputFormat).
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
*/
|
*/
|
||||||
public class JdbcExportJob extends ExportJobBase {
|
public class JdbcExportJob
|
||||||
|
extends org.apache.sqoop.mapreduce.JdbcExportJob {
|
||||||
private FileType fileType;
|
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
JdbcExportJob.class.getName());
|
|
||||||
|
|
||||||
public JdbcExportJob(final ExportJobContext context) {
|
public JdbcExportJob(final ExportJobContext context) {
|
||||||
super(context);
|
super(context);
|
||||||
@ -59,84 +40,5 @@ public JdbcExportJob(final ExportJobContext ctxt,
|
|||||||
super(ctxt, mapperClass, inputFormatClass, outputFormatClass);
|
super(ctxt, mapperClass, inputFormatClass, outputFormatClass);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void configureInputFormat(Job job, String tableName,
|
|
||||||
String tableClassName, String splitByCol)
|
|
||||||
throws ClassNotFoundException, IOException {
|
|
||||||
|
|
||||||
fileType = getInputFileType();
|
|
||||||
|
|
||||||
super.configureInputFormat(job, tableName, tableClassName, splitByCol);
|
|
||||||
|
|
||||||
if (fileType == FileType.AVRO_DATA_FILE) {
|
|
||||||
LOG.debug("Configuring for Avro export");
|
|
||||||
ConnManager connManager = context.getConnManager();
|
|
||||||
Map<String, Integer> columnTypeInts =
|
|
||||||
connManager.getColumnTypes(tableName, options.getSqlQuery());
|
|
||||||
MapWritable columnTypes = new MapWritable();
|
|
||||||
for (Map.Entry<String, Integer> e : columnTypeInts.entrySet()) {
|
|
||||||
Text columnName = new Text(e.getKey());
|
|
||||||
Text columnText = new Text(connManager.toJavaType(e.getValue()));
|
|
||||||
columnTypes.put(columnName, columnText);
|
|
||||||
}
|
|
||||||
DefaultStringifier.store(job.getConfiguration(), columnTypes,
|
|
||||||
AvroExportMapper.AVRO_COLUMN_TYPES_MAP);
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected Class<? extends InputFormat> getInputFormatClass()
|
|
||||||
throws ClassNotFoundException {
|
|
||||||
if (fileType == FileType.AVRO_DATA_FILE) {
|
|
||||||
return AvroInputFormat.class;
|
|
||||||
}
|
|
||||||
return super.getInputFormatClass();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected Class<? extends Mapper> getMapperClass() {
|
|
||||||
switch (fileType) {
|
|
||||||
case SEQUENCE_FILE:
|
|
||||||
return SequenceFileExportMapper.class;
|
|
||||||
case AVRO_DATA_FILE:
|
|
||||||
return AvroExportMapper.class;
|
|
||||||
case UNKNOWN:
|
|
||||||
default:
|
|
||||||
return TextExportMapper.class;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void configureOutputFormat(Job job, String tableName,
|
|
||||||
String tableClassName) throws IOException {
|
|
||||||
|
|
||||||
ConnManager mgr = context.getConnManager();
|
|
||||||
try {
|
|
||||||
String username = options.getUsername();
|
|
||||||
if (null == username || username.length() == 0) {
|
|
||||||
DBConfiguration.configureDB(job.getConfiguration(),
|
|
||||||
mgr.getDriverClass(),
|
|
||||||
options.getConnectString());
|
|
||||||
} else {
|
|
||||||
DBConfiguration.configureDB(job.getConfiguration(),
|
|
||||||
mgr.getDriverClass(),
|
|
||||||
options.getConnectString(),
|
|
||||||
username, options.getPassword());
|
|
||||||
}
|
|
||||||
|
|
||||||
String [] colNames = options.getColumns();
|
|
||||||
if (null == colNames) {
|
|
||||||
colNames = mgr.getColumnNames(tableName);
|
|
||||||
}
|
|
||||||
DBOutputFormat.setOutput(job, tableName, colNames);
|
|
||||||
|
|
||||||
job.setOutputFormatClass(getOutputFormatClass());
|
|
||||||
job.getConfiguration().set(SQOOP_EXPORT_TABLE_CLASS_KEY, tableClassName);
|
|
||||||
} catch (ClassNotFoundException cnfe) {
|
|
||||||
throw new IOException("Could not load OutputFormat", cnfe);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -21,43 +19,20 @@
|
|||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.LinkedHashSet;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.StringTokenizer;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.mapreduce.InputFormat;
|
import org.apache.hadoop.mapreduce.InputFormat;
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
|
||||||
import org.apache.hadoop.mapreduce.Mapper;
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
import org.apache.hadoop.mapreduce.OutputFormat;
|
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||||
|
|
||||||
import com.cloudera.sqoop.manager.ConnManager;
|
|
||||||
import com.cloudera.sqoop.manager.ExportJobContext;
|
import com.cloudera.sqoop.manager.ExportJobContext;
|
||||||
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
|
||||||
import com.cloudera.sqoop.mapreduce.db.DBOutputFormat;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Run an update-based export using JDBC (JDBC-based UpdateOutputFormat).
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
*/
|
*/
|
||||||
public class JdbcUpdateExportJob extends ExportJobBase {
|
public class JdbcUpdateExportJob
|
||||||
|
extends org.apache.sqoop.mapreduce.JdbcUpdateExportJob {
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
JdbcUpdateExportJob.class.getName());
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Return an instance of the UpdateOutputFormat class object loaded
|
|
||||||
* from the shim jar.
|
|
||||||
*/
|
|
||||||
private static Class<? extends OutputFormat> getUpdateOutputFormat()
|
|
||||||
throws IOException {
|
|
||||||
return UpdateOutputFormat.class;
|
|
||||||
}
|
|
||||||
|
|
||||||
public JdbcUpdateExportJob(final ExportJobContext context)
|
public JdbcUpdateExportJob(final ExportJobContext context)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
super(context, null, null, getUpdateOutputFormat());
|
super(context);
|
||||||
}
|
}
|
||||||
|
|
||||||
public JdbcUpdateExportJob(final ExportJobContext ctxt,
|
public JdbcUpdateExportJob(final ExportJobContext ctxt,
|
||||||
@ -67,82 +42,5 @@ public JdbcUpdateExportJob(final ExportJobContext ctxt,
|
|||||||
super(ctxt, mapperClass, inputFormatClass, outputFormatClass);
|
super(ctxt, mapperClass, inputFormatClass, outputFormatClass);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected Class<? extends Mapper> getMapperClass() {
|
|
||||||
if (inputIsSequenceFiles()) {
|
|
||||||
return SequenceFileExportMapper.class;
|
|
||||||
} else {
|
|
||||||
return TextExportMapper.class;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void configureOutputFormat(Job job, String tableName,
|
|
||||||
String tableClassName) throws IOException {
|
|
||||||
|
|
||||||
ConnManager mgr = context.getConnManager();
|
|
||||||
try {
|
|
||||||
String username = options.getUsername();
|
|
||||||
if (null == username || username.length() == 0) {
|
|
||||||
DBConfiguration.configureDB(job.getConfiguration(),
|
|
||||||
mgr.getDriverClass(),
|
|
||||||
options.getConnectString());
|
|
||||||
} else {
|
|
||||||
DBConfiguration.configureDB(job.getConfiguration(),
|
|
||||||
mgr.getDriverClass(),
|
|
||||||
options.getConnectString(),
|
|
||||||
username, options.getPassword());
|
|
||||||
}
|
|
||||||
|
|
||||||
String [] colNames = options.getColumns();
|
|
||||||
if (null == colNames) {
|
|
||||||
colNames = mgr.getColumnNames(tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (null == colNames) {
|
|
||||||
throw new IOException(
|
|
||||||
"Export column names could not be determined for " + tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
String updateKeyColumns = options.getUpdateKeyCol();
|
|
||||||
if (null == updateKeyColumns) {
|
|
||||||
throw new IOException("Update key column not set in export job");
|
|
||||||
}
|
|
||||||
// Update key columns lookup and removal
|
|
||||||
Set<String> updateKeys = new LinkedHashSet<String>();
|
|
||||||
Set<String> updateKeysUppercase = new HashSet<String>();
|
|
||||||
StringTokenizer stok = new StringTokenizer(updateKeyColumns, ",");
|
|
||||||
while (stok.hasMoreTokens()) {
|
|
||||||
String nextUpdateKey = stok.nextToken().trim();
|
|
||||||
if (nextUpdateKey.length() > 0) {
|
|
||||||
updateKeys.add(nextUpdateKey);
|
|
||||||
updateKeysUppercase.add(nextUpdateKey.toUpperCase());
|
|
||||||
} else {
|
|
||||||
throw new RuntimeException("Invalid update key column value specified"
|
|
||||||
+ ": '" + updateKeyColumns + "'");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (updateKeys.size() == 0) {
|
|
||||||
throw new IOException("Unpdate key columns not valid in export job");
|
|
||||||
}
|
|
||||||
|
|
||||||
// Make sure we strip out the key column from this list.
|
|
||||||
String [] outColNames = new String[colNames.length - updateKeys.size()];
|
|
||||||
int j = 0;
|
|
||||||
for (int i = 0; i < colNames.length; i++) {
|
|
||||||
if (!updateKeysUppercase.contains(colNames[i].toUpperCase())) {
|
|
||||||
outColNames[j++] = colNames[i];
|
|
||||||
}
|
|
||||||
}
|
|
||||||
DBOutputFormat.setOutput(job, tableName, outColNames);
|
|
||||||
|
|
||||||
job.setOutputFormatClass(getOutputFormatClass());
|
|
||||||
job.getConfiguration().set(SQOOP_EXPORT_TABLE_CLASS_KEY, tableClassName);
|
|
||||||
job.getConfiguration().set(SQOOP_EXPORT_UPDATE_COL_KEY, updateKeyColumns);
|
|
||||||
} catch (ClassNotFoundException cnfe) {
|
|
||||||
throw new IOException("Could not load OutputFormat", cnfe);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -21,89 +19,20 @@
|
|||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.LinkedHashSet;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.StringTokenizer;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
|
||||||
import org.apache.hadoop.mapreduce.OutputFormat;
|
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||||
|
|
||||||
import com.cloudera.sqoop.manager.ConnManager;
|
|
||||||
import com.cloudera.sqoop.manager.ExportJobContext;
|
import com.cloudera.sqoop.manager.ExportJobContext;
|
||||||
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
|
||||||
import com.cloudera.sqoop.mapreduce.db.DBOutputFormat;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Run an update/insert export using JDBC (JDBC-based UpsertOutputFormat).
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
*/
|
*/
|
||||||
public class JdbcUpsertExportJob extends JdbcUpdateExportJob {
|
public class JdbcUpsertExportJob
|
||||||
|
extends org.apache.sqoop.mapreduce.JdbcUpsertExportJob {
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
JdbcUpsertExportJob.class.getName());
|
|
||||||
|
|
||||||
public JdbcUpsertExportJob(final ExportJobContext context,
|
public JdbcUpsertExportJob(final ExportJobContext context,
|
||||||
final Class<? extends OutputFormat> outputFormatClass)
|
final Class<? extends OutputFormat> outputFormatClass)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
super(context, null, null, outputFormatClass);
|
super(context, outputFormatClass);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void configureOutputFormat(Job job, String tableName,
|
|
||||||
String tableClassName) throws IOException {
|
|
||||||
|
|
||||||
ConnManager mgr = context.getConnManager();
|
|
||||||
try {
|
|
||||||
String username = options.getUsername();
|
|
||||||
if (null == username || username.length() == 0) {
|
|
||||||
DBConfiguration.configureDB(job.getConfiguration(),
|
|
||||||
mgr.getDriverClass(),
|
|
||||||
options.getConnectString());
|
|
||||||
} else {
|
|
||||||
DBConfiguration.configureDB(job.getConfiguration(),
|
|
||||||
mgr.getDriverClass(),
|
|
||||||
options.getConnectString(),
|
|
||||||
username, options.getPassword());
|
|
||||||
}
|
|
||||||
|
|
||||||
String [] colNames = options.getColumns();
|
|
||||||
if (null == colNames) {
|
|
||||||
colNames = mgr.getColumnNames(tableName);
|
|
||||||
}
|
|
||||||
if (null == colNames) {
|
|
||||||
throw new IOException(
|
|
||||||
"Export column names could not be determined for " + tableName);
|
|
||||||
}
|
|
||||||
DBOutputFormat.setOutput(job, tableName, colNames);
|
|
||||||
|
|
||||||
String updateKeyColumns = options.getUpdateKeyCol();
|
|
||||||
if (null == updateKeyColumns) {
|
|
||||||
throw new IOException("Update key column not set in export job");
|
|
||||||
}
|
|
||||||
// Update key columns lookup and removal
|
|
||||||
Set<String> updateKeys = new LinkedHashSet<String>();
|
|
||||||
StringTokenizer stok = new StringTokenizer(updateKeyColumns, ",");
|
|
||||||
while (stok.hasMoreTokens()) {
|
|
||||||
String nextUpdateKey = stok.nextToken().trim();
|
|
||||||
if (nextUpdateKey.length() > 0) {
|
|
||||||
updateKeys.add(nextUpdateKey);
|
|
||||||
} else {
|
|
||||||
throw new RuntimeException("Invalid update key column value specified"
|
|
||||||
+ ": '" + updateKeyColumns + "'");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (updateKeys.size() == 0) {
|
|
||||||
throw new IOException("Unpdate key columns not valid in export job");
|
|
||||||
}
|
|
||||||
|
|
||||||
job.setOutputFormatClass(getOutputFormatClass());
|
|
||||||
job.getConfiguration().set(SQOOP_EXPORT_TABLE_CLASS_KEY, tableClassName);
|
|
||||||
job.getConfiguration().set(SQOOP_EXPORT_UPDATE_COL_KEY, updateKeyColumns);
|
|
||||||
} catch (ClassNotFoundException cnfe) {
|
|
||||||
throw new IOException("Could not load OutputFormat", cnfe);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,316 +18,30 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.File;
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Set;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.mapreduce.InputFormat;
|
import org.apache.hadoop.mapreduce.InputFormat;
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
|
||||||
import org.apache.hadoop.mapreduce.Mapper;
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
import org.apache.hadoop.mapreduce.OutputFormat;
|
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||||
|
|
||||||
import org.apache.hadoop.util.StringUtils;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.SqoopOptions;
|
import com.cloudera.sqoop.SqoopOptions;
|
||||||
|
|
||||||
import com.cloudera.sqoop.config.ConfigurationHelper;
|
|
||||||
import com.cloudera.sqoop.manager.ConnManager;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.tool.SqoopTool;
|
|
||||||
import com.cloudera.sqoop.util.ClassLoaderStack;
|
|
||||||
import com.cloudera.sqoop.util.Jars;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Base class for configuring and running a MapReduce job.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* Allows dependency injection, etc, for easy customization of import job types.
|
|
||||||
*/
|
*/
|
||||||
public class JobBase {
|
public class JobBase
|
||||||
|
extends org.apache.sqoop.mapreduce.JobBase {
|
||||||
public static final Log LOG = LogFactory.getLog(JobBase.class.getName());
|
|
||||||
|
|
||||||
protected SqoopOptions options;
|
|
||||||
protected Class<? extends Mapper> mapperClass;
|
|
||||||
protected Class<? extends InputFormat> inputFormatClass;
|
|
||||||
protected Class<? extends OutputFormat> outputFormatClass;
|
|
||||||
|
|
||||||
private Job mrJob;
|
|
||||||
|
|
||||||
private ClassLoader prevClassLoader = null;
|
|
||||||
|
|
||||||
public JobBase() {
|
public JobBase() {
|
||||||
this(null);
|
super();
|
||||||
}
|
}
|
||||||
|
|
||||||
public JobBase(final SqoopOptions opts) {
|
public JobBase(final SqoopOptions opts) {
|
||||||
this(opts, null, null, null);
|
super(opts);
|
||||||
}
|
}
|
||||||
|
|
||||||
public JobBase(final SqoopOptions opts,
|
public JobBase(final SqoopOptions opts,
|
||||||
final Class<? extends Mapper> mapperClass,
|
final Class<? extends Mapper> mapperClass,
|
||||||
final Class<? extends InputFormat> inputFormatClass,
|
final Class<? extends InputFormat> inputFormatClass,
|
||||||
final Class<? extends OutputFormat> outputFormatClass) {
|
final Class<? extends OutputFormat> outputFormatClass) {
|
||||||
|
super(opts, mapperClass, inputFormatClass, outputFormatClass);
|
||||||
this.options = opts;
|
|
||||||
this.mapperClass = mapperClass;
|
|
||||||
this.inputFormatClass = inputFormatClass;
|
|
||||||
this.outputFormatClass = outputFormatClass;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @return the mapper class to use for the job.
|
|
||||||
*/
|
|
||||||
protected Class<? extends Mapper> getMapperClass()
|
|
||||||
throws ClassNotFoundException {
|
|
||||||
return this.mapperClass;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return the inputformat class to use for the job.
|
|
||||||
*/
|
|
||||||
protected Class<? extends InputFormat> getInputFormatClass()
|
|
||||||
throws ClassNotFoundException {
|
|
||||||
return this.inputFormatClass;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return the outputformat class to use for the job.
|
|
||||||
*/
|
|
||||||
protected Class<? extends OutputFormat> getOutputFormatClass()
|
|
||||||
throws ClassNotFoundException {
|
|
||||||
return this.outputFormatClass;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Set the OutputFormat class to use for this job. */
|
|
||||||
public void setOutputFormatClass(Class<? extends OutputFormat> cls) {
|
|
||||||
this.outputFormatClass = cls;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Set the InputFormat class to use for this job. */
|
|
||||||
public void setInputFormatClass(Class<? extends InputFormat> cls) {
|
|
||||||
this.inputFormatClass = cls;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Set the Mapper class to use for this job. */
|
|
||||||
public void setMapperClass(Class<? extends Mapper> cls) {
|
|
||||||
this.mapperClass = cls;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Set the SqoopOptions configuring this job.
|
|
||||||
*/
|
|
||||||
public void setOptions(SqoopOptions opts) {
|
|
||||||
this.options = opts;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Put jar files required by Sqoop into the DistributedCache.
|
|
||||||
* @param job the Job being submitted.
|
|
||||||
* @param mgr the ConnManager to use.
|
|
||||||
*/
|
|
||||||
protected void cacheJars(Job job, ConnManager mgr)
|
|
||||||
throws IOException {
|
|
||||||
|
|
||||||
Configuration conf = job.getConfiguration();
|
|
||||||
FileSystem fs = FileSystem.getLocal(conf);
|
|
||||||
Set<String> localUrls = new HashSet<String>();
|
|
||||||
|
|
||||||
addToCache(Jars.getSqoopJarPath(), fs, localUrls);
|
|
||||||
if (null != mgr) {
|
|
||||||
addToCache(Jars.getDriverClassJar(mgr), fs, localUrls);
|
|
||||||
addToCache(Jars.getJarPathForClass(mgr.getClass()), fs, localUrls);
|
|
||||||
}
|
|
||||||
|
|
||||||
SqoopTool tool = this.options.getActiveSqoopTool();
|
|
||||||
if (null != tool) {
|
|
||||||
// Make sure the jar for the tool itself is on the classpath. (In case
|
|
||||||
// this is a third-party plugin tool.)
|
|
||||||
addToCache(Jars.getJarPathForClass(tool.getClass()), fs, localUrls);
|
|
||||||
List<String> toolDeps = tool.getDependencyJars();
|
|
||||||
if (null != toolDeps) {
|
|
||||||
for (String depFile : toolDeps) {
|
|
||||||
addToCache(depFile, fs, localUrls);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// If the user specified a particular jar file name,
|
|
||||||
|
|
||||||
// Add anything in $SQOOP_HOME/lib, if this is set.
|
|
||||||
String sqoopHome = System.getenv("SQOOP_HOME");
|
|
||||||
if (null != sqoopHome) {
|
|
||||||
File sqoopHomeFile = new File(sqoopHome);
|
|
||||||
File sqoopLibFile = new File(sqoopHomeFile, "lib");
|
|
||||||
if (sqoopLibFile.exists()) {
|
|
||||||
addDirToCache(sqoopLibFile, fs, localUrls);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
LOG.warn("SQOOP_HOME is unset. May not be able to find "
|
|
||||||
+ "all job dependencies.");
|
|
||||||
}
|
|
||||||
|
|
||||||
// If we didn't put anything in our set, then there's nothing to cache.
|
|
||||||
if (localUrls.isEmpty()) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
// Add these to the 'tmpjars' array, which the MR JobSubmitter
|
|
||||||
// will upload to HDFS and put in the DistributedCache libjars.
|
|
||||||
String tmpjars = conf.get("tmpjars");
|
|
||||||
StringBuilder sb = new StringBuilder();
|
|
||||||
if (null != tmpjars) {
|
|
||||||
sb.append(tmpjars);
|
|
||||||
sb.append(",");
|
|
||||||
}
|
|
||||||
sb.append(StringUtils.arrayToString(localUrls.toArray(new String[0])));
|
|
||||||
conf.set("tmpjars", sb.toString());
|
|
||||||
}
|
|
||||||
|
|
||||||
private void addToCache(String file, FileSystem fs, Set<String> localUrls) {
|
|
||||||
if (null == file) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
Path p = new Path(file);
|
|
||||||
String qualified = p.makeQualified(fs).toString();
|
|
||||||
LOG.debug("Adding to job classpath: " + qualified);
|
|
||||||
localUrls.add(qualified);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Add the .jar elements of a directory to the DCache classpath,
|
|
||||||
* nonrecursively.
|
|
||||||
*/
|
|
||||||
private void addDirToCache(File dir, FileSystem fs, Set<String> localUrls) {
|
|
||||||
if (null == dir) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
for (File libfile : dir.listFiles()) {
|
|
||||||
if (libfile.exists() && !libfile.isDirectory()
|
|
||||||
&& libfile.getName().endsWith("jar")) {
|
|
||||||
addToCache(libfile.toString(), fs, localUrls);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* If jars must be loaded into the local environment, do so here.
|
|
||||||
*/
|
|
||||||
protected void loadJars(Configuration conf, String ormJarFile,
|
|
||||||
String tableClassName) throws IOException {
|
|
||||||
boolean isLocal = "local".equals(conf.get("mapreduce.jobtracker.address"))
|
|
||||||
|| "local".equals(conf.get("mapred.job.tracker"));
|
|
||||||
if (isLocal) {
|
|
||||||
// If we're using the LocalJobRunner, then instead of using the compiled
|
|
||||||
// jar file as the job source, we're running in the current thread. Push
|
|
||||||
// on another classloader that loads from that jar in addition to
|
|
||||||
// everything currently on the classpath.
|
|
||||||
this.prevClassLoader = ClassLoaderStack.addJarFile(ormJarFile,
|
|
||||||
tableClassName);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* If any classloader was invoked by loadJars, free it here.
|
|
||||||
*/
|
|
||||||
protected void unloadJars() {
|
|
||||||
if (null != this.prevClassLoader) {
|
|
||||||
// unload the special classloader for this jar.
|
|
||||||
ClassLoaderStack.setCurrentClassLoader(this.prevClassLoader);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Configure the inputformat to use for the job.
|
|
||||||
*/
|
|
||||||
protected void configureInputFormat(Job job, String tableName,
|
|
||||||
String tableClassName, String splitByCol)
|
|
||||||
throws ClassNotFoundException, IOException {
|
|
||||||
//TODO: 'splitByCol' is import-job specific; lift it out of this API.
|
|
||||||
Class<? extends InputFormat> ifClass = getInputFormatClass();
|
|
||||||
LOG.debug("Using InputFormat: " + ifClass);
|
|
||||||
job.setInputFormatClass(ifClass);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Configure the output format to use for the job.
|
|
||||||
*/
|
|
||||||
protected void configureOutputFormat(Job job, String tableName,
|
|
||||||
String tableClassName) throws ClassNotFoundException, IOException {
|
|
||||||
Class<? extends OutputFormat> ofClass = getOutputFormatClass();
|
|
||||||
LOG.debug("Using OutputFormat: " + ofClass);
|
|
||||||
job.setOutputFormatClass(ofClass);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Set the mapper class implementation to use in the job,
|
|
||||||
* as well as any related configuration (e.g., map output types).
|
|
||||||
*/
|
|
||||||
protected void configureMapper(Job job, String tableName,
|
|
||||||
String tableClassName) throws ClassNotFoundException, IOException {
|
|
||||||
job.setMapperClass(getMapperClass());
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Configure the number of map/reduce tasks to use in the job.
|
|
||||||
*/
|
|
||||||
protected int configureNumTasks(Job job) throws IOException {
|
|
||||||
int numMapTasks = options.getNumMappers();
|
|
||||||
if (numMapTasks < 1) {
|
|
||||||
numMapTasks = SqoopOptions.DEFAULT_NUM_MAPPERS;
|
|
||||||
LOG.warn("Invalid mapper count; using " + numMapTasks + " mappers.");
|
|
||||||
}
|
|
||||||
|
|
||||||
ConfigurationHelper.setJobNumMaps(job, numMapTasks);
|
|
||||||
job.setNumReduceTasks(0);
|
|
||||||
return numMapTasks;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Set the main job that will be run. */
|
|
||||||
protected void setJob(Job job) {
|
|
||||||
mrJob = job;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return the main MapReduce job that is being run, or null if no
|
|
||||||
* job has started.
|
|
||||||
*/
|
|
||||||
public Job getJob() {
|
|
||||||
return mrJob;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Actually run the MapReduce job.
|
|
||||||
*/
|
|
||||||
protected boolean runJob(Job job) throws ClassNotFoundException, IOException,
|
|
||||||
InterruptedException {
|
|
||||||
return job.waitForCompletion(true);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Display a notice on the log that the current MapReduce job has
|
|
||||||
* been retired, and thus Counters are unavailable.
|
|
||||||
* @param log the Log to display the info to.
|
|
||||||
*/
|
|
||||||
protected void displayRetiredJobNotice(Log log) {
|
|
||||||
log.info("The MapReduce job has already been retired. Performance");
|
|
||||||
log.info("counters are unavailable. To get this information, ");
|
|
||||||
log.info("you will need to enable the completed job store on ");
|
|
||||||
log.info("the jobtracker with:");
|
|
||||||
log.info("mapreduce.jobtracker.persist.jobstatus.active = true");
|
|
||||||
log.info("mapreduce.jobtracker.persist.jobstatus.hours = 1");
|
|
||||||
log.info("A jobtracker restart is required for these settings");
|
|
||||||
log.info("to take effect.");
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,128 +18,27 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.io.NullWritable;
|
|
||||||
import org.apache.hadoop.io.Text;
|
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.SqoopOptions;
|
import com.cloudera.sqoop.SqoopOptions;
|
||||||
import com.cloudera.sqoop.util.Jars;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Run a MapReduce job that merges two datasets.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
*/
|
*/
|
||||||
public class MergeJob extends JobBase {
|
public class MergeJob
|
||||||
|
extends org.apache.sqoop.mapreduce.MergeJob {
|
||||||
|
|
||||||
/** Configuration key specifying the path to the "old" dataset. */
|
public static final String MERGE_OLD_PATH_KEY =
|
||||||
public static final String MERGE_OLD_PATH_KEY = "sqoop.merge.old.path";
|
org.apache.sqoop.mapreduce.MergeJob.MERGE_OLD_PATH_KEY;
|
||||||
|
public static final String MERGE_NEW_PATH_KEY =
|
||||||
/** Configuration key specifying the path to the "new" dataset. */
|
org.apache.sqoop.mapreduce.MergeJob.MERGE_NEW_PATH_KEY;
|
||||||
public static final String MERGE_NEW_PATH_KEY = "sqoop.merge.new.path";
|
public static final String MERGE_KEY_COL_KEY =
|
||||||
|
org.apache.sqoop.mapreduce.MergeJob.MERGE_KEY_COL_KEY;
|
||||||
/** Configuration key specifying the name of the key column for joins. */
|
public static final String MERGE_SQOOP_RECORD_KEY =
|
||||||
public static final String MERGE_KEY_COL_KEY = "sqoop.merge.key.col";
|
org.apache.sqoop.mapreduce.MergeJob.MERGE_SQOOP_RECORD_KEY;
|
||||||
|
|
||||||
/** Configuration key specifying the SqoopRecord class name for
|
|
||||||
* the records we are merging.
|
|
||||||
*/
|
|
||||||
public static final String MERGE_SQOOP_RECORD_KEY = "sqoop.merge.class";
|
|
||||||
|
|
||||||
public MergeJob(final SqoopOptions opts) {
|
public MergeJob(final SqoopOptions opts) {
|
||||||
super(opts, null, null, null);
|
super(opts);
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean runMergeJob() throws IOException {
|
|
||||||
Configuration conf = options.getConf();
|
|
||||||
Job job = new Job(conf);
|
|
||||||
|
|
||||||
String userClassName = options.getClassName();
|
|
||||||
if (null == userClassName) {
|
|
||||||
// Shouldn't get here.
|
|
||||||
throw new IOException("Record class name not specified with "
|
|
||||||
+ "--class-name.");
|
|
||||||
}
|
|
||||||
|
|
||||||
// Set the external jar to use for the job.
|
|
||||||
String existingJar = options.getExistingJarName();
|
|
||||||
if (existingJar != null) {
|
|
||||||
// User explicitly identified a jar path.
|
|
||||||
LOG.debug("Setting job jar to user-specified jar: " + existingJar);
|
|
||||||
job.getConfiguration().set("mapred.jar", existingJar);
|
|
||||||
} else {
|
|
||||||
// Infer it from the location of the specified class, if it's on the
|
|
||||||
// classpath.
|
|
||||||
try {
|
|
||||||
Class<? extends Object> userClass = conf.getClassByName(userClassName);
|
|
||||||
if (null != userClass) {
|
|
||||||
String userJar = Jars.getJarPathForClass(userClass);
|
|
||||||
LOG.debug("Setting job jar based on user class " + userClassName
|
|
||||||
+ ": " + userJar);
|
|
||||||
job.getConfiguration().set("mapred.jar", userJar);
|
|
||||||
} else {
|
|
||||||
LOG.warn("Specified class " + userClassName + " is not in a jar. "
|
|
||||||
+ "MapReduce may not find the class");
|
|
||||||
}
|
|
||||||
} catch (ClassNotFoundException cnfe) {
|
|
||||||
throw new IOException(cnfe);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
try {
|
|
||||||
Path oldPath = new Path(options.getMergeOldPath());
|
|
||||||
Path newPath = new Path(options.getMergeNewPath());
|
|
||||||
|
|
||||||
Configuration jobConf = job.getConfiguration();
|
|
||||||
FileSystem fs = FileSystem.get(jobConf);
|
|
||||||
oldPath = oldPath.makeQualified(fs);
|
|
||||||
newPath = newPath.makeQualified(fs);
|
|
||||||
|
|
||||||
FileInputFormat.addInputPath(job, oldPath);
|
|
||||||
FileInputFormat.addInputPath(job, newPath);
|
|
||||||
|
|
||||||
jobConf.set(MERGE_OLD_PATH_KEY, oldPath.toString());
|
|
||||||
jobConf.set(MERGE_NEW_PATH_KEY, newPath.toString());
|
|
||||||
jobConf.set(MERGE_KEY_COL_KEY, options.getMergeKeyCol());
|
|
||||||
jobConf.set(MERGE_SQOOP_RECORD_KEY, userClassName);
|
|
||||||
|
|
||||||
FileOutputFormat.setOutputPath(job, new Path(options.getTargetDir()));
|
|
||||||
|
|
||||||
if (ExportJobBase.isSequenceFiles(jobConf, newPath)) {
|
|
||||||
job.setInputFormatClass(SequenceFileInputFormat.class);
|
|
||||||
job.setOutputFormatClass(SequenceFileOutputFormat.class);
|
|
||||||
job.setMapperClass(MergeRecordMapper.class);
|
|
||||||
} else {
|
|
||||||
job.setMapperClass(MergeTextMapper.class);
|
|
||||||
job.setOutputFormatClass(RawKeyTextOutputFormat.class);
|
|
||||||
}
|
|
||||||
|
|
||||||
jobConf.set("mapred.output.key.class", userClassName);
|
|
||||||
job.setOutputValueClass(NullWritable.class);
|
|
||||||
|
|
||||||
job.setReducerClass(MergeReducer.class);
|
|
||||||
|
|
||||||
// Set the intermediate data types.
|
|
||||||
job.setMapOutputKeyClass(Text.class);
|
|
||||||
job.setMapOutputValueClass(MergeRecord.class);
|
|
||||||
|
|
||||||
// Make sure Sqoop and anything else we need is on the classpath.
|
|
||||||
cacheJars(job, null);
|
|
||||||
setJob(job);
|
|
||||||
return this.runJob(job);
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
throw new IOException(ie);
|
|
||||||
} catch (ClassNotFoundException cnfe) {
|
|
||||||
throw new IOException(cnfe);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,72 +18,9 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.io.Text;
|
|
||||||
import org.apache.hadoop.mapreduce.InputSplit;
|
|
||||||
import org.apache.hadoop.mapreduce.Mapper;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.input.FileSplit;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Given a set of SqoopRecord instances which are from a "new" dataset
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* or an "old" dataset, extract a key column from the record and tag
|
|
||||||
* each record with a bit specifying whether it is a new or old record.
|
|
||||||
*/
|
*/
|
||||||
public class MergeMapperBase<INKEY, INVAL>
|
public class MergeMapperBase<INKEY, INVAL>
|
||||||
extends Mapper<INKEY, INVAL, Text, MergeRecord> {
|
extends org.apache.sqoop.mapreduce.MergeMapperBase<INKEY, INVAL> {
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
MergeMapperBase.class.getName());
|
|
||||||
|
|
||||||
private String keyColName; // name of the key column.
|
|
||||||
private boolean isNew; // true if this split is from the new dataset.
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void setup(Context context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
Configuration conf = context.getConfiguration();
|
|
||||||
keyColName = conf.get(MergeJob.MERGE_KEY_COL_KEY);
|
|
||||||
|
|
||||||
InputSplit is = context.getInputSplit();
|
|
||||||
FileSplit fs = (FileSplit) is;
|
|
||||||
Path splitPath = fs.getPath();
|
|
||||||
|
|
||||||
if (splitPath.toString().startsWith(
|
|
||||||
conf.get(MergeJob.MERGE_NEW_PATH_KEY))) {
|
|
||||||
this.isNew = true;
|
|
||||||
} else if (splitPath.toString().startsWith(
|
|
||||||
conf.get(MergeJob.MERGE_OLD_PATH_KEY))) {
|
|
||||||
this.isNew = false;
|
|
||||||
} else {
|
|
||||||
throw new IOException("File " + splitPath + " is not under new path "
|
|
||||||
+ conf.get(MergeJob.MERGE_NEW_PATH_KEY) + " or old path "
|
|
||||||
+ conf.get(MergeJob.MERGE_OLD_PATH_KEY));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
protected void processRecord(SqoopRecord r, Context c)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
MergeRecord mr = new MergeRecord(r, isNew);
|
|
||||||
Map<String, Object> fieldMap = r.getFieldMap();
|
|
||||||
if (null == fieldMap) {
|
|
||||||
throw new IOException("No field map in record " + r);
|
|
||||||
}
|
|
||||||
Object keyObj = fieldMap.get(keyColName);
|
|
||||||
if (null == keyObj) {
|
|
||||||
throw new IOException("Cannot join values on null key. "
|
|
||||||
+ "Did you specify a key column that exists?");
|
|
||||||
} else {
|
|
||||||
c.write(new Text(keyObj.toString()), mr);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,117 +18,20 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.DataInput;
|
|
||||||
import java.io.DataOutput;
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configurable;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
|
|
||||||
import org.apache.hadoop.io.Text;
|
|
||||||
import org.apache.hadoop.io.Writable;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Class that holds a record to be merged. This contains a SqoopRecord which
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* is the "guts" of the item, and a boolean value indicating whether it is a
|
|
||||||
* "new" record or an "old" record. In the Reducer, we prefer to emit a new
|
|
||||||
* record rather than an old one, if a new one is available.
|
|
||||||
*/
|
*/
|
||||||
public class MergeRecord implements Configurable, Writable {
|
public class MergeRecord
|
||||||
private SqoopRecord sqoopRecord;
|
extends org.apache.sqoop.mapreduce.MergeRecord {
|
||||||
private boolean isNew;
|
|
||||||
private Configuration config;
|
|
||||||
|
|
||||||
/** Construct an empty MergeRecord. */
|
|
||||||
public MergeRecord() {
|
public MergeRecord() {
|
||||||
this.sqoopRecord = null;
|
super();
|
||||||
this.isNew = false;
|
|
||||||
this.config = new Configuration();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Construct a MergeRecord with all fields initialized.
|
|
||||||
*/
|
|
||||||
public MergeRecord(SqoopRecord sr, boolean recordIsNew) {
|
public MergeRecord(SqoopRecord sr, boolean recordIsNew) {
|
||||||
this.sqoopRecord = sr;
|
super(sr, recordIsNew);
|
||||||
this.isNew = recordIsNew;
|
|
||||||
this.config = new Configuration();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public void setConf(Configuration conf) {
|
|
||||||
this.config = conf;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public Configuration getConf() {
|
|
||||||
return this.config;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** @return true if this record came from the "new" dataset. */
|
|
||||||
public boolean isNewRecord() {
|
|
||||||
return isNew;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Set the isNew field to 'newVal'.
|
|
||||||
*/
|
|
||||||
public void setNewRecord(boolean newVal) {
|
|
||||||
this.isNew = newVal;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return the underlying SqoopRecord we're shipping.
|
|
||||||
*/
|
|
||||||
public SqoopRecord getSqoopRecord() {
|
|
||||||
return this.sqoopRecord;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Set the SqoopRecord instance we should pass from the mapper to the
|
|
||||||
* reducer.
|
|
||||||
*/
|
|
||||||
public void setSqoopRecord(SqoopRecord record) {
|
|
||||||
this.sqoopRecord = record;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/**
|
|
||||||
* {@inheritDoc}
|
|
||||||
*/
|
|
||||||
public void readFields(DataInput in) throws IOException {
|
|
||||||
this.isNew = in.readBoolean();
|
|
||||||
String className = Text.readString(in);
|
|
||||||
if (null == this.sqoopRecord) {
|
|
||||||
// If we haven't already instantiated an inner SqoopRecord, do so here.
|
|
||||||
try {
|
|
||||||
Class<? extends SqoopRecord> recordClass =
|
|
||||||
(Class<? extends SqoopRecord>) config.getClassByName(className);
|
|
||||||
this.sqoopRecord = recordClass.newInstance();
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new IOException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
this.sqoopRecord.readFields(in);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/**
|
|
||||||
* {@inheritDoc}
|
|
||||||
*/
|
|
||||||
public void write(DataOutput out) throws IOException {
|
|
||||||
out.writeBoolean(this.isNew);
|
|
||||||
Text.writeString(out, this.sqoopRecord.getClass().getName());
|
|
||||||
this.sqoopRecord.write(out);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString() {
|
|
||||||
return "" + this.sqoopRecord;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,20 +18,9 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.io.LongWritable;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Mapper for the merge program which operates on SequenceFiles.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
*/
|
*/
|
||||||
public class MergeRecordMapper
|
public class MergeRecordMapper
|
||||||
extends MergeMapperBase<LongWritable, SqoopRecord> {
|
extends org.apache.sqoop.mapreduce.MergeRecordMapper {
|
||||||
|
|
||||||
public void map(LongWritable key, SqoopRecord val, Context c)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
processRecord(val, c);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,41 +18,10 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.io.NullWritable;
|
|
||||||
import org.apache.hadoop.io.Text;
|
|
||||||
import org.apache.hadoop.mapreduce.Reducer;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Reducer for merge tool. Given records tagged as 'old' or 'new', emit
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* a new one if possible; otherwise, an old one.
|
|
||||||
*/
|
*/
|
||||||
public class MergeReducer
|
public class MergeReducer
|
||||||
extends Reducer<Text, MergeRecord, SqoopRecord, NullWritable> {
|
extends org.apache.sqoop.mapreduce.MergeReducer {
|
||||||
|
|
||||||
@Override
|
|
||||||
public void reduce(Text key, Iterable<MergeRecord> vals, Context c)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
SqoopRecord bestRecord = null;
|
|
||||||
try {
|
|
||||||
for (MergeRecord val : vals) {
|
|
||||||
if (null == bestRecord && !val.isNewRecord()) {
|
|
||||||
// Use an old record if we don't have a new record.
|
|
||||||
bestRecord = (SqoopRecord) val.getSqoopRecord().clone();
|
|
||||||
} else if (val.isNewRecord()) {
|
|
||||||
bestRecord = (SqoopRecord) val.getSqoopRecord().clone();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} catch (CloneNotSupportedException cnse) {
|
|
||||||
throw new IOException(cnse);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (null != bestRecord) {
|
|
||||||
c.write(bestRecord, NullWritable.get());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,44 +18,9 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.io.LongWritable;
|
|
||||||
import org.apache.hadoop.io.Text;
|
|
||||||
import org.apache.hadoop.util.ReflectionUtils;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.lib.RecordParser;
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Mapper for the merge program which operates on text files that we need to
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* parse into SqoopRecord instances.
|
|
||||||
*/
|
*/
|
||||||
public class MergeTextMapper extends MergeMapperBase<LongWritable, Text> {
|
public class MergeTextMapper
|
||||||
|
extends org.apache.sqoop.mapreduce.MergeTextMapper {
|
||||||
private SqoopRecord record;
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void setup(Context c) throws IOException, InterruptedException {
|
|
||||||
Configuration conf = c.getConfiguration();
|
|
||||||
|
|
||||||
Class<? extends SqoopRecord> recordClass =
|
|
||||||
(Class<? extends SqoopRecord>) conf.getClass(
|
|
||||||
MergeJob.MERGE_SQOOP_RECORD_KEY, SqoopRecord.class);
|
|
||||||
this.record = ReflectionUtils.newInstance(recordClass, conf);
|
|
||||||
|
|
||||||
super.setup(c);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void map(LongWritable key, Text val, Context c)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
try {
|
|
||||||
this.record.parse(val);
|
|
||||||
} catch (RecordParser.ParseError pe) {
|
|
||||||
throw new IOException(pe);
|
|
||||||
}
|
|
||||||
|
|
||||||
processRecord(this.record, c);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,115 +18,18 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.io.NullWritable;
|
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.db.DBWritable;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.SqoopOptions;
|
import com.cloudera.sqoop.SqoopOptions;
|
||||||
import com.cloudera.sqoop.manager.ConnManager;
|
|
||||||
import com.cloudera.sqoop.manager.ImportJobContext;
|
import com.cloudera.sqoop.manager.ImportJobContext;
|
||||||
import com.cloudera.sqoop.manager.MySQLUtils;
|
|
||||||
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
|
||||||
import com.cloudera.sqoop.mapreduce.db.DataDrivenDBInputFormat;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Class that runs an import job using mysqldump in the mapper.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
*/
|
*/
|
||||||
public class MySQLDumpImportJob extends ImportJobBase {
|
public class MySQLDumpImportJob
|
||||||
|
extends org.apache.sqoop.mapreduce.MySQLDumpImportJob {
|
||||||
public static final Log LOG =
|
|
||||||
LogFactory.getLog(MySQLDumpImportJob.class.getName());
|
|
||||||
|
|
||||||
public MySQLDumpImportJob(final SqoopOptions opts, ImportJobContext context)
|
public MySQLDumpImportJob(final SqoopOptions opts, ImportJobContext context)
|
||||||
throws ClassNotFoundException {
|
throws ClassNotFoundException {
|
||||||
super(opts, MySQLDumpMapper.class, MySQLDumpInputFormat.class,
|
super(opts, context);
|
||||||
RawKeyTextOutputFormat.class, context);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Configure the inputformat to use for the job.
|
|
||||||
*/
|
|
||||||
protected void configureInputFormat(Job job, String tableName,
|
|
||||||
String tableClassName, String splitByCol)
|
|
||||||
throws ClassNotFoundException, IOException {
|
|
||||||
|
|
||||||
if (null == tableName) {
|
|
||||||
LOG.error(
|
|
||||||
"mysqldump-based import cannot support free-form query imports.");
|
|
||||||
LOG.error("Do not use --direct and --query together for MySQL.");
|
|
||||||
throw new IOException("null tableName for MySQLDumpImportJob.");
|
|
||||||
}
|
|
||||||
|
|
||||||
ConnManager mgr = getContext().getConnManager();
|
|
||||||
String username = options.getUsername();
|
|
||||||
if (null == username || username.length() == 0) {
|
|
||||||
DBConfiguration.configureDB(job.getConfiguration(),
|
|
||||||
mgr.getDriverClass(), options.getConnectString());
|
|
||||||
} else {
|
|
||||||
DBConfiguration.configureDB(job.getConfiguration(),
|
|
||||||
mgr.getDriverClass(), options.getConnectString(), username,
|
|
||||||
options.getPassword());
|
|
||||||
}
|
|
||||||
|
|
||||||
String [] colNames = options.getColumns();
|
|
||||||
if (null == colNames) {
|
|
||||||
colNames = mgr.getColumnNames(tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
String [] sqlColNames = null;
|
|
||||||
if (null != colNames) {
|
|
||||||
sqlColNames = new String[colNames.length];
|
|
||||||
for (int i = 0; i < colNames.length; i++) {
|
|
||||||
sqlColNames[i] = mgr.escapeColName(colNames[i]);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// It's ok if the where clause is null in DBInputFormat.setInput.
|
|
||||||
String whereClause = options.getWhereClause();
|
|
||||||
|
|
||||||
// We can't set the class properly in here, because we may not have the
|
|
||||||
// jar loaded in this JVM. So we start by calling setInput() with
|
|
||||||
// DBWritable and then overriding the string manually.
|
|
||||||
|
|
||||||
// Note that mysqldump also does *not* want a quoted table name.
|
|
||||||
DataDrivenDBInputFormat.setInput(job, DBWritable.class,
|
|
||||||
tableName, whereClause,
|
|
||||||
mgr.escapeColName(splitByCol), sqlColNames);
|
|
||||||
|
|
||||||
Configuration conf = job.getConfiguration();
|
|
||||||
conf.setInt(MySQLUtils.OUTPUT_FIELD_DELIM_KEY,
|
|
||||||
options.getOutputFieldDelim());
|
|
||||||
conf.setInt(MySQLUtils.OUTPUT_RECORD_DELIM_KEY,
|
|
||||||
options.getOutputRecordDelim());
|
|
||||||
conf.setInt(MySQLUtils.OUTPUT_ENCLOSED_BY_KEY,
|
|
||||||
options.getOutputEnclosedBy());
|
|
||||||
conf.setInt(MySQLUtils.OUTPUT_ESCAPED_BY_KEY,
|
|
||||||
options.getOutputEscapedBy());
|
|
||||||
conf.setBoolean(MySQLUtils.OUTPUT_ENCLOSE_REQUIRED_KEY,
|
|
||||||
options.isOutputEncloseRequired());
|
|
||||||
String [] extraArgs = options.getExtraArgs();
|
|
||||||
if (null != extraArgs) {
|
|
||||||
conf.setStrings(MySQLUtils.EXTRA_ARGS_KEY, extraArgs);
|
|
||||||
}
|
|
||||||
|
|
||||||
LOG.debug("Using InputFormat: " + inputFormatClass);
|
|
||||||
job.setInputFormatClass(getInputFormatClass());
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Set the mapper class implementation to use in the job,
|
|
||||||
* as well as any related configuration (e.g., map output types).
|
|
||||||
*/
|
|
||||||
protected void configureMapper(Job job, String tableName,
|
|
||||||
String tableClassName) throws ClassNotFoundException, IOException {
|
|
||||||
job.setMapperClass(getMapperClass());
|
|
||||||
job.setOutputKeyClass(String.class);
|
|
||||||
job.setOutputValueClass(NullWritable.class);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,81 +18,10 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.io.NullWritable;
|
|
||||||
import org.apache.hadoop.mapreduce.InputSplit;
|
|
||||||
import org.apache.hadoop.mapreduce.RecordReader;
|
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
||||||
import com.cloudera.sqoop.mapreduce.db.DataDrivenDBInputFormat;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* InputFormat designed to take data-driven splits and feed them to a mysqldump
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* invocation running in the mapper.
|
|
||||||
*
|
|
||||||
* The key emitted by this mapper is a WHERE clause to use in the command
|
|
||||||
* to mysqldump.
|
|
||||||
*/
|
*/
|
||||||
public class MySQLDumpInputFormat extends DataDrivenDBInputFormat {
|
public class MySQLDumpInputFormat
|
||||||
|
extends org.apache.sqoop.mapreduce.MySQLDumpInputFormat {
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
MySQLDumpInputFormat.class.getName());
|
|
||||||
|
|
||||||
/**
|
|
||||||
* A RecordReader that just takes the WHERE conditions from the DBInputSplit
|
|
||||||
* and relates them to the mapper as a single input record.
|
|
||||||
*/
|
|
||||||
public static class MySQLDumpRecordReader
|
|
||||||
extends RecordReader<String, NullWritable> {
|
|
||||||
|
|
||||||
private boolean delivered;
|
|
||||||
private String clause;
|
|
||||||
|
|
||||||
public MySQLDumpRecordReader(InputSplit split) {
|
|
||||||
initialize(split, null);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean nextKeyValue() {
|
|
||||||
boolean hasNext = !delivered;
|
|
||||||
delivered = true;
|
|
||||||
return hasNext;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String getCurrentKey() {
|
|
||||||
return clause;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public NullWritable getCurrentValue() {
|
|
||||||
return NullWritable.get();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void close() {
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public float getProgress() {
|
|
||||||
return delivered ? 1.0f : 0.0f;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void initialize(InputSplit split, TaskAttemptContext context) {
|
|
||||||
DataDrivenDBInputFormat.DataDrivenDBInputSplit dbSplit =
|
|
||||||
(DataDrivenDBInputFormat.DataDrivenDBInputSplit) split;
|
|
||||||
|
|
||||||
this.clause = "(" + dbSplit.getLowerClause() + ") AND ("
|
|
||||||
+ dbSplit.getUpperClause() + ")";
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public RecordReader<String, NullWritable> createRecordReader(InputSplit split,
|
|
||||||
TaskAttemptContext context) {
|
|
||||||
return new MySQLDumpRecordReader(split);
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,478 +18,40 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.BufferedReader;
|
|
||||||
import java.io.File;
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.io.InputStream;
|
|
||||||
import java.io.InputStreamReader;
|
|
||||||
import java.nio.CharBuffer;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.io.NullWritable;
|
|
||||||
import org.apache.hadoop.mapreduce.Mapper;
|
|
||||||
import com.cloudera.sqoop.lib.DelimiterSet;
|
|
||||||
import com.cloudera.sqoop.lib.FieldFormatter;
|
|
||||||
import com.cloudera.sqoop.lib.RecordParser;
|
|
||||||
import com.cloudera.sqoop.manager.MySQLUtils;
|
|
||||||
import com.cloudera.sqoop.util.AsyncSink;
|
|
||||||
import com.cloudera.sqoop.util.ErrorableAsyncSink;
|
|
||||||
import com.cloudera.sqoop.util.ErrorableThread;
|
|
||||||
import com.cloudera.sqoop.util.JdbcUrl;
|
|
||||||
import com.cloudera.sqoop.util.LoggingAsyncSink;
|
|
||||||
import com.cloudera.sqoop.util.PerfCounters;
|
import com.cloudera.sqoop.util.PerfCounters;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Mapper that opens up a pipe to mysqldump and pulls data directly.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
*/
|
*/
|
||||||
public class MySQLDumpMapper
|
public class MySQLDumpMapper
|
||||||
extends Mapper<String, NullWritable, String, NullWritable> {
|
extends org.apache.sqoop.mapreduce.MySQLDumpMapper {
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
MySQLDumpMapper.class.getName());
|
|
||||||
|
|
||||||
private Configuration conf;
|
|
||||||
|
|
||||||
// AsyncSinks used to import data from mysqldump directly into HDFS.
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Copies data directly from mysqldump into HDFS, after stripping some
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* header and footer characters that are attached to each line in mysqldump.
|
|
||||||
*/
|
*/
|
||||||
static class CopyingAsyncSink extends ErrorableAsyncSink {
|
public static class CopyingAsyncSink
|
||||||
private final MySQLDumpMapper.Context context;
|
extends org.apache.sqoop.mapreduce.MySQLDumpMapper.CopyingAsyncSink {
|
||||||
private final PerfCounters counters;
|
|
||||||
|
|
||||||
CopyingAsyncSink(final MySQLDumpMapper.Context context,
|
protected CopyingAsyncSink(final MySQLDumpMapper.Context context,
|
||||||
final PerfCounters ctrs) {
|
final PerfCounters ctrs) {
|
||||||
this.context = context;
|
super(context, ctrs);
|
||||||
this.counters = ctrs;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void processStream(InputStream is) {
|
|
||||||
child = new CopyingStreamThread(is, context, counters);
|
|
||||||
child.start();
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class CopyingStreamThread extends ErrorableThread {
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
CopyingStreamThread.class.getName());
|
|
||||||
|
|
||||||
private final MySQLDumpMapper.Context context;
|
|
||||||
private final InputStream stream;
|
|
||||||
private final PerfCounters counters;
|
|
||||||
|
|
||||||
CopyingStreamThread(final InputStream is,
|
|
||||||
final Context c, final PerfCounters ctrs) {
|
|
||||||
this.context = c;
|
|
||||||
this.stream = is;
|
|
||||||
this.counters = ctrs;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void run() {
|
|
||||||
BufferedReader r = null;
|
|
||||||
|
|
||||||
try {
|
|
||||||
r = new BufferedReader(new InputStreamReader(this.stream));
|
|
||||||
|
|
||||||
// Actually do the read/write transfer loop here.
|
|
||||||
int preambleLen = -1; // set to this for "undefined"
|
|
||||||
while (true) {
|
|
||||||
String inLine = r.readLine();
|
|
||||||
if (null == inLine) {
|
|
||||||
break; // EOF.
|
|
||||||
}
|
|
||||||
|
|
||||||
// this line is of the form "INSERT .. VALUES ( actual value text
|
|
||||||
// );" strip the leading preamble up to the '(' and the trailing
|
|
||||||
// ');'.
|
|
||||||
if (preambleLen == -1) {
|
|
||||||
// we haven't determined how long the preamble is. It's constant
|
|
||||||
// across all lines, so just figure this out once.
|
|
||||||
String recordStartMark = "VALUES (";
|
|
||||||
preambleLen = inLine.indexOf(recordStartMark)
|
|
||||||
+ recordStartMark.length();
|
|
||||||
}
|
|
||||||
|
|
||||||
// chop off the leading and trailing text as we write the
|
|
||||||
// output to HDFS.
|
|
||||||
int len = inLine.length() - 2 - preambleLen;
|
|
||||||
context.write(inLine.substring(preambleLen, inLine.length() - 2),
|
|
||||||
null);
|
|
||||||
context.write("\n", null);
|
|
||||||
counters.addBytes(1 + len);
|
|
||||||
}
|
|
||||||
} catch (IOException ioe) {
|
|
||||||
LOG.error("IOException reading from mysqldump: " + ioe.toString());
|
|
||||||
// flag this error so we get an error status back in the caller.
|
|
||||||
setError();
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
LOG.error("InterruptedException reading from mysqldump: "
|
|
||||||
+ ie.toString());
|
|
||||||
// flag this error so we get an error status back in the caller.
|
|
||||||
setError();
|
|
||||||
} finally {
|
|
||||||
if (null != r) {
|
|
||||||
try {
|
|
||||||
r.close();
|
|
||||||
} catch (IOException ioe) {
|
|
||||||
LOG.info("Error closing FIFO stream: " + ioe.toString());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The ReparsingAsyncSink will instantiate a RecordParser to read mysqldump's
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* output, and re-emit the text in the user's specified output format.
|
|
||||||
*/
|
*/
|
||||||
static class ReparsingAsyncSink extends ErrorableAsyncSink {
|
public static class ReparsingAsyncSink
|
||||||
private final MySQLDumpMapper.Context context;
|
extends org.apache.sqoop.mapreduce.MySQLDumpMapper.ReparsingAsyncSink {
|
||||||
private final Configuration conf;
|
|
||||||
private final PerfCounters counters;
|
|
||||||
|
|
||||||
ReparsingAsyncSink(final MySQLDumpMapper.Context c,
|
protected ReparsingAsyncSink(final MySQLDumpMapper.Context c,
|
||||||
final Configuration conf, final PerfCounters ctrs) {
|
final Configuration conf, final PerfCounters ctrs) {
|
||||||
this.context = c;
|
super(c, conf, ctrs);
|
||||||
this.conf = conf;
|
|
||||||
this.counters = ctrs;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void processStream(InputStream is) {
|
|
||||||
child = new ReparsingStreamThread(is, context, conf, counters);
|
|
||||||
child.start();
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class ReparsingStreamThread extends ErrorableThread {
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
ReparsingStreamThread.class.getName());
|
|
||||||
|
|
||||||
private final MySQLDumpMapper.Context context;
|
|
||||||
private final Configuration conf;
|
|
||||||
private final InputStream stream;
|
|
||||||
private final PerfCounters counters;
|
|
||||||
|
|
||||||
ReparsingStreamThread(final InputStream is,
|
|
||||||
final MySQLDumpMapper.Context c, Configuration conf,
|
|
||||||
final PerfCounters ctrs) {
|
|
||||||
this.context = c;
|
|
||||||
this.conf = conf;
|
|
||||||
this.stream = is;
|
|
||||||
this.counters = ctrs;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static final char MYSQL_FIELD_DELIM = ',';
|
|
||||||
private static final char MYSQL_RECORD_DELIM = '\n';
|
|
||||||
private static final char MYSQL_ENCLOSE_CHAR = '\'';
|
|
||||||
private static final char MYSQL_ESCAPE_CHAR = '\\';
|
|
||||||
private static final boolean MYSQL_ENCLOSE_REQUIRED = false;
|
|
||||||
|
|
||||||
private static final RecordParser MYSQLDUMP_PARSER;
|
|
||||||
|
|
||||||
static {
|
|
||||||
// build a record parser for mysqldump's format
|
|
||||||
MYSQLDUMP_PARSER = new RecordParser(DelimiterSet.MYSQL_DELIMITERS);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void run() {
|
|
||||||
BufferedReader r = null;
|
|
||||||
|
|
||||||
try {
|
|
||||||
r = new BufferedReader(new InputStreamReader(this.stream));
|
|
||||||
|
|
||||||
// Configure the output with the user's delimiters.
|
|
||||||
char outputFieldDelim = (char) conf.getInt(
|
|
||||||
MySQLUtils.OUTPUT_FIELD_DELIM_KEY,
|
|
||||||
DelimiterSet.NULL_CHAR);
|
|
||||||
String outputFieldDelimStr = "" + outputFieldDelim;
|
|
||||||
char outputRecordDelim = (char) conf.getInt(
|
|
||||||
MySQLUtils.OUTPUT_RECORD_DELIM_KEY,
|
|
||||||
DelimiterSet.NULL_CHAR);
|
|
||||||
String outputRecordDelimStr = "" + outputRecordDelim;
|
|
||||||
char outputEnclose = (char) conf.getInt(
|
|
||||||
MySQLUtils.OUTPUT_ENCLOSED_BY_KEY,
|
|
||||||
DelimiterSet.NULL_CHAR);
|
|
||||||
char outputEscape = (char) conf.getInt(
|
|
||||||
MySQLUtils.OUTPUT_ESCAPED_BY_KEY,
|
|
||||||
DelimiterSet.NULL_CHAR);
|
|
||||||
boolean outputEncloseRequired = conf.getBoolean(
|
|
||||||
MySQLUtils.OUTPUT_ENCLOSE_REQUIRED_KEY, false);
|
|
||||||
|
|
||||||
DelimiterSet delimiters = new DelimiterSet(
|
|
||||||
outputFieldDelim,
|
|
||||||
outputRecordDelim,
|
|
||||||
outputEnclose,
|
|
||||||
outputEscape,
|
|
||||||
outputEncloseRequired);
|
|
||||||
|
|
||||||
// Actually do the read/write transfer loop here.
|
|
||||||
int preambleLen = -1; // set to this for "undefined"
|
|
||||||
while (true) {
|
|
||||||
String inLine = r.readLine();
|
|
||||||
if (null == inLine) {
|
|
||||||
break; // EOF.
|
|
||||||
}
|
|
||||||
|
|
||||||
// this line is of the form "INSERT .. VALUES ( actual value text
|
|
||||||
// );" strip the leading preamble up to the '(' and the trailing
|
|
||||||
// ');'.
|
|
||||||
if (preambleLen == -1) {
|
|
||||||
// we haven't determined how long the preamble is. It's constant
|
|
||||||
// across all lines, so just figure this out once.
|
|
||||||
String recordStartMark = "VALUES (";
|
|
||||||
preambleLen = inLine.indexOf(recordStartMark)
|
|
||||||
+ recordStartMark.length();
|
|
||||||
}
|
|
||||||
|
|
||||||
// Wrap the input string in a char buffer that ignores the leading
|
|
||||||
// and trailing text.
|
|
||||||
CharBuffer charbuf = CharBuffer.wrap(inLine, preambleLen,
|
|
||||||
inLine.length() - 2);
|
|
||||||
|
|
||||||
// Pass this along to the parser
|
|
||||||
List<String> fields = null;
|
|
||||||
try {
|
|
||||||
fields = MYSQLDUMP_PARSER.parseRecord(charbuf);
|
|
||||||
} catch (RecordParser.ParseError pe) {
|
|
||||||
LOG.warn("ParseError reading from mysqldump: "
|
|
||||||
+ pe.toString() + "; record skipped");
|
|
||||||
continue; // Skip emitting this row.
|
|
||||||
}
|
|
||||||
|
|
||||||
// For all of the output fields, emit them using the delimiters
|
|
||||||
// the user chooses.
|
|
||||||
boolean first = true;
|
|
||||||
int recordLen = 1; // for the delimiter.
|
|
||||||
for (String field : fields) {
|
|
||||||
if (!first) {
|
|
||||||
context.write(outputFieldDelimStr, null);
|
|
||||||
} else {
|
|
||||||
first = false;
|
|
||||||
}
|
|
||||||
|
|
||||||
String fieldStr = FieldFormatter.escapeAndEnclose(field,
|
|
||||||
delimiters);
|
|
||||||
context.write(fieldStr, null);
|
|
||||||
recordLen += fieldStr.length();
|
|
||||||
}
|
|
||||||
|
|
||||||
context.write(outputRecordDelimStr, null);
|
|
||||||
counters.addBytes(recordLen);
|
|
||||||
}
|
|
||||||
} catch (IOException ioe) {
|
|
||||||
LOG.error("IOException reading from mysqldump: " + ioe.toString());
|
|
||||||
// flag this error so the parent can handle it appropriately.
|
|
||||||
setError();
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
LOG.error("InterruptedException reading from mysqldump: "
|
|
||||||
+ ie.toString());
|
|
||||||
// flag this error so we get an error status back in the caller.
|
|
||||||
setError();
|
|
||||||
} finally {
|
|
||||||
if (null != r) {
|
|
||||||
try {
|
|
||||||
r.close();
|
|
||||||
} catch (IOException ioe) {
|
|
||||||
LOG.info("Error closing FIFO stream: " + ioe.toString());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO(aaron): Refactor this method to be much shorter.
|
|
||||||
// CHECKSTYLE:OFF
|
|
||||||
/**
|
|
||||||
* Import the table into HDFS by using mysqldump to pull out the data from
|
|
||||||
* the database and upload the files directly to HDFS.
|
|
||||||
*/
|
|
||||||
public void map(String splitConditions, NullWritable val, Context context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
|
|
||||||
LOG.info("Beginning mysqldump fast path import");
|
|
||||||
|
|
||||||
ArrayList<String> args = new ArrayList<String>();
|
|
||||||
String tableName = conf.get(MySQLUtils.TABLE_NAME_KEY);
|
|
||||||
|
|
||||||
// We need to parse the connect string URI to determine the database name.
|
|
||||||
// Using java.net.URL directly on the connect string will fail because
|
|
||||||
// Java doesn't respect arbitrary JDBC-based schemes. So we chop off the
|
|
||||||
// scheme (everything before '://') and replace it with 'http', which we
|
|
||||||
// know will work.
|
|
||||||
String connectString = conf.get(MySQLUtils.CONNECT_STRING_KEY);
|
|
||||||
String databaseName = JdbcUrl.getDatabaseName(connectString);
|
|
||||||
String hostname = JdbcUrl.getHostName(connectString);
|
|
||||||
int port = JdbcUrl.getPort(connectString);
|
|
||||||
|
|
||||||
if (null == databaseName) {
|
|
||||||
throw new IOException("Could not determine database name");
|
|
||||||
}
|
|
||||||
|
|
||||||
LOG.info("Performing import of table " + tableName + " from database "
|
|
||||||
+ databaseName);
|
|
||||||
|
|
||||||
args.add(MySQLUtils.MYSQL_DUMP_CMD); // requires that this is on the path.
|
|
||||||
|
|
||||||
String password = conf.get(MySQLUtils.PASSWORD_KEY);
|
|
||||||
String passwordFile = null;
|
|
||||||
|
|
||||||
Process p = null;
|
|
||||||
AsyncSink sink = null;
|
|
||||||
AsyncSink errSink = null;
|
|
||||||
PerfCounters counters = new PerfCounters();
|
|
||||||
try {
|
|
||||||
// --defaults-file must be the first argument.
|
|
||||||
if (null != password && password.length() > 0) {
|
|
||||||
passwordFile = MySQLUtils.writePasswordFile(conf);
|
|
||||||
args.add("--defaults-file=" + passwordFile);
|
|
||||||
}
|
|
||||||
|
|
||||||
// Don't use the --where="<whereClause>" version because spaces in it can
|
|
||||||
// confuse Java, and adding in surrounding quotes confuses Java as well.
|
|
||||||
String whereClause = conf.get(MySQLUtils.WHERE_CLAUSE_KEY, "(1=1)")
|
|
||||||
+ " AND (" + splitConditions + ")";
|
|
||||||
args.add("-w");
|
|
||||||
args.add(whereClause);
|
|
||||||
|
|
||||||
args.add("--host=" + hostname);
|
|
||||||
if (-1 != port) {
|
|
||||||
args.add("--port=" + Integer.toString(port));
|
|
||||||
}
|
|
||||||
args.add("--skip-opt");
|
|
||||||
args.add("--compact");
|
|
||||||
args.add("--no-create-db");
|
|
||||||
args.add("--no-create-info");
|
|
||||||
args.add("--quick"); // no buffering
|
|
||||||
args.add("--single-transaction");
|
|
||||||
|
|
||||||
String username = conf.get(MySQLUtils.USERNAME_KEY);
|
|
||||||
if (null != username) {
|
|
||||||
args.add("--user=" + username);
|
|
||||||
}
|
|
||||||
|
|
||||||
// If the user supplied extra args, add them here.
|
|
||||||
String [] extra = conf.getStrings(MySQLUtils.EXTRA_ARGS_KEY);
|
|
||||||
if (null != extra) {
|
|
||||||
for (String arg : extra) {
|
|
||||||
args.add(arg);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
args.add(databaseName);
|
|
||||||
args.add(tableName);
|
|
||||||
|
|
||||||
// begin the import in an external process.
|
|
||||||
LOG.debug("Starting mysqldump with arguments:");
|
|
||||||
for (String arg : args) {
|
|
||||||
LOG.debug(" " + arg);
|
|
||||||
}
|
|
||||||
|
|
||||||
// Actually start the mysqldump.
|
|
||||||
p = Runtime.getRuntime().exec(args.toArray(new String[0]));
|
|
||||||
|
|
||||||
// read from the stdout pipe into the HDFS writer.
|
|
||||||
InputStream is = p.getInputStream();
|
|
||||||
|
|
||||||
if (MySQLUtils.outputDelimsAreMySQL(conf)) {
|
|
||||||
LOG.debug("Output delimiters conform to mysqldump; "
|
|
||||||
+ "using straight copy");
|
|
||||||
sink = new CopyingAsyncSink(context, counters);
|
|
||||||
} else {
|
|
||||||
LOG.debug("User-specified delimiters; using reparsing import");
|
|
||||||
LOG.info("Converting data to use specified delimiters.");
|
|
||||||
LOG.info("(For the fastest possible import, use");
|
|
||||||
LOG.info("--mysql-delimiters to specify the same field");
|
|
||||||
LOG.info("delimiters as are used by mysqldump.)");
|
|
||||||
sink = new ReparsingAsyncSink(context, conf, counters);
|
|
||||||
}
|
|
||||||
|
|
||||||
// Start an async thread to read and upload the whole stream.
|
|
||||||
counters.startClock();
|
|
||||||
sink.processStream(is);
|
|
||||||
|
|
||||||
// Start an async thread to send stderr to log4j.
|
|
||||||
errSink = new LoggingAsyncSink(LOG);
|
|
||||||
errSink.processStream(p.getErrorStream());
|
|
||||||
} finally {
|
|
||||||
|
|
||||||
// block until the process is done.
|
|
||||||
int result = 0;
|
|
||||||
if (null != p) {
|
|
||||||
while (true) {
|
|
||||||
try {
|
|
||||||
result = p.waitFor();
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
// interrupted; loop around.
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Remove the password file.
|
|
||||||
if (null != passwordFile) {
|
|
||||||
if (!new File(passwordFile).delete()) {
|
|
||||||
LOG.error("Could not remove mysql password file " + passwordFile);
|
|
||||||
LOG.error("You should remove this file to protect your credentials.");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// block until the stream sink is done too.
|
|
||||||
int streamResult = 0;
|
|
||||||
if (null != sink) {
|
|
||||||
while (true) {
|
|
||||||
try {
|
|
||||||
streamResult = sink.join();
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
// interrupted; loop around.
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Try to wait for stderr to finish, but regard any errors as advisory.
|
|
||||||
if (null != errSink) {
|
|
||||||
try {
|
|
||||||
if (0 != errSink.join()) {
|
|
||||||
LOG.info("Encountered exception reading stderr stream");
|
|
||||||
}
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
LOG.info("Thread interrupted waiting for stderr to complete: "
|
|
||||||
+ ie.toString());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
LOG.info("Transfer loop complete.");
|
|
||||||
|
|
||||||
if (0 != result) {
|
|
||||||
throw new IOException("mysqldump terminated with status "
|
|
||||||
+ Integer.toString(result));
|
|
||||||
}
|
|
||||||
|
|
||||||
if (0 != streamResult) {
|
|
||||||
throw new IOException("Encountered exception in stream sink");
|
|
||||||
}
|
|
||||||
|
|
||||||
counters.stopClock();
|
|
||||||
LOG.info("Transferred " + counters.toString());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// CHECKSTYLE:ON
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void setup(Context context) {
|
|
||||||
this.conf = context.getConfiguration();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,99 +18,16 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
|
||||||
import org.apache.hadoop.mapreduce.Mapper;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
|
|
||||||
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
|
||||||
import com.cloudera.sqoop.mapreduce.db.DataDrivenDBInputFormat;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.db.DBWritable;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.manager.ConnManager;
|
|
||||||
import com.cloudera.sqoop.manager.ExportJobContext;
|
import com.cloudera.sqoop.manager.ExportJobContext;
|
||||||
import com.cloudera.sqoop.manager.MySQLUtils;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Class that runs an export job using mysqlimport in the mapper.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
*/
|
*/
|
||||||
public class MySQLExportJob extends ExportJobBase {
|
public class MySQLExportJob
|
||||||
|
extends org.apache.sqoop.mapreduce.MySQLExportJob {
|
||||||
public static final Log LOG =
|
|
||||||
LogFactory.getLog(MySQLExportJob.class.getName());
|
|
||||||
|
|
||||||
public MySQLExportJob(final ExportJobContext context) {
|
public MySQLExportJob(final ExportJobContext context) {
|
||||||
super(context, null, null, NullOutputFormat.class);
|
super(context);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
/**
|
|
||||||
* Configure the inputformat to use for the job.
|
|
||||||
*/
|
|
||||||
protected void configureInputFormat(Job job, String tableName,
|
|
||||||
String tableClassName, String splitByCol)
|
|
||||||
throws ClassNotFoundException, IOException {
|
|
||||||
|
|
||||||
// Configure the delimiters, etc.
|
|
||||||
Configuration conf = job.getConfiguration();
|
|
||||||
conf.setInt(MySQLUtils.OUTPUT_FIELD_DELIM_KEY,
|
|
||||||
options.getOutputFieldDelim());
|
|
||||||
conf.setInt(MySQLUtils.OUTPUT_RECORD_DELIM_KEY,
|
|
||||||
options.getOutputRecordDelim());
|
|
||||||
conf.setInt(MySQLUtils.OUTPUT_ENCLOSED_BY_KEY,
|
|
||||||
options.getOutputEnclosedBy());
|
|
||||||
conf.setInt(MySQLUtils.OUTPUT_ESCAPED_BY_KEY,
|
|
||||||
options.getOutputEscapedBy());
|
|
||||||
conf.setBoolean(MySQLUtils.OUTPUT_ENCLOSE_REQUIRED_KEY,
|
|
||||||
options.isOutputEncloseRequired());
|
|
||||||
String [] extraArgs = options.getExtraArgs();
|
|
||||||
if (null != extraArgs) {
|
|
||||||
conf.setStrings(MySQLUtils.EXTRA_ARGS_KEY, extraArgs);
|
|
||||||
}
|
|
||||||
|
|
||||||
ConnManager mgr = context.getConnManager();
|
|
||||||
String username = options.getUsername();
|
|
||||||
if (null == username || username.length() == 0) {
|
|
||||||
DBConfiguration.configureDB(job.getConfiguration(),
|
|
||||||
mgr.getDriverClass(), options.getConnectString());
|
|
||||||
} else {
|
|
||||||
DBConfiguration.configureDB(job.getConfiguration(),
|
|
||||||
mgr.getDriverClass(), options.getConnectString(), username,
|
|
||||||
options.getPassword());
|
|
||||||
}
|
|
||||||
|
|
||||||
String [] colNames = options.getColumns();
|
|
||||||
if (null == colNames) {
|
|
||||||
colNames = mgr.getColumnNames(tableName);
|
|
||||||
}
|
|
||||||
|
|
||||||
String [] sqlColNames = null;
|
|
||||||
if (null != colNames) {
|
|
||||||
sqlColNames = new String[colNames.length];
|
|
||||||
for (int i = 0; i < colNames.length; i++) {
|
|
||||||
sqlColNames[i] = mgr.escapeColName(colNames[i]);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Note that mysqldump also does *not* want a quoted table name.
|
|
||||||
DataDrivenDBInputFormat.setInput(job, DBWritable.class,
|
|
||||||
tableName, null, null, sqlColNames);
|
|
||||||
|
|
||||||
// Configure the actual InputFormat to use.
|
|
||||||
super.configureInputFormat(job, tableName, tableClassName, splitByCol);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected Class<? extends Mapper> getMapperClass() {
|
|
||||||
if (inputIsSequenceFiles()) {
|
|
||||||
return MySQLRecordExportMapper.class;
|
|
||||||
} else {
|
|
||||||
return MySQLTextExportMapper.class;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,342 +18,16 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.BufferedOutputStream;
|
|
||||||
import java.io.File;
|
|
||||||
import java.io.FileOutputStream;
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.io.OutputStream;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.io.NullWritable;
|
|
||||||
import org.apache.hadoop.mapreduce.Mapper;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.io.NamedFifo;
|
|
||||||
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
|
||||||
import com.cloudera.sqoop.manager.MySQLUtils;
|
|
||||||
import com.cloudera.sqoop.util.AsyncSink;
|
|
||||||
import com.cloudera.sqoop.util.JdbcUrl;
|
|
||||||
import com.cloudera.sqoop.util.LoggingAsyncSink;
|
|
||||||
import com.cloudera.sqoop.util.NullAsyncSink;
|
|
||||||
import com.cloudera.sqoop.util.TaskId;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Mapper that starts a 'mysqlimport' process and uses that to export rows from
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* HDFS to a MySQL database at high speed.
|
|
||||||
*
|
|
||||||
* map() methods are actually provided by subclasses that read from
|
|
||||||
* SequenceFiles (containing existing SqoopRecords) or text files
|
|
||||||
* (containing delimited lines) and deliver these results to the fifo
|
|
||||||
* used to interface with mysqlimport.
|
|
||||||
*/
|
*/
|
||||||
public class MySQLExportMapper<KEYIN, VALIN>
|
public class MySQLExportMapper<KEYIN, VALIN>
|
||||||
extends Mapper<KEYIN, VALIN, NullWritable, NullWritable> {
|
extends org.apache.sqoop.mapreduce.MySQLExportMapper<KEYIN, VALIN> {
|
||||||
|
|
||||||
public static final Log LOG = LogFactory.getLog(
|
|
||||||
MySQLExportMapper.class.getName());
|
|
||||||
|
|
||||||
/** Configuration key that specifies the number of bytes before which it
|
|
||||||
* commits the current export transaction and opens a new one.
|
|
||||||
* Default is 32 MB; setting this to 0 will use no checkpoints.
|
|
||||||
*/
|
|
||||||
public static final String MYSQL_CHECKPOINT_BYTES_KEY =
|
public static final String MYSQL_CHECKPOINT_BYTES_KEY =
|
||||||
"sqoop.mysql.export.checkpoint.bytes";
|
org.apache.sqoop.mapreduce.MySQLExportMapper.MYSQL_CHECKPOINT_BYTES_KEY;
|
||||||
|
|
||||||
public static final long DEFAULT_CHECKPOINT_BYTES = 32 * 1024 * 1024;
|
public static final long DEFAULT_CHECKPOINT_BYTES =
|
||||||
|
org.apache.sqoop.mapreduce.MySQLExportMapper.DEFAULT_CHECKPOINT_BYTES;
|
||||||
|
|
||||||
// Configured value for MSYQL_CHECKPOINT_BYTES_KEY.
|
|
||||||
protected long checkpointDistInBytes;
|
|
||||||
|
|
||||||
protected Configuration conf;
|
|
||||||
|
|
||||||
/** The FIFO being used to communicate with mysqlimport. */
|
|
||||||
protected File fifoFile;
|
|
||||||
|
|
||||||
/** The process object representing the active connection to mysqlimport. */
|
|
||||||
protected Process mysqlImportProcess;
|
|
||||||
|
|
||||||
/** The stream to write to stdin for mysqlimport. */
|
|
||||||
protected OutputStream importStream;
|
|
||||||
|
|
||||||
// Handlers for stdout and stderr from mysqlimport.
|
|
||||||
protected AsyncSink outSink;
|
|
||||||
protected AsyncSink errSink;
|
|
||||||
|
|
||||||
/** File object where we wrote the user's password to pass to mysqlimport. */
|
|
||||||
protected File passwordFile;
|
|
||||||
|
|
||||||
/** Character set used to write to mysqlimport. */
|
|
||||||
protected String mysqlCharSet;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Tally of bytes written to current mysqlimport instance.
|
|
||||||
* We commit an interim tx and open a new mysqlimport after this
|
|
||||||
* gets too big. */
|
|
||||||
private long bytesWritten;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Create a named FIFO, and start mysqlimport connected to that FIFO.
|
|
||||||
* A File object representing the FIFO is in 'fifoFile'.
|
|
||||||
*/
|
|
||||||
private void initMySQLImportProcess() throws IOException {
|
|
||||||
File taskAttemptDir = TaskId.getLocalWorkPath(conf);
|
|
||||||
|
|
||||||
this.fifoFile = new File(taskAttemptDir,
|
|
||||||
conf.get(MySQLUtils.TABLE_NAME_KEY, "UNKNOWN_TABLE") + ".txt");
|
|
||||||
String filename = fifoFile.toString();
|
|
||||||
|
|
||||||
// Create the FIFO itself.
|
|
||||||
try {
|
|
||||||
new NamedFifo(this.fifoFile).create();
|
|
||||||
} catch (IOException ioe) {
|
|
||||||
// Command failed.
|
|
||||||
LOG.error("Could not mknod " + filename);
|
|
||||||
this.fifoFile = null;
|
|
||||||
throw new IOException(
|
|
||||||
"Could not create FIFO to interface with mysqlimport", ioe);
|
|
||||||
}
|
|
||||||
|
|
||||||
// Now open the connection to mysqlimport.
|
|
||||||
ArrayList<String> args = new ArrayList<String>();
|
|
||||||
|
|
||||||
String connectString = conf.get(MySQLUtils.CONNECT_STRING_KEY);
|
|
||||||
String databaseName = JdbcUrl.getDatabaseName(connectString);
|
|
||||||
String hostname = JdbcUrl.getHostName(connectString);
|
|
||||||
int port = JdbcUrl.getPort(connectString);
|
|
||||||
|
|
||||||
if (null == databaseName) {
|
|
||||||
throw new IOException("Could not determine database name");
|
|
||||||
}
|
|
||||||
|
|
||||||
args.add(MySQLUtils.MYSQL_IMPORT_CMD); // needs to be on the path.
|
|
||||||
String password = conf.get(MySQLUtils.PASSWORD_KEY);
|
|
||||||
|
|
||||||
if (null != password && password.length() > 0) {
|
|
||||||
passwordFile = new File(MySQLUtils.writePasswordFile(conf));
|
|
||||||
args.add("--defaults-file=" + passwordFile);
|
|
||||||
}
|
|
||||||
|
|
||||||
String username = conf.get(MySQLUtils.USERNAME_KEY);
|
|
||||||
if (null != username) {
|
|
||||||
args.add("--user=" + username);
|
|
||||||
}
|
|
||||||
|
|
||||||
args.add("--host=" + hostname);
|
|
||||||
if (-1 != port) {
|
|
||||||
args.add("--port=" + Integer.toString(port));
|
|
||||||
}
|
|
||||||
|
|
||||||
args.add("--compress");
|
|
||||||
args.add("--local");
|
|
||||||
args.add("--silent");
|
|
||||||
|
|
||||||
// Specify the subset of columns we're importing.
|
|
||||||
DBConfiguration dbConf = new DBConfiguration(conf);
|
|
||||||
String [] cols = dbConf.getInputFieldNames();
|
|
||||||
if (null != cols) {
|
|
||||||
StringBuilder sb = new StringBuilder();
|
|
||||||
boolean first = true;
|
|
||||||
for (String col : cols) {
|
|
||||||
if (!first) {
|
|
||||||
sb.append(",");
|
|
||||||
}
|
|
||||||
sb.append(col);
|
|
||||||
first = false;
|
|
||||||
}
|
|
||||||
|
|
||||||
args.add("--columns=" + sb.toString());
|
|
||||||
}
|
|
||||||
|
|
||||||
// Specify the delimiters to use.
|
|
||||||
int outputFieldDelim = conf.getInt(MySQLUtils.OUTPUT_FIELD_DELIM_KEY,
|
|
||||||
(int) ',');
|
|
||||||
int outputRecordDelim = conf.getInt(MySQLUtils.OUTPUT_RECORD_DELIM_KEY,
|
|
||||||
(int) '\n');
|
|
||||||
int enclosedBy = conf.getInt(MySQLUtils.OUTPUT_ENCLOSED_BY_KEY, 0);
|
|
||||||
int escapedBy = conf.getInt(MySQLUtils.OUTPUT_ESCAPED_BY_KEY, 0);
|
|
||||||
boolean encloseRequired = conf.getBoolean(
|
|
||||||
MySQLUtils.OUTPUT_ENCLOSE_REQUIRED_KEY, false);
|
|
||||||
|
|
||||||
args.add("--fields-terminated-by=0x"
|
|
||||||
+ Integer.toString(outputFieldDelim, 16));
|
|
||||||
args.add("--lines-terminated-by=0x"
|
|
||||||
+ Integer.toString(outputRecordDelim, 16));
|
|
||||||
if (0 != enclosedBy) {
|
|
||||||
if (encloseRequired) {
|
|
||||||
args.add("--fields-enclosed-by=0x" + Integer.toString(enclosedBy, 16));
|
|
||||||
} else {
|
|
||||||
args.add("--fields-optionally-enclosed-by=0x"
|
|
||||||
+ Integer.toString(enclosedBy, 16));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (0 != escapedBy) {
|
|
||||||
args.add("--escaped-by=0x" + Integer.toString(escapedBy, 16));
|
|
||||||
}
|
|
||||||
|
|
||||||
// These two arguments are positional and must be last.
|
|
||||||
args.add(databaseName);
|
|
||||||
args.add(filename);
|
|
||||||
|
|
||||||
// Begin the export in an external process.
|
|
||||||
LOG.debug("Starting mysqlimport with arguments:");
|
|
||||||
for (String arg : args) {
|
|
||||||
LOG.debug(" " + arg);
|
|
||||||
}
|
|
||||||
|
|
||||||
// Actually start mysqlimport.
|
|
||||||
mysqlImportProcess = Runtime.getRuntime().exec(args.toArray(new String[0]));
|
|
||||||
|
|
||||||
// Log everything it writes to stderr.
|
|
||||||
// Ignore anything on stdout.
|
|
||||||
this.outSink = new NullAsyncSink();
|
|
||||||
this.outSink.processStream(mysqlImportProcess.getInputStream());
|
|
||||||
|
|
||||||
this.errSink = new LoggingAsyncSink(LOG);
|
|
||||||
this.errSink.processStream(mysqlImportProcess.getErrorStream());
|
|
||||||
|
|
||||||
// Open the named FIFO after starting mysqlimport.
|
|
||||||
this.importStream = new BufferedOutputStream(
|
|
||||||
new FileOutputStream(fifoFile));
|
|
||||||
|
|
||||||
// At this point, mysqlimport is running and hooked up to our FIFO.
|
|
||||||
// The mapper just needs to populate it with data.
|
|
||||||
|
|
||||||
this.bytesWritten = 0;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void run(Context context) throws IOException, InterruptedException {
|
|
||||||
this.conf = context.getConfiguration();
|
|
||||||
setup(context);
|
|
||||||
initMySQLImportProcess();
|
|
||||||
try {
|
|
||||||
while (context.nextKeyValue()) {
|
|
||||||
map(context.getCurrentKey(), context.getCurrentValue(), context);
|
|
||||||
}
|
|
||||||
cleanup(context);
|
|
||||||
} finally {
|
|
||||||
// Shut down the mysqlimport process.
|
|
||||||
closeExportHandles();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void closeExportHandles() throws IOException, InterruptedException {
|
|
||||||
int ret = 0;
|
|
||||||
if (null != this.importStream) {
|
|
||||||
// Close the stream that writes to mysqlimport's stdin first.
|
|
||||||
LOG.debug("Closing import stream");
|
|
||||||
this.importStream.close();
|
|
||||||
this.importStream = null;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (null != this.mysqlImportProcess) {
|
|
||||||
// We started mysqlimport; wait for it to finish.
|
|
||||||
LOG.info("Waiting for mysqlimport to complete");
|
|
||||||
ret = this.mysqlImportProcess.waitFor();
|
|
||||||
LOG.info("mysqlimport closed connection");
|
|
||||||
this.mysqlImportProcess = null;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (null != this.passwordFile && this.passwordFile.exists()) {
|
|
||||||
if (!this.passwordFile.delete()) {
|
|
||||||
LOG.error("Could not remove mysql password file " + passwordFile);
|
|
||||||
LOG.error("You should remove this file to protect your credentials.");
|
|
||||||
}
|
|
||||||
|
|
||||||
this.passwordFile = null;
|
|
||||||
}
|
|
||||||
|
|
||||||
// Finish processing any output from mysqlimport.
|
|
||||||
// This is informational only, so we don't care about return codes.
|
|
||||||
if (null != outSink) {
|
|
||||||
LOG.debug("Waiting for any additional stdout from mysqlimport");
|
|
||||||
outSink.join();
|
|
||||||
outSink = null;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (null != errSink) {
|
|
||||||
LOG.debug("Waiting for any additional stderr from mysqlimport");
|
|
||||||
errSink.join();
|
|
||||||
errSink = null;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (this.fifoFile != null && this.fifoFile.exists()) {
|
|
||||||
// Clean up the resources we created.
|
|
||||||
LOG.debug("Removing fifo file");
|
|
||||||
if (!this.fifoFile.delete()) {
|
|
||||||
LOG.error("Could not clean up named FIFO after completing mapper");
|
|
||||||
}
|
|
||||||
|
|
||||||
// We put the FIFO file in a one-off subdir. Remove that.
|
|
||||||
File fifoParentDir = this.fifoFile.getParentFile();
|
|
||||||
LOG.debug("Removing task attempt tmpdir");
|
|
||||||
if (!fifoParentDir.delete()) {
|
|
||||||
LOG.error("Could not clean up task dir after completing mapper");
|
|
||||||
}
|
|
||||||
|
|
||||||
this.fifoFile = null;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (0 != ret) {
|
|
||||||
// Don't mark the task as successful if mysqlimport returns an error.
|
|
||||||
throw new IOException("mysqlimport terminated with error code " + ret);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void setup(Context context) {
|
|
||||||
this.conf = context.getConfiguration();
|
|
||||||
|
|
||||||
// TODO: Support additional encodings.
|
|
||||||
this.mysqlCharSet = MySQLUtils.MYSQL_DEFAULT_CHARSET;
|
|
||||||
|
|
||||||
this.checkpointDistInBytes = conf.getLong(
|
|
||||||
MYSQL_CHECKPOINT_BYTES_KEY, DEFAULT_CHECKPOINT_BYTES);
|
|
||||||
if (this.checkpointDistInBytes < 0) {
|
|
||||||
LOG.warn("Invalid value for " + MYSQL_CHECKPOINT_BYTES_KEY);
|
|
||||||
this.checkpointDistInBytes = DEFAULT_CHECKPOINT_BYTES;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Takes a delimited text record (e.g., the output of a 'Text' object),
|
|
||||||
* re-encodes it for consumption by mysqlimport, and writes it to the pipe.
|
|
||||||
* @param record A delimited text representation of one record.
|
|
||||||
* @param terminator an optional string that contains delimiters that
|
|
||||||
* terminate the record (if not included in 'record' itself).
|
|
||||||
*/
|
|
||||||
protected void writeRecord(String record, String terminator)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
|
|
||||||
// We've already set up mysqlimport to accept the same delimiters,
|
|
||||||
// so we don't need to convert those. But our input text is UTF8
|
|
||||||
// encoded; mysql allows configurable encoding, but defaults to
|
|
||||||
// latin-1 (ISO8859_1). We'll convert to latin-1 for now.
|
|
||||||
// TODO: Support user-configurable encodings.
|
|
||||||
|
|
||||||
byte [] mysqlBytes = record.getBytes(this.mysqlCharSet);
|
|
||||||
this.importStream.write(mysqlBytes, 0, mysqlBytes.length);
|
|
||||||
this.bytesWritten += mysqlBytes.length;
|
|
||||||
|
|
||||||
if (null != terminator) {
|
|
||||||
byte [] termBytes = terminator.getBytes(this.mysqlCharSet);
|
|
||||||
this.importStream.write(termBytes, 0, termBytes.length);
|
|
||||||
this.bytesWritten += termBytes.length;
|
|
||||||
}
|
|
||||||
|
|
||||||
// If bytesWritten is too big, then we should start a new tx by closing
|
|
||||||
// mysqlimport and opening a new instance of the process.
|
|
||||||
if (this.checkpointDistInBytes != 0
|
|
||||||
&& this.bytesWritten > this.checkpointDistInBytes) {
|
|
||||||
LOG.info("Checkpointing current export.");
|
|
||||||
closeExportHandles();
|
|
||||||
initMySQLImportProcess();
|
|
||||||
this.bytesWritten = 0;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,32 +18,9 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.io.LongWritable;
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* mysqlimport-based exporter which accepts SqoopRecords (e.g., from
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* SequenceFiles) to emit to the database.
|
|
||||||
*/
|
*/
|
||||||
public class MySQLRecordExportMapper
|
public class MySQLRecordExportMapper
|
||||||
extends MySQLExportMapper<LongWritable, SqoopRecord> {
|
extends org.apache.sqoop.mapreduce.MySQLRecordExportMapper {
|
||||||
|
|
||||||
/**
|
|
||||||
* Export the table to MySQL by using mysqlimport to write the data to the
|
|
||||||
* database.
|
|
||||||
*
|
|
||||||
* Expects one SqoopRecord as the value. Ignores the key.
|
|
||||||
*/
|
|
||||||
@Override
|
|
||||||
public void map(LongWritable key, SqoopRecord val, Context context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
|
|
||||||
writeRecord(val.toString(), null);
|
|
||||||
|
|
||||||
// We don't emit anything to the OutputCollector because we wrote
|
|
||||||
// straight to mysql. Send a progress indicator to prevent a timeout.
|
|
||||||
context.progress();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,46 +18,9 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.io.LongWritable;
|
|
||||||
import org.apache.hadoop.io.Text;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.manager.MySQLUtils;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* mysqlimport-based exporter which accepts lines of text from files
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* in HDFS to emit to the database.
|
|
||||||
*/
|
*/
|
||||||
public class MySQLTextExportMapper
|
public class MySQLTextExportMapper
|
||||||
extends MySQLExportMapper<LongWritable, Text> {
|
extends org.apache.sqoop.mapreduce.MySQLTextExportMapper {
|
||||||
|
|
||||||
// End-of-record delimiter.
|
|
||||||
private String recordEndStr;
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void setup(Context context) {
|
|
||||||
super.setup(context);
|
|
||||||
|
|
||||||
char recordDelim = (char) conf.getInt(MySQLUtils.OUTPUT_RECORD_DELIM_KEY,
|
|
||||||
(int) '\n');
|
|
||||||
this.recordEndStr = "" + recordDelim;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Export the table to MySQL by using mysqlimport to write the data to the
|
|
||||||
* database.
|
|
||||||
*
|
|
||||||
* Expects one delimited text record as the 'val'; ignores the key.
|
|
||||||
*/
|
|
||||||
@Override
|
|
||||||
public void map(LongWritable key, Text val, Context context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
|
|
||||||
writeRecord(val.toString(), this.recordEndStr);
|
|
||||||
|
|
||||||
// We don't emit anything to the OutputCollector because we wrote
|
|
||||||
// straight to mysql. Send a progress indicator to prevent a timeout.
|
|
||||||
context.progress();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,27 +18,10 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import org.apache.hadoop.mapreduce.JobContext;
|
|
||||||
import org.apache.hadoop.mapreduce.OutputCommitter;
|
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* OutputCommitter instance that does nothing.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
*/
|
*/
|
||||||
public class NullOutputCommitter extends OutputCommitter {
|
public class NullOutputCommitter
|
||||||
public void abortTask(TaskAttemptContext taskContext) { }
|
extends org.apache.sqoop.mapreduce.NullOutputCommitter {
|
||||||
|
|
||||||
public void cleanupJob(JobContext jobContext) { }
|
|
||||||
|
|
||||||
public void commitTask(TaskAttemptContext taskContext) { }
|
|
||||||
|
|
||||||
public boolean needsTaskCommit(TaskAttemptContext taskContext) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setupJob(JobContext jobContext) { }
|
|
||||||
|
|
||||||
public void setupTask(TaskAttemptContext taskContext) { }
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,94 +18,11 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.sql.SQLException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.mapreduce.RecordWriter;
|
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Oracle-specific SQL formatting overrides default ExportOutputFormat's.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
*/
|
*/
|
||||||
public class OracleExportOutputFormat<K extends SqoopRecord, V>
|
public class OracleExportOutputFormat<K extends SqoopRecord, V>
|
||||||
extends ExportOutputFormat<K, V> {
|
extends org.apache.sqoop.mapreduce.OracleExportOutputFormat<K, V> {
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
|
||||||
throws IOException {
|
|
||||||
try {
|
|
||||||
return new OracleExportRecordWriter(context);
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new IOException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* RecordWriter to write the output to a row in a database table.
|
|
||||||
* The actual database updates are executed in a second thread.
|
|
||||||
*/
|
|
||||||
public class OracleExportRecordWriter extends ExportRecordWriter {
|
|
||||||
|
|
||||||
public OracleExportRecordWriter(TaskAttemptContext context)
|
|
||||||
throws ClassNotFoundException, SQLException {
|
|
||||||
super(context);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/**
|
|
||||||
* @return an INSERT statement suitable for inserting 'numRows' rows.
|
|
||||||
*/
|
|
||||||
protected String getInsertStatement(int numRows) {
|
|
||||||
StringBuilder sb = new StringBuilder();
|
|
||||||
|
|
||||||
sb.append("INSERT INTO " + getTableName() + " ");
|
|
||||||
|
|
||||||
int numSlots;
|
|
||||||
String [] colNames = getColumnNames();
|
|
||||||
if (colNames != null) {
|
|
||||||
numSlots = colNames.length;
|
|
||||||
|
|
||||||
sb.append("(");
|
|
||||||
boolean first = true;
|
|
||||||
for (String col : colNames) {
|
|
||||||
if (!first) {
|
|
||||||
sb.append(", ");
|
|
||||||
}
|
|
||||||
|
|
||||||
sb.append(col);
|
|
||||||
first = false;
|
|
||||||
}
|
|
||||||
|
|
||||||
sb.append(") ");
|
|
||||||
} else {
|
|
||||||
numSlots = getColumnCount(); // set if columnNames is null.
|
|
||||||
}
|
|
||||||
|
|
||||||
// generates the (?, ?, ?...) used for each row.
|
|
||||||
StringBuilder sbRow = new StringBuilder();
|
|
||||||
sbRow.append("SELECT ");
|
|
||||||
for (int i = 0; i < numSlots; i++) {
|
|
||||||
if (i != 0) {
|
|
||||||
sbRow.append(", ");
|
|
||||||
}
|
|
||||||
|
|
||||||
sbRow.append("?");
|
|
||||||
}
|
|
||||||
sbRow.append(" FROM DUAL ");
|
|
||||||
|
|
||||||
// Now append that numRows times.
|
|
||||||
for (int i = 0; i < numRows; i++) {
|
|
||||||
if (i != 0) {
|
|
||||||
sb.append("UNION ALL ");
|
|
||||||
}
|
|
||||||
|
|
||||||
sb.append(sbRow);
|
|
||||||
}
|
|
||||||
|
|
||||||
return sb.toString();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,115 +18,11 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.sql.SQLException;
|
|
||||||
import java.util.LinkedHashSet;
|
|
||||||
import java.util.Set;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.mapreduce.RecordWriter;
|
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Update an existing table with new value if the table already
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* contains the row, or insert the data into the table if the table
|
|
||||||
* does not contain the row yet.
|
|
||||||
*/
|
*/
|
||||||
public class OracleUpsertOutputFormat<K extends SqoopRecord, V>
|
public class OracleUpsertOutputFormat<K extends SqoopRecord, V>
|
||||||
extends UpdateOutputFormat<K, V> {
|
extends org.apache.sqoop.mapreduce.OracleUpsertOutputFormat<K, V> {
|
||||||
|
|
||||||
private static final Log LOG =
|
|
||||||
LogFactory.getLog(OracleUpsertOutputFormat.class);
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
|
||||||
throws IOException {
|
|
||||||
try {
|
|
||||||
return new OracleUpsertRecordWriter(context);
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new IOException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* RecordWriter to write the output to UPDATE/INSERT statements.
|
|
||||||
*/
|
|
||||||
public class OracleUpsertRecordWriter extends UpdateRecordWriter {
|
|
||||||
|
|
||||||
public OracleUpsertRecordWriter(TaskAttemptContext context)
|
|
||||||
throws ClassNotFoundException, SQLException {
|
|
||||||
super(context);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return an UPDATE/INSERT statement that modifies/inserts a row
|
|
||||||
* depending on whether the row already exist in the table or not.
|
|
||||||
*/
|
|
||||||
protected String getUpdateStatement() {
|
|
||||||
boolean first;
|
|
||||||
|
|
||||||
// lookup table for update columns
|
|
||||||
Set<String> updateKeyLookup = new LinkedHashSet<String>();
|
|
||||||
for (String updateKey : updateCols) {
|
|
||||||
updateKeyLookup.add(updateKey);
|
|
||||||
}
|
|
||||||
|
|
||||||
StringBuilder sb = new StringBuilder();
|
|
||||||
sb.append("MERGE INTO ");
|
|
||||||
sb.append(tableName);
|
|
||||||
sb.append(" USING dual ON ( ");
|
|
||||||
first = true;
|
|
||||||
for (int i = 0; i < updateCols.length; i++) {
|
|
||||||
if (first) {
|
|
||||||
first = false;
|
|
||||||
} else {
|
|
||||||
sb.append(" AND ");
|
|
||||||
}
|
|
||||||
sb.append(updateCols[i]).append(" = ?");
|
|
||||||
}
|
|
||||||
sb.append(" )");
|
|
||||||
|
|
||||||
sb.append(" WHEN MATCHED THEN UPDATE SET ");
|
|
||||||
first = true;
|
|
||||||
for (String col : columnNames) {
|
|
||||||
if (!updateKeyLookup.contains(col)) {
|
|
||||||
if (first) {
|
|
||||||
first = false;
|
|
||||||
} else {
|
|
||||||
sb.append(", ");
|
|
||||||
}
|
|
||||||
sb.append(col);
|
|
||||||
sb.append(" = ?");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
sb.append(" WHEN NOT MATCHED THEN INSERT ( ");
|
|
||||||
first = true;
|
|
||||||
for (String col : columnNames) {
|
|
||||||
if (first) {
|
|
||||||
first = false;
|
|
||||||
} else {
|
|
||||||
sb.append(", ");
|
|
||||||
}
|
|
||||||
sb.append(col);
|
|
||||||
}
|
|
||||||
sb.append(" ) VALUES ( ");
|
|
||||||
first = true;
|
|
||||||
for (String col : columnNames) {
|
|
||||||
if (first) {
|
|
||||||
first = false;
|
|
||||||
} else {
|
|
||||||
sb.append(", ");
|
|
||||||
}
|
|
||||||
sb.append("?");
|
|
||||||
}
|
|
||||||
sb.append(" )");
|
|
||||||
|
|
||||||
return sb.toString();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -21,85 +19,25 @@
|
|||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.DataOutputStream;
|
import java.io.DataOutputStream;
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
/**
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
||||||
import org.apache.hadoop.io.Text;
|
|
||||||
import org.apache.hadoop.io.compress.CompressionCodec;
|
|
||||||
import org.apache.hadoop.io.compress.GzipCodec;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
|
||||||
import org.apache.hadoop.mapreduce.RecordWriter;
|
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
||||||
import org.apache.hadoop.util.*;
|
|
||||||
|
|
||||||
/** An {@link OutputFormat} that writes plain text files.
|
|
||||||
* Only writes the key. Does not write any delimiter/newline after the key.
|
|
||||||
*/
|
*/
|
||||||
public class RawKeyTextOutputFormat<K, V> extends FileOutputFormat<K, V> {
|
public class RawKeyTextOutputFormat<K, V>
|
||||||
|
extends org.apache.sqoop.mapreduce.RawKeyTextOutputFormat<K, V> {
|
||||||
|
|
||||||
protected static class RawKeyRecordWriter<K, V> extends RecordWriter<K, V> {
|
/**
|
||||||
private static final String UTF8 = "UTF-8";
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
|
*/
|
||||||
protected DataOutputStream out;
|
public static class RawKeyRecordWriter<K, V>
|
||||||
|
extends org.apache.sqoop.mapreduce.RawKeyTextOutputFormat.
|
||||||
|
RawKeyRecordWriter<K, V> {
|
||||||
|
|
||||||
public RawKeyRecordWriter(DataOutputStream out) {
|
public RawKeyRecordWriter(DataOutputStream out) {
|
||||||
this.out = out;
|
super(out);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Write the object to the byte stream, handling Text as a special
|
|
||||||
* case.
|
|
||||||
* @param o the object to print
|
|
||||||
* @throws IOException if the write throws, we pass it on
|
|
||||||
*/
|
|
||||||
private void writeObject(Object o) throws IOException {
|
|
||||||
if (o instanceof Text) {
|
|
||||||
Text to = (Text) o;
|
|
||||||
out.write(to.getBytes(), 0, to.getLength());
|
|
||||||
} else {
|
|
||||||
out.write(o.toString().getBytes(UTF8));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public synchronized void write(K key, V value) throws IOException {
|
|
||||||
writeObject(key);
|
|
||||||
}
|
|
||||||
|
|
||||||
public synchronized void close(TaskAttemptContext context)
|
|
||||||
throws IOException {
|
|
||||||
out.close();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
|
||||||
throws IOException {
|
|
||||||
boolean isCompressed = getCompressOutput(context);
|
|
||||||
Configuration conf = context.getConfiguration();
|
|
||||||
String ext = "";
|
|
||||||
CompressionCodec codec = null;
|
|
||||||
|
|
||||||
if (isCompressed) {
|
|
||||||
// create the named codec
|
|
||||||
Class<? extends CompressionCodec> codecClass =
|
|
||||||
getOutputCompressorClass(context, GzipCodec.class);
|
|
||||||
codec = ReflectionUtils.newInstance(codecClass, conf);
|
|
||||||
|
|
||||||
ext = codec.getDefaultExtension();
|
|
||||||
}
|
|
||||||
|
|
||||||
Path file = getDefaultWorkFile(context, ext);
|
|
||||||
FileSystem fs = file.getFileSystem(conf);
|
|
||||||
FSDataOutputStream fileOut = fs.create(file, false);
|
|
||||||
DataOutputStream ostream = fileOut;
|
|
||||||
|
|
||||||
if (isCompressed) {
|
|
||||||
ostream = new DataOutputStream(codec.createOutputStream(fileOut));
|
|
||||||
}
|
|
||||||
|
|
||||||
return new RawKeyRecordWriter<K, V>(ostream);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,94 +18,11 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.sql.SQLException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.mapreduce.RecordWriter;
|
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* SQLServer-specific SQL formatting overrides default ExportOutputFormat's.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
*/
|
*/
|
||||||
public class SQLServerExportOutputFormat<K extends SqoopRecord, V>
|
public class SQLServerExportOutputFormat<K extends SqoopRecord, V>
|
||||||
extends ExportOutputFormat<K, V> {
|
extends org.apache.sqoop.mapreduce.SQLServerExportOutputFormat<K, V> {
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
|
||||||
throws IOException {
|
|
||||||
try {
|
|
||||||
return new SQLServerExportRecordWriter(context);
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new IOException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* RecordWriter to write the output to a row in a database table.
|
|
||||||
* The actual database updates are executed in a second thread.
|
|
||||||
*/
|
|
||||||
public class SQLServerExportRecordWriter extends ExportRecordWriter {
|
|
||||||
|
|
||||||
public SQLServerExportRecordWriter(TaskAttemptContext context)
|
|
||||||
throws ClassNotFoundException, SQLException {
|
|
||||||
super(context);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/**
|
|
||||||
* @return an INSERT statement suitable for inserting 'numRows' rows.
|
|
||||||
*/
|
|
||||||
protected String getInsertStatement(int numRows) {
|
|
||||||
StringBuilder sb = new StringBuilder();
|
|
||||||
|
|
||||||
sb.append("INSERT INTO " + getTableName() + " ");
|
|
||||||
|
|
||||||
int numSlots;
|
|
||||||
String [] colNames = getColumnNames();
|
|
||||||
if (colNames != null) {
|
|
||||||
numSlots = colNames.length;
|
|
||||||
|
|
||||||
sb.append("(");
|
|
||||||
boolean first = true;
|
|
||||||
for (String col : colNames) {
|
|
||||||
if (!first) {
|
|
||||||
sb.append(", ");
|
|
||||||
}
|
|
||||||
|
|
||||||
sb.append(col);
|
|
||||||
first = false;
|
|
||||||
}
|
|
||||||
|
|
||||||
sb.append(") ");
|
|
||||||
} else {
|
|
||||||
numSlots = getColumnCount(); // set if columnNames is null.
|
|
||||||
}
|
|
||||||
|
|
||||||
// generates the (?, ?, ?...) used for each row.
|
|
||||||
StringBuilder sbRow = new StringBuilder();
|
|
||||||
sbRow.append("(SELECT ");
|
|
||||||
for (int i = 0; i < numSlots; i++) {
|
|
||||||
if (i != 0) {
|
|
||||||
sbRow.append(", ");
|
|
||||||
}
|
|
||||||
|
|
||||||
sbRow.append("?");
|
|
||||||
}
|
|
||||||
sbRow.append(") ");
|
|
||||||
|
|
||||||
// Now append that numRows times.
|
|
||||||
for (int i = 0; i < numRows; i++) {
|
|
||||||
if (i != 0) {
|
|
||||||
sb.append("UNION ALL ");
|
|
||||||
}
|
|
||||||
|
|
||||||
sb.append(sbRow);
|
|
||||||
}
|
|
||||||
|
|
||||||
return sb.toString();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,27 +18,9 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.io.LongWritable;
|
|
||||||
import org.apache.hadoop.io.NullWritable;
|
|
||||||
import org.apache.hadoop.mapreduce.Mapper.Context;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Reads a SqoopRecord from the SequenceFile in which it's packed and emits
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* that DBWritable to the OutputFormat for writeback to the database.
|
|
||||||
*/
|
*/
|
||||||
public class SequenceFileExportMapper
|
public class SequenceFileExportMapper
|
||||||
extends AutoProgressMapper<LongWritable, SqoopRecord, SqoopRecord,
|
extends org.apache.sqoop.mapreduce.SequenceFileExportMapper {
|
||||||
NullWritable> {
|
|
||||||
|
|
||||||
public SequenceFileExportMapper() {
|
|
||||||
}
|
|
||||||
|
|
||||||
public void map(LongWritable key, SqoopRecord val, Context context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
context.write(val, NullWritable.get());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,50 +18,10 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.sql.SQLException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.io.LongWritable;
|
|
||||||
import org.apache.hadoop.mapreduce.Mapper.Context;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
|
||||||
import com.cloudera.sqoop.lib.LargeObjectLoader;
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Imports records by writing them to a SequenceFile.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
*/
|
*/
|
||||||
public class SequenceFileImportMapper
|
public class SequenceFileImportMapper
|
||||||
extends AutoProgressMapper<LongWritable, SqoopRecord, LongWritable,
|
extends org.apache.sqoop.mapreduce.SequenceFileImportMapper {
|
||||||
SqoopRecord> {
|
|
||||||
|
|
||||||
private LargeObjectLoader lobLoader;
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void setup(Context context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
this.lobLoader = new LargeObjectLoader(context.getConfiguration(),
|
|
||||||
FileOutputFormat.getWorkOutputPath(context));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void map(LongWritable key, SqoopRecord val, Context context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
|
|
||||||
try {
|
|
||||||
// Loading of LOBs was delayed until we have a Context.
|
|
||||||
val.loadLargeObjects(lobLoader);
|
|
||||||
} catch (SQLException sqlE) {
|
|
||||||
throw new IOException(sqlE);
|
|
||||||
}
|
|
||||||
|
|
||||||
context.write(key, val);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void cleanup(Context context) throws IOException {
|
|
||||||
if (null != lobLoader) {
|
|
||||||
lobLoader.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,68 +18,9 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.io.LongWritable;
|
|
||||||
import org.apache.hadoop.io.NullWritable;
|
|
||||||
import org.apache.hadoop.io.Text;
|
|
||||||
import org.apache.hadoop.mapreduce.Mapper.Context;
|
|
||||||
import org.apache.hadoop.util.ReflectionUtils;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.lib.RecordParser;
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Converts an input record from a string representation to a parsed Sqoop
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* record and emits that DBWritable to the OutputFormat for writeback to the
|
|
||||||
* database.
|
|
||||||
*/
|
*/
|
||||||
public class TextExportMapper
|
public class TextExportMapper
|
||||||
extends AutoProgressMapper<LongWritable, Text, SqoopRecord, NullWritable> {
|
extends org.apache.sqoop.mapreduce.TextExportMapper {
|
||||||
|
|
||||||
private SqoopRecord recordImpl;
|
|
||||||
|
|
||||||
public TextExportMapper() {
|
|
||||||
}
|
|
||||||
|
|
||||||
protected void setup(Context context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
super.setup(context);
|
|
||||||
|
|
||||||
Configuration conf = context.getConfiguration();
|
|
||||||
|
|
||||||
// Instantiate a copy of the user's class to hold and parse the record.
|
|
||||||
String recordClassName = conf.get(
|
|
||||||
ExportJobBase.SQOOP_EXPORT_TABLE_CLASS_KEY);
|
|
||||||
if (null == recordClassName) {
|
|
||||||
throw new IOException("Export table class name ("
|
|
||||||
+ ExportJobBase.SQOOP_EXPORT_TABLE_CLASS_KEY
|
|
||||||
+ ") is not set!");
|
|
||||||
}
|
|
||||||
|
|
||||||
try {
|
|
||||||
Class cls = Class.forName(recordClassName, true,
|
|
||||||
Thread.currentThread().getContextClassLoader());
|
|
||||||
recordImpl = (SqoopRecord) ReflectionUtils.newInstance(cls, conf);
|
|
||||||
} catch (ClassNotFoundException cnfe) {
|
|
||||||
throw new IOException(cnfe);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (null == recordImpl) {
|
|
||||||
throw new IOException("Could not instantiate object of type "
|
|
||||||
+ recordClassName);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
public void map(LongWritable key, Text val, Context context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
try {
|
|
||||||
recordImpl.parse(val);
|
|
||||||
context.write(recordImpl, NullWritable.get());
|
|
||||||
} catch (RecordParser.ParseError pe) {
|
|
||||||
throw new IOException("Could not parse record: " + val, pe);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,57 +18,10 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.sql.SQLException;
|
|
||||||
|
|
||||||
import org.apache.hadoop.io.LongWritable;
|
|
||||||
import org.apache.hadoop.io.NullWritable;
|
|
||||||
import org.apache.hadoop.io.Text;
|
|
||||||
import org.apache.hadoop.mapreduce.Mapper.Context;
|
|
||||||
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
|
||||||
import com.cloudera.sqoop.lib.LargeObjectLoader;
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Imports records by transforming them to strings for a plain-text flat file.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
*/
|
*/
|
||||||
public class TextImportMapper
|
public class TextImportMapper
|
||||||
extends AutoProgressMapper<LongWritable, SqoopRecord, Text, NullWritable> {
|
extends org.apache.sqoop.mapreduce.TextImportMapper {
|
||||||
|
|
||||||
private Text outkey;
|
|
||||||
private LargeObjectLoader lobLoader;
|
|
||||||
|
|
||||||
public TextImportMapper() {
|
|
||||||
outkey = new Text();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void setup(Context context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
this.lobLoader = new LargeObjectLoader(context.getConfiguration(),
|
|
||||||
FileOutputFormat.getWorkOutputPath(context));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void map(LongWritable key, SqoopRecord val, Context context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
|
|
||||||
try {
|
|
||||||
// Loading of LOBs was delayed until we have a Context.
|
|
||||||
val.loadLargeObjects(lobLoader);
|
|
||||||
} catch (SQLException sqlE) {
|
|
||||||
throw new IOException(sqlE);
|
|
||||||
}
|
|
||||||
|
|
||||||
outkey.set(val.toString());
|
|
||||||
context.write(outkey, NullWritable.get());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void cleanup(Context context) throws IOException {
|
|
||||||
if (null != lobLoader) {
|
|
||||||
lobLoader.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,6 +1,4 @@
|
|||||||
/**
|
/**
|
||||||
* Copyright 2011 The Apache Software Foundation
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
@ -20,196 +18,11 @@
|
|||||||
|
|
||||||
package com.cloudera.sqoop.mapreduce;
|
package com.cloudera.sqoop.mapreduce;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.sql.Connection;
|
|
||||||
import java.sql.PreparedStatement;
|
|
||||||
import java.sql.SQLException;
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.LinkedHashSet;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.StringTokenizer;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
|
||||||
import org.apache.commons.logging.LogFactory;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.mapreduce.JobContext;
|
|
||||||
import org.apache.hadoop.mapreduce.RecordWriter;
|
|
||||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
|
||||||
|
|
||||||
import com.cloudera.sqoop.lib.SqoopRecord;
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Update an existing table of data with new value data.
|
* @deprecated Moving to use org.apache.sqoop namespace.
|
||||||
* This requires a designated 'key column' for the WHERE clause
|
|
||||||
* of an UPDATE statement.
|
|
||||||
*
|
|
||||||
* Updates are executed en batch in the PreparedStatement.
|
|
||||||
*
|
|
||||||
* Uses DBOutputFormat/DBConfiguration for configuring the output.
|
|
||||||
*/
|
*/
|
||||||
public class UpdateOutputFormat<K extends SqoopRecord, V>
|
public class UpdateOutputFormat<K extends SqoopRecord, V>
|
||||||
extends AsyncSqlOutputFormat<K, V> {
|
extends org.apache.sqoop.mapreduce.UpdateOutputFormat<K, V> {
|
||||||
|
|
||||||
private static final Log LOG = LogFactory.getLog(UpdateOutputFormat.class);
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public void checkOutputSpecs(JobContext context)
|
|
||||||
throws IOException, InterruptedException {
|
|
||||||
Configuration conf = context.getConfiguration();
|
|
||||||
DBConfiguration dbConf = new DBConfiguration(conf);
|
|
||||||
|
|
||||||
// Sanity check all the configuration values we need.
|
|
||||||
if (null == conf.get(DBConfiguration.URL_PROPERTY)) {
|
|
||||||
throw new IOException("Database connection URL is not set.");
|
|
||||||
} else if (null == dbConf.getOutputTableName()) {
|
|
||||||
throw new IOException("Table name is not set for export.");
|
|
||||||
} else if (null == dbConf.getOutputFieldNames()) {
|
|
||||||
throw new IOException(
|
|
||||||
"Output field names are null.");
|
|
||||||
} else if (null == conf.get(ExportJobBase.SQOOP_EXPORT_UPDATE_COL_KEY)) {
|
|
||||||
throw new IOException("Update key column is not set for export.");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
|
||||||
throws IOException {
|
|
||||||
try {
|
|
||||||
return new UpdateRecordWriter(context);
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new IOException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* RecordWriter to write the output to UPDATE statements modifying rows
|
|
||||||
* in the database.
|
|
||||||
*/
|
|
||||||
public class UpdateRecordWriter extends AsyncSqlRecordWriter<K, V> {
|
|
||||||
|
|
||||||
protected String tableName;
|
|
||||||
protected String [] columnNames; // The columns to update.
|
|
||||||
protected String [] updateCols; // The columns containing the fixed key.
|
|
||||||
|
|
||||||
public UpdateRecordWriter(TaskAttemptContext context)
|
|
||||||
throws ClassNotFoundException, SQLException {
|
|
||||||
super(context);
|
|
||||||
|
|
||||||
Configuration conf = getConf();
|
|
||||||
|
|
||||||
DBConfiguration dbConf = new DBConfiguration(conf);
|
|
||||||
this.tableName = dbConf.getOutputTableName();
|
|
||||||
this.columnNames = dbConf.getOutputFieldNames();
|
|
||||||
String updateKeyColumns =
|
|
||||||
conf.get(ExportJobBase.SQOOP_EXPORT_UPDATE_COL_KEY);
|
|
||||||
|
|
||||||
Set<String> updateKeys = new LinkedHashSet<String>();
|
|
||||||
StringTokenizer stok = new StringTokenizer(updateKeyColumns, ",");
|
|
||||||
while (stok.hasMoreTokens()) {
|
|
||||||
String nextUpdateKey = stok.nextToken().trim();
|
|
||||||
if (nextUpdateKey.length() > 0) {
|
|
||||||
updateKeys.add(nextUpdateKey);
|
|
||||||
} else {
|
|
||||||
throw new RuntimeException("Invalid update key column value specified"
|
|
||||||
+ ": '" + updateKeyColumns + "'");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
updateCols = updateKeys.toArray(new String[updateKeys.size()]);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
protected boolean isBatchExec() {
|
|
||||||
// We use batches here.
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return the name of the table we are inserting into.
|
|
||||||
*/
|
|
||||||
protected final String getTableName() {
|
|
||||||
return tableName;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return the list of columns we are updating.
|
|
||||||
*/
|
|
||||||
protected final String [] getColumnNames() {
|
|
||||||
if (null == columnNames) {
|
|
||||||
return null;
|
|
||||||
} else {
|
|
||||||
return Arrays.copyOf(columnNames, columnNames.length);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return the column we are using to determine the row to update.
|
|
||||||
*/
|
|
||||||
protected final String[] getUpdateColumns() {
|
|
||||||
return updateCols;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
protected PreparedStatement getPreparedStatement(
|
|
||||||
List<SqoopRecord> userRecords) throws SQLException {
|
|
||||||
|
|
||||||
PreparedStatement stmt = null;
|
|
||||||
|
|
||||||
// Synchronize on connection to ensure this does not conflict
|
|
||||||
// with the operations in the update thread.
|
|
||||||
Connection conn = getConnection();
|
|
||||||
synchronized (conn) {
|
|
||||||
stmt = conn.prepareStatement(getUpdateStatement());
|
|
||||||
}
|
|
||||||
|
|
||||||
// Inject the record parameters into the UPDATE and WHERE clauses. This
|
|
||||||
// assumes that the update key column is the last column serialized in
|
|
||||||
// by the underlying record. Our code auto-gen process for exports was
|
|
||||||
// responsible for taking care of this constraint.
|
|
||||||
for (SqoopRecord record : userRecords) {
|
|
||||||
record.write(stmt, 0);
|
|
||||||
stmt.addBatch();
|
|
||||||
}
|
|
||||||
|
|
||||||
return stmt;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return an UPDATE statement that modifies rows based on a single key
|
|
||||||
* column (with the intent of modifying a single row).
|
|
||||||
*/
|
|
||||||
protected String getUpdateStatement() {
|
|
||||||
StringBuilder sb = new StringBuilder();
|
|
||||||
sb.append("UPDATE " + this.tableName + " SET ");
|
|
||||||
|
|
||||||
boolean first = true;
|
|
||||||
for (String col : this.columnNames) {
|
|
||||||
if (!first) {
|
|
||||||
sb.append(", ");
|
|
||||||
}
|
|
||||||
|
|
||||||
sb.append(col);
|
|
||||||
sb.append("=?");
|
|
||||||
first = false;
|
|
||||||
}
|
|
||||||
|
|
||||||
sb.append(" WHERE ");
|
|
||||||
first = true;
|
|
||||||
for (int i = 0; i < updateCols.length; i++) {
|
|
||||||
if (first) {
|
|
||||||
first = false;
|
|
||||||
} else {
|
|
||||||
sb.append(" AND ");
|
|
||||||
}
|
|
||||||
sb.append(updateCols[i]).append("=?");
|
|
||||||
}
|
|
||||||
return sb.toString();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
304
src/java/org/apache/sqoop/mapreduce/AsyncSqlOutputFormat.java
Normal file
304
src/java/org/apache/sqoop/mapreduce/AsyncSqlOutputFormat.java
Normal file
@ -0,0 +1,304 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.Connection;
|
||||||
|
import java.sql.PreparedStatement;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.util.concurrent.SynchronousQueue;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.mapreduce.JobContext;
|
||||||
|
import org.apache.hadoop.mapreduce.OutputCommitter;
|
||||||
|
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
import org.apache.hadoop.util.StringUtils;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Abstract OutputFormat class that allows the RecordWriter to buffer
|
||||||
|
* up SQL commands which should be executed in a separate thread after
|
||||||
|
* enough commands are created.
|
||||||
|
*
|
||||||
|
* This supports a configurable "spill threshold" at which
|
||||||
|
* point intermediate transactions are committed.
|
||||||
|
*
|
||||||
|
* Uses DBOutputFormat/DBConfiguration for configuring the output.
|
||||||
|
* This is used in conjunction with the abstract AsyncSqlRecordWriter
|
||||||
|
* class.
|
||||||
|
*
|
||||||
|
* Clients of this OutputFormat must implement getRecordWriter(); the
|
||||||
|
* returned RecordWriter is intended to subclass AsyncSqlRecordWriter.
|
||||||
|
*/
|
||||||
|
public abstract class AsyncSqlOutputFormat<K extends SqoopRecord, V>
|
||||||
|
extends OutputFormat<K, V> {
|
||||||
|
|
||||||
|
/** conf key: number of rows to export per INSERT statement. */
|
||||||
|
public static final String RECORDS_PER_STATEMENT_KEY =
|
||||||
|
"sqoop.export.records.per.statement";
|
||||||
|
|
||||||
|
/** conf key: number of INSERT statements to bundle per tx.
|
||||||
|
* If this is set to -1, then a single transaction will be used
|
||||||
|
* per task. Note that each statement may encompass multiple
|
||||||
|
* rows, depending on the value of sqoop.export.records.per.statement.
|
||||||
|
*/
|
||||||
|
public static final String STATEMENTS_PER_TRANSACTION_KEY =
|
||||||
|
"sqoop.export.statements.per.transaction";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Default number of records to put in an INSERT statement or
|
||||||
|
* other batched update statement.
|
||||||
|
*/
|
||||||
|
public static final int DEFAULT_RECORDS_PER_STATEMENT = 100;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Default number of statements to execute before committing the
|
||||||
|
* current transaction.
|
||||||
|
*/
|
||||||
|
public static final int DEFAULT_STATEMENTS_PER_TRANSACTION = 100;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Value for STATEMENTS_PER_TRANSACTION_KEY signifying that we should
|
||||||
|
* not commit until the RecordWriter is being closed, regardless of
|
||||||
|
* the number of statements we execute.
|
||||||
|
*/
|
||||||
|
public static final int UNLIMITED_STATEMENTS_PER_TRANSACTION = -1;
|
||||||
|
|
||||||
|
private static final Log LOG = LogFactory.getLog(AsyncSqlOutputFormat.class);
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public void checkOutputSpecs(JobContext context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public OutputCommitter getOutputCommitter(TaskAttemptContext context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
return new NullOutputCommitter();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Represents a database update operation that should be performed
|
||||||
|
* by an asynchronous background thread.
|
||||||
|
* AsyncDBOperation objects are immutable.
|
||||||
|
* They MAY contain a statement which should be executed. The
|
||||||
|
* statement may also be null.
|
||||||
|
*
|
||||||
|
* They may also set 'commitAndClose' to true. If true, then the
|
||||||
|
* executor of this operation should commit the current
|
||||||
|
* transaction, even if stmt is null, and then stop the executor
|
||||||
|
* thread.
|
||||||
|
*/
|
||||||
|
public static class AsyncDBOperation {
|
||||||
|
private final PreparedStatement stmt;
|
||||||
|
private final boolean isBatch;
|
||||||
|
private final boolean commit;
|
||||||
|
private final boolean stopThread;
|
||||||
|
|
||||||
|
@Deprecated
|
||||||
|
/** Do not use AsyncDBOperation(PreparedStatement s, boolean
|
||||||
|
* commitAndClose, boolean batch). Use AsyncDBOperation(PreparedStatement
|
||||||
|
* s, boolean batch, boolean commit, boolean stopThread) instead.
|
||||||
|
*/
|
||||||
|
public AsyncDBOperation(PreparedStatement s, boolean commitAndClose,
|
||||||
|
boolean batch) {
|
||||||
|
this(s, batch, commitAndClose, commitAndClose);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create an asynchronous database operation.
|
||||||
|
* @param s the statement, if any, to execute.
|
||||||
|
* @param batch is true if this is a batch PreparedStatement, or false
|
||||||
|
* if it's a normal singleton statement.
|
||||||
|
* @param commit is true if this statement should be committed to the
|
||||||
|
* database.
|
||||||
|
* @param stopThread if true, the executor thread should stop after this
|
||||||
|
* operation.
|
||||||
|
*/
|
||||||
|
public AsyncDBOperation(PreparedStatement s, boolean batch,
|
||||||
|
boolean commit, boolean stopThread) {
|
||||||
|
this.stmt = s;
|
||||||
|
this.isBatch = batch;
|
||||||
|
this.commit = commit;
|
||||||
|
this.stopThread = stopThread;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a statement to run as an update.
|
||||||
|
*/
|
||||||
|
public PreparedStatement getStatement() {
|
||||||
|
return stmt;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return true if the executor should commit the current transaction.
|
||||||
|
* If getStatement() is non-null, the statement is run first.
|
||||||
|
*/
|
||||||
|
public boolean requiresCommit() {
|
||||||
|
return this.commit;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return true if the executor should stop after this command.
|
||||||
|
*/
|
||||||
|
public boolean stop() {
|
||||||
|
return this.stopThread;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return true if this is a batch SQL statement.
|
||||||
|
*/
|
||||||
|
public boolean execAsBatch() {
|
||||||
|
return this.isBatch;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A thread that runs the database interactions asynchronously
|
||||||
|
* from the OutputCollector.
|
||||||
|
*/
|
||||||
|
public static class AsyncSqlExecThread extends Thread {
|
||||||
|
|
||||||
|
private final Connection conn; // The connection to the database.
|
||||||
|
private SQLException err; // Error from a previously-run statement.
|
||||||
|
|
||||||
|
// How we receive database operations from the RecordWriter.
|
||||||
|
private SynchronousQueue<AsyncDBOperation> opsQueue;
|
||||||
|
|
||||||
|
protected int curNumStatements; // statements executed thus far in the tx.
|
||||||
|
protected final int stmtsPerTx; // statements per transaction.
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a new update thread that interacts with the database.
|
||||||
|
* @param conn the connection to use. This must only be used by this
|
||||||
|
* thread.
|
||||||
|
* @param stmtsPerTx the number of statements to execute before committing
|
||||||
|
* the current transaction.
|
||||||
|
*/
|
||||||
|
public AsyncSqlExecThread(Connection conn, int stmtsPerTx) {
|
||||||
|
this.conn = conn;
|
||||||
|
this.err = null;
|
||||||
|
this.opsQueue = new SynchronousQueue<AsyncDBOperation>();
|
||||||
|
this.stmtsPerTx = stmtsPerTx;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void run() {
|
||||||
|
while (true) {
|
||||||
|
AsyncDBOperation op = null;
|
||||||
|
try {
|
||||||
|
op = opsQueue.take();
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
LOG.warn("Interrupted retrieving from operation queue: "
|
||||||
|
+ StringUtils.stringifyException(ie));
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (null == op) {
|
||||||
|
// This shouldn't be allowed to happen.
|
||||||
|
LOG.warn("Null operation in queue; illegal state.");
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
PreparedStatement stmt = op.getStatement();
|
||||||
|
// Synchronize on the connection to ensure it does not conflict
|
||||||
|
// with the prepareStatement() call in the main thread.
|
||||||
|
synchronized (conn) {
|
||||||
|
try {
|
||||||
|
if (null != stmt) {
|
||||||
|
if (op.execAsBatch()) {
|
||||||
|
stmt.executeBatch();
|
||||||
|
} else {
|
||||||
|
stmt.execute();
|
||||||
|
}
|
||||||
|
stmt.close();
|
||||||
|
stmt = null;
|
||||||
|
this.curNumStatements++;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (op.requiresCommit() || (curNumStatements >= stmtsPerTx
|
||||||
|
&& stmtsPerTx != UNLIMITED_STATEMENTS_PER_TRANSACTION)) {
|
||||||
|
LOG.debug("Committing transaction of " + curNumStatements
|
||||||
|
+ " statements");
|
||||||
|
this.conn.commit();
|
||||||
|
this.curNumStatements = 0;
|
||||||
|
}
|
||||||
|
} catch (SQLException sqlE) {
|
||||||
|
setLastError(sqlE);
|
||||||
|
} finally {
|
||||||
|
// Close the statement on our way out if that didn't happen
|
||||||
|
// via the normal execution path.
|
||||||
|
if (null != stmt) {
|
||||||
|
try {
|
||||||
|
stmt.close();
|
||||||
|
} catch (SQLException sqlE) {
|
||||||
|
setLastError(sqlE);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Always check whether we should end the loop, regardless
|
||||||
|
// of the presence of an exception.
|
||||||
|
if (op.stop()) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
} // try .. catch .. finally.
|
||||||
|
} // synchronized (conn)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Allows a user to enqueue the next database operation to run.
|
||||||
|
* Since the connection can only execute a single operation at a time,
|
||||||
|
* the put() method may block if another operation is already underway.
|
||||||
|
* @param op the database operation to perform.
|
||||||
|
*/
|
||||||
|
public void put(AsyncDBOperation op) throws InterruptedException {
|
||||||
|
opsQueue.put(op);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* If a previously-executed statement resulted in an error, post it here.
|
||||||
|
* If the error slot was already filled, then subsequent errors are
|
||||||
|
* squashed until the user calls this method (which clears the error
|
||||||
|
* slot).
|
||||||
|
* @return any SQLException that occurred due to a previously-run
|
||||||
|
* statement.
|
||||||
|
*/
|
||||||
|
public synchronized SQLException getLastError() {
|
||||||
|
SQLException e = this.err;
|
||||||
|
this.err = null;
|
||||||
|
return e;
|
||||||
|
}
|
||||||
|
|
||||||
|
private synchronized void setLastError(SQLException e) {
|
||||||
|
if (this.err == null) {
|
||||||
|
// Just set it.
|
||||||
|
LOG.error("Got exception in update thread: "
|
||||||
|
+ StringUtils.stringifyException(e));
|
||||||
|
this.err = e;
|
||||||
|
} else {
|
||||||
|
// Slot is full. Log it and discard.
|
||||||
|
LOG.error("SQLException in update thread but error slot full: "
|
||||||
|
+ StringUtils.stringifyException(e));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
223
src/java/org/apache/sqoop/mapreduce/AsyncSqlRecordWriter.java
Normal file
223
src/java/org/apache/sqoop/mapreduce/AsyncSqlRecordWriter.java
Normal file
@ -0,0 +1,223 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.Connection;
|
||||||
|
import java.sql.PreparedStatement;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.mapreduce.RecordWriter;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
import org.apache.sqoop.util.LoggingUtils;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Abstract RecordWriter base class that buffers SqoopRecords to be injected
|
||||||
|
* into JDBC SQL PreparedStatements to be executed by the
|
||||||
|
* AsyncSqlOutputFormat's background thread.
|
||||||
|
*
|
||||||
|
* Record objects are buffered before actually performing the INSERT
|
||||||
|
* statements; this requires that the key implement the SqoopRecord interface.
|
||||||
|
*
|
||||||
|
* Uses DBOutputFormat/DBConfiguration for configuring the output.
|
||||||
|
*/
|
||||||
|
public abstract class AsyncSqlRecordWriter<K extends SqoopRecord, V>
|
||||||
|
extends RecordWriter<K, V> {
|
||||||
|
|
||||||
|
private static final Log LOG = LogFactory.getLog(AsyncSqlRecordWriter.class);
|
||||||
|
|
||||||
|
private Connection connection;
|
||||||
|
|
||||||
|
private Configuration conf;
|
||||||
|
|
||||||
|
protected final int rowsPerStmt; // rows to insert per statement.
|
||||||
|
|
||||||
|
// Buffer for records to be put into export SQL statements.
|
||||||
|
private List<SqoopRecord> records;
|
||||||
|
|
||||||
|
// Background thread to actually perform the updates.
|
||||||
|
private AsyncSqlOutputFormat.AsyncSqlExecThread execThread;
|
||||||
|
private boolean startedExecThread;
|
||||||
|
|
||||||
|
public AsyncSqlRecordWriter(TaskAttemptContext context)
|
||||||
|
throws ClassNotFoundException, SQLException {
|
||||||
|
this.conf = context.getConfiguration();
|
||||||
|
|
||||||
|
this.rowsPerStmt = conf.getInt(
|
||||||
|
AsyncSqlOutputFormat.RECORDS_PER_STATEMENT_KEY,
|
||||||
|
AsyncSqlOutputFormat.DEFAULT_RECORDS_PER_STATEMENT);
|
||||||
|
int stmtsPerTx = conf.getInt(
|
||||||
|
AsyncSqlOutputFormat.STATEMENTS_PER_TRANSACTION_KEY,
|
||||||
|
AsyncSqlOutputFormat.DEFAULT_STATEMENTS_PER_TRANSACTION);
|
||||||
|
|
||||||
|
DBConfiguration dbConf = new DBConfiguration(conf);
|
||||||
|
this.connection = dbConf.getConnection();
|
||||||
|
this.connection.setAutoCommit(false);
|
||||||
|
|
||||||
|
this.records = new ArrayList<SqoopRecord>(this.rowsPerStmt);
|
||||||
|
|
||||||
|
this.execThread = new AsyncSqlOutputFormat.AsyncSqlExecThread(
|
||||||
|
connection, stmtsPerTx);
|
||||||
|
this.execThread.setDaemon(true);
|
||||||
|
this.startedExecThread = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Allow subclasses access to the Connection instance we hold.
|
||||||
|
* This Connection is shared with the asynchronous SQL exec thread.
|
||||||
|
* Any uses of the Connection must be synchronized on it.
|
||||||
|
* @return the Connection object used for this SQL transaction.
|
||||||
|
*/
|
||||||
|
protected final Connection getConnection() {
|
||||||
|
return this.connection;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Allow subclasses access to the Configuration.
|
||||||
|
* @return the Configuration for this MapReduc task.
|
||||||
|
*/
|
||||||
|
protected final Configuration getConf() {
|
||||||
|
return this.conf;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Should return 'true' if the PreparedStatements generated by the
|
||||||
|
* RecordWriter are intended to be executed in "batch" mode, or false
|
||||||
|
* if it's just one big statement.
|
||||||
|
*/
|
||||||
|
protected boolean isBatchExec() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generate the PreparedStatement object that will be fed into the execution
|
||||||
|
* thread. All parameterized fields of the PreparedStatement must be set in
|
||||||
|
* this method as well; this is usually based on the records collected from
|
||||||
|
* the user in the userRecords list.
|
||||||
|
*
|
||||||
|
* Note that any uses of the Connection object here must be synchronized on
|
||||||
|
* the Connection.
|
||||||
|
*
|
||||||
|
* @param userRecords a list of records that should be injected into SQL
|
||||||
|
* statements.
|
||||||
|
* @return a PreparedStatement to be populated with rows
|
||||||
|
* from the collected record list.
|
||||||
|
*/
|
||||||
|
protected abstract PreparedStatement getPreparedStatement(
|
||||||
|
List<SqoopRecord> userRecords) throws SQLException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Takes the current contents of 'records' and formats and executes the
|
||||||
|
* INSERT statement.
|
||||||
|
* @param closeConn if true, commits the transaction and closes the
|
||||||
|
* connection.
|
||||||
|
*/
|
||||||
|
private void execUpdate(boolean commit, boolean stopThread)
|
||||||
|
throws InterruptedException, SQLException {
|
||||||
|
|
||||||
|
if (!startedExecThread) {
|
||||||
|
this.execThread.start();
|
||||||
|
this.startedExecThread = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
PreparedStatement stmt = null;
|
||||||
|
boolean successfulPut = false;
|
||||||
|
try {
|
||||||
|
if (records.size() > 0) {
|
||||||
|
stmt = getPreparedStatement(records);
|
||||||
|
this.records.clear();
|
||||||
|
}
|
||||||
|
|
||||||
|
// Pass this operation off to the update thread. This will block if
|
||||||
|
// the update thread is already performing an update.
|
||||||
|
AsyncSqlOutputFormat.AsyncDBOperation op =
|
||||||
|
new AsyncSqlOutputFormat.AsyncDBOperation(stmt, isBatchExec(),
|
||||||
|
commit, stopThread);
|
||||||
|
execThread.put(op);
|
||||||
|
successfulPut = true; // op has been posted to the other thread.
|
||||||
|
} finally {
|
||||||
|
if (!successfulPut && null != stmt) {
|
||||||
|
// We created a statement but failed to enqueue it. Close it.
|
||||||
|
stmt.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Check for any previous SQLException. If one happened, rethrow it here.
|
||||||
|
SQLException lastException = execThread.getLastError();
|
||||||
|
if (null != lastException) {
|
||||||
|
LoggingUtils.logAll(LOG, lastException);
|
||||||
|
throw lastException;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public void close(TaskAttemptContext context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
try {
|
||||||
|
try {
|
||||||
|
execUpdate(true, true);
|
||||||
|
execThread.join();
|
||||||
|
} catch (SQLException sqle) {
|
||||||
|
throw new IOException(sqle);
|
||||||
|
}
|
||||||
|
|
||||||
|
// If we're not leaving on an error return path already,
|
||||||
|
// now that execThread is definitely stopped, check that the
|
||||||
|
// error slot remains empty.
|
||||||
|
SQLException lastErr = execThread.getLastError();
|
||||||
|
if (null != lastErr) {
|
||||||
|
throw new IOException(lastErr);
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
try {
|
||||||
|
closeConnection(context);
|
||||||
|
} catch (SQLException sqle) {
|
||||||
|
throw new IOException(sqle);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void closeConnection(TaskAttemptContext context)
|
||||||
|
throws SQLException {
|
||||||
|
this.connection.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public void write(K key, V value)
|
||||||
|
throws InterruptedException, IOException {
|
||||||
|
try {
|
||||||
|
records.add((SqoopRecord) key.clone());
|
||||||
|
if (records.size() >= this.rowsPerStmt) {
|
||||||
|
execUpdate(false, false);
|
||||||
|
}
|
||||||
|
} catch (CloneNotSupportedException cnse) {
|
||||||
|
throw new IOException("Could not buffer record", cnse);
|
||||||
|
} catch (SQLException sqlException) {
|
||||||
|
throw new IOException(sqlException);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
199
src/java/org/apache/sqoop/mapreduce/AutoProgressMapper.java
Normal file
199
src/java/org/apache/sqoop/mapreduce/AutoProgressMapper.java
Normal file
@ -0,0 +1,199 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Identity mapper that continuously reports progress via a background thread.
|
||||||
|
*/
|
||||||
|
public class AutoProgressMapper<KEYIN, VALIN, KEYOUT, VALOUT>
|
||||||
|
extends Mapper<KEYIN, VALIN, KEYOUT, VALOUT> {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(
|
||||||
|
AutoProgressMapper.class.getName());
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Total number of millis for which progress will be reported by the
|
||||||
|
* auto-progress thread. If this is zero, then the auto-progress thread will
|
||||||
|
* never voluntarily exit.
|
||||||
|
*/
|
||||||
|
private int maxProgressPeriod;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Number of milliseconds to sleep for between loop iterations. Must be less
|
||||||
|
* than report interval.
|
||||||
|
*/
|
||||||
|
private int sleepInterval;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Number of milliseconds between calls to Reporter.progress().
|
||||||
|
* Should be a multiple of the sleepInterval.
|
||||||
|
*/
|
||||||
|
private int reportInterval;
|
||||||
|
|
||||||
|
public static final String MAX_PROGRESS_PERIOD_KEY =
|
||||||
|
"sqoop.mapred.auto.progress.max";
|
||||||
|
public static final String SLEEP_INTERVAL_KEY =
|
||||||
|
"sqoop.mapred.auto.progress.sleep";
|
||||||
|
public static final String REPORT_INTERVAL_KEY =
|
||||||
|
"sqoop.mapred.auto.progress.report";
|
||||||
|
|
||||||
|
// Sleep for 10 seconds at a time.
|
||||||
|
public static final int DEFAULT_SLEEP_INTERVAL = 10000;
|
||||||
|
|
||||||
|
// Report progress every 30 seconds.
|
||||||
|
public static final int DEFAULT_REPORT_INTERVAL = 30000;
|
||||||
|
|
||||||
|
// Disable max progress, by default.
|
||||||
|
public static final int DEFAULT_MAX_PROGRESS = 0;
|
||||||
|
|
||||||
|
private class ProgressThread extends Thread {
|
||||||
|
|
||||||
|
private volatile boolean keepGoing; // While this is true, thread runs.
|
||||||
|
|
||||||
|
private Context context;
|
||||||
|
private long startTimeMillis;
|
||||||
|
private long lastReportMillis;
|
||||||
|
|
||||||
|
public ProgressThread(final Context ctxt) {
|
||||||
|
this.context = ctxt;
|
||||||
|
this.keepGoing = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void signalShutdown() {
|
||||||
|
this.keepGoing = false; // volatile update.
|
||||||
|
this.interrupt();
|
||||||
|
}
|
||||||
|
|
||||||
|
public void run() {
|
||||||
|
this.lastReportMillis = System.currentTimeMillis();
|
||||||
|
this.startTimeMillis = this.lastReportMillis;
|
||||||
|
|
||||||
|
final long MAX_PROGRESS = AutoProgressMapper.this.maxProgressPeriod;
|
||||||
|
final long REPORT_INTERVAL = AutoProgressMapper.this.reportInterval;
|
||||||
|
final long SLEEP_INTERVAL = AutoProgressMapper.this.sleepInterval;
|
||||||
|
|
||||||
|
// In a loop:
|
||||||
|
// * Check that we haven't run for too long (maxProgressPeriod).
|
||||||
|
// * If it's been a report interval since we last made progress,
|
||||||
|
// make more.
|
||||||
|
// * Sleep for a bit.
|
||||||
|
// * If the parent thread has signaled for exit, do so.
|
||||||
|
while (this.keepGoing) {
|
||||||
|
long curTimeMillis = System.currentTimeMillis();
|
||||||
|
|
||||||
|
if (MAX_PROGRESS != 0
|
||||||
|
&& curTimeMillis - this.startTimeMillis > MAX_PROGRESS) {
|
||||||
|
this.keepGoing = false;
|
||||||
|
LOG.info("Auto-progress thread exiting after " + MAX_PROGRESS
|
||||||
|
+ " ms.");
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (curTimeMillis - this.lastReportMillis > REPORT_INTERVAL) {
|
||||||
|
// It's been a full report interval -- claim progress.
|
||||||
|
LOG.debug("Auto-progress thread reporting progress");
|
||||||
|
this.context.progress();
|
||||||
|
this.lastReportMillis = curTimeMillis;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Unless we got an interrupt while we were working,
|
||||||
|
// sleep a bit before doing more work.
|
||||||
|
if (!Thread.interrupted()) {
|
||||||
|
try {
|
||||||
|
Thread.sleep(SLEEP_INTERVAL);
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
// we were notified on something; not necessarily an error.
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
LOG.info("Auto-progress thread is finished. keepGoing=" + this.keepGoing);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set configuration parameters for the auto-progress thread.
|
||||||
|
*/
|
||||||
|
private void configureAutoProgress(Configuration job) {
|
||||||
|
this.maxProgressPeriod = job.getInt(MAX_PROGRESS_PERIOD_KEY,
|
||||||
|
DEFAULT_MAX_PROGRESS);
|
||||||
|
this.sleepInterval = job.getInt(SLEEP_INTERVAL_KEY,
|
||||||
|
DEFAULT_SLEEP_INTERVAL);
|
||||||
|
this.reportInterval = job.getInt(REPORT_INTERVAL_KEY,
|
||||||
|
DEFAULT_REPORT_INTERVAL);
|
||||||
|
|
||||||
|
if (this.reportInterval < 1) {
|
||||||
|
LOG.warn("Invalid " + REPORT_INTERVAL_KEY + "; setting to "
|
||||||
|
+ DEFAULT_REPORT_INTERVAL);
|
||||||
|
this.reportInterval = DEFAULT_REPORT_INTERVAL;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (this.sleepInterval > this.reportInterval || this.sleepInterval < 1) {
|
||||||
|
LOG.warn("Invalid " + SLEEP_INTERVAL_KEY + "; setting to "
|
||||||
|
+ DEFAULT_SLEEP_INTERVAL);
|
||||||
|
this.sleepInterval = DEFAULT_SLEEP_INTERVAL;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (this.maxProgressPeriod < 0) {
|
||||||
|
LOG.warn("Invalid " + MAX_PROGRESS_PERIOD_KEY + "; setting to "
|
||||||
|
+ DEFAULT_MAX_PROGRESS);
|
||||||
|
this.maxProgressPeriod = DEFAULT_MAX_PROGRESS;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
// map() method intentionally omitted; Mapper.map() is the identity mapper.
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Run the mapping process for this task, wrapped in an auto-progress system.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void run(Context context) throws IOException, InterruptedException {
|
||||||
|
configureAutoProgress(context.getConfiguration());
|
||||||
|
ProgressThread thread = this.new ProgressThread(context);
|
||||||
|
|
||||||
|
try {
|
||||||
|
thread.setDaemon(true);
|
||||||
|
thread.start();
|
||||||
|
|
||||||
|
// use default run() method to actually drive the mapping.
|
||||||
|
super.run(context);
|
||||||
|
} finally {
|
||||||
|
// Tell the progress thread to exit..
|
||||||
|
LOG.debug("Instructing auto-progress thread to quit.");
|
||||||
|
thread.signalShutdown();
|
||||||
|
try {
|
||||||
|
// And wait for that to happen.
|
||||||
|
LOG.debug("Waiting for progress thread shutdown...");
|
||||||
|
thread.join();
|
||||||
|
LOG.debug("Progress thread shutdown detected.");
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
LOG.warn("Interrupted when waiting on auto-progress thread: "
|
||||||
|
+ ie.toString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
204
src/java/org/apache/sqoop/mapreduce/AvroExportMapper.java
Normal file
204
src/java/org/apache/sqoop/mapreduce/AvroExportMapper.java
Normal file
@ -0,0 +1,204 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.math.BigDecimal;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.sql.Date;
|
||||||
|
import java.sql.Time;
|
||||||
|
import java.sql.Timestamp;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import org.apache.avro.Schema;
|
||||||
|
import org.apache.avro.Schema.Field;
|
||||||
|
import org.apache.avro.generic.GenericEnumSymbol;
|
||||||
|
import org.apache.avro.generic.GenericFixed;
|
||||||
|
import org.apache.avro.generic.GenericRecord;
|
||||||
|
import org.apache.avro.mapred.AvroWrapper;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.io.BytesWritable;
|
||||||
|
import org.apache.hadoop.io.DefaultStringifier;
|
||||||
|
import org.apache.hadoop.io.MapWritable;
|
||||||
|
import org.apache.hadoop.io.NullWritable;
|
||||||
|
import org.apache.hadoop.io.Writable;
|
||||||
|
import org.apache.hadoop.util.ReflectionUtils;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.mapreduce.AutoProgressMapper;
|
||||||
|
import com.cloudera.sqoop.orm.ClassWriter;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Exports records from an Avro data file.
|
||||||
|
*/
|
||||||
|
public class AvroExportMapper
|
||||||
|
extends AutoProgressMapper<AvroWrapper<GenericRecord>, NullWritable,
|
||||||
|
SqoopRecord, NullWritable> {
|
||||||
|
|
||||||
|
private static final String TIMESTAMP_TYPE = "java.sql.Timestamp";
|
||||||
|
|
||||||
|
private static final String TIME_TYPE = "java.sql.Time";
|
||||||
|
|
||||||
|
private static final String DATE_TYPE = "java.sql.Date";
|
||||||
|
|
||||||
|
private static final String BIG_DECIMAL_TYPE = "java.math.BigDecimal";
|
||||||
|
|
||||||
|
public static final String AVRO_COLUMN_TYPES_MAP = "sqoop.avro.column.types.map";
|
||||||
|
|
||||||
|
private MapWritable columnTypes;
|
||||||
|
private SqoopRecord recordImpl;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void setup(Context context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
|
||||||
|
super.setup(context);
|
||||||
|
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
|
||||||
|
// Instantiate a copy of the user's class to hold and parse the record.
|
||||||
|
String recordClassName = conf.get(
|
||||||
|
ExportJobBase.SQOOP_EXPORT_TABLE_CLASS_KEY);
|
||||||
|
if (null == recordClassName) {
|
||||||
|
throw new IOException("Export table class name ("
|
||||||
|
+ ExportJobBase.SQOOP_EXPORT_TABLE_CLASS_KEY
|
||||||
|
+ ") is not set!");
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
Class cls = Class.forName(recordClassName, true,
|
||||||
|
Thread.currentThread().getContextClassLoader());
|
||||||
|
recordImpl = (SqoopRecord) ReflectionUtils.newInstance(cls, conf);
|
||||||
|
} catch (ClassNotFoundException cnfe) {
|
||||||
|
throw new IOException(cnfe);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (null == recordImpl) {
|
||||||
|
throw new IOException("Could not instantiate object of type "
|
||||||
|
+ recordClassName);
|
||||||
|
}
|
||||||
|
|
||||||
|
columnTypes = DefaultStringifier.load(conf, AVRO_COLUMN_TYPES_MAP,
|
||||||
|
MapWritable.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void map(AvroWrapper<GenericRecord> key, NullWritable value,
|
||||||
|
Context context) throws IOException, InterruptedException {
|
||||||
|
context.write(toSqoopRecord(key.datum()), NullWritable.get());
|
||||||
|
}
|
||||||
|
|
||||||
|
private SqoopRecord toSqoopRecord(GenericRecord record) throws IOException {
|
||||||
|
Schema avroSchema = record.getSchema();
|
||||||
|
for (Map.Entry<Writable, Writable> e : columnTypes.entrySet()) {
|
||||||
|
String columnName = e.getKey().toString();
|
||||||
|
String columnType = e.getValue().toString();
|
||||||
|
String cleanedCol = ClassWriter.toIdentifier(columnName);
|
||||||
|
Field field = getField(avroSchema, cleanedCol, record);
|
||||||
|
if (field == null) {
|
||||||
|
throw new IOException("Cannot find field " + cleanedCol
|
||||||
|
+ " in Avro schema " + avroSchema);
|
||||||
|
} else {
|
||||||
|
Object avroObject = record.get(field.name());
|
||||||
|
Object fieldVal = fromAvro(avroObject, field.schema(), columnType);
|
||||||
|
recordImpl.setField(cleanedCol, fieldVal);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return recordImpl;
|
||||||
|
}
|
||||||
|
|
||||||
|
private Field getField(Schema avroSchema, String fieldName,
|
||||||
|
GenericRecord record) {
|
||||||
|
for (Field field : avroSchema.getFields()) {
|
||||||
|
if (field.name().equalsIgnoreCase(fieldName)) {
|
||||||
|
return field;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
private Object fromAvro(Object avroObject, Schema fieldSchema,
|
||||||
|
String columnType) {
|
||||||
|
// map from Avro type to Sqoop's Java representation of the SQL type
|
||||||
|
// see SqlManager#toJavaType
|
||||||
|
|
||||||
|
if (avroObject == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
switch (fieldSchema.getType()) {
|
||||||
|
case NULL:
|
||||||
|
return null;
|
||||||
|
case BOOLEAN:
|
||||||
|
case INT:
|
||||||
|
case FLOAT:
|
||||||
|
case DOUBLE:
|
||||||
|
return avroObject;
|
||||||
|
case LONG:
|
||||||
|
if (columnType.equals(DATE_TYPE)) {
|
||||||
|
return new Date((Long) avroObject);
|
||||||
|
} else if (columnType.equals(TIME_TYPE)) {
|
||||||
|
return new Time((Long) avroObject);
|
||||||
|
} else if (columnType.equals(TIMESTAMP_TYPE)) {
|
||||||
|
return new Timestamp((Long) avroObject);
|
||||||
|
}
|
||||||
|
return avroObject;
|
||||||
|
case BYTES:
|
||||||
|
ByteBuffer bb = (ByteBuffer) avroObject;
|
||||||
|
BytesWritable bw = new BytesWritable();
|
||||||
|
bw.set(bb.array(), bb.arrayOffset() + bb.position(), bb.remaining());
|
||||||
|
return bw;
|
||||||
|
case STRING:
|
||||||
|
if (columnType.equals(BIG_DECIMAL_TYPE)) {
|
||||||
|
return new BigDecimal(avroObject.toString());
|
||||||
|
} else if (columnType.equals(DATE_TYPE)) {
|
||||||
|
return Date.valueOf(avroObject.toString());
|
||||||
|
} else if (columnType.equals(TIME_TYPE)) {
|
||||||
|
return Time.valueOf(avroObject.toString());
|
||||||
|
} else if (columnType.equals(TIMESTAMP_TYPE)) {
|
||||||
|
return Timestamp.valueOf(avroObject.toString());
|
||||||
|
}
|
||||||
|
return avroObject.toString();
|
||||||
|
case ENUM:
|
||||||
|
return ((GenericEnumSymbol) avroObject).toString();
|
||||||
|
case UNION:
|
||||||
|
List<Schema> types = fieldSchema.getTypes();
|
||||||
|
if (types.size() != 2) {
|
||||||
|
throw new IllegalArgumentException("Only support union with null");
|
||||||
|
}
|
||||||
|
Schema s1 = types.get(0);
|
||||||
|
Schema s2 = types.get(1);
|
||||||
|
if (s1.getType() == Schema.Type.NULL) {
|
||||||
|
return fromAvro(avroObject, s2, columnType);
|
||||||
|
} else if (s2.getType() == Schema.Type.NULL) {
|
||||||
|
return fromAvro(avroObject, s1, columnType);
|
||||||
|
} else {
|
||||||
|
throw new IllegalArgumentException("Only support union with null");
|
||||||
|
}
|
||||||
|
case FIXED:
|
||||||
|
return new BytesWritable(((GenericFixed) avroObject).bytes());
|
||||||
|
case RECORD:
|
||||||
|
case ARRAY:
|
||||||
|
case MAP:
|
||||||
|
default:
|
||||||
|
throw new IllegalArgumentException("Cannot convert Avro type "
|
||||||
|
+ fieldSchema.getType());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
101
src/java/org/apache/sqoop/mapreduce/AvroImportMapper.java
Normal file
101
src/java/org/apache/sqoop/mapreduce/AvroImportMapper.java
Normal file
@ -0,0 +1,101 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.math.BigDecimal;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.sql.Date;
|
||||||
|
import java.sql.Time;
|
||||||
|
import java.sql.Timestamp;
|
||||||
|
import java.util.Map;
|
||||||
|
import org.apache.avro.Schema;
|
||||||
|
import org.apache.avro.generic.GenericData;
|
||||||
|
import org.apache.avro.generic.GenericRecord;
|
||||||
|
import org.apache.avro.mapred.AvroWrapper;
|
||||||
|
import org.apache.hadoop.io.BytesWritable;
|
||||||
|
import org.apache.hadoop.io.LongWritable;
|
||||||
|
import org.apache.hadoop.io.NullWritable;
|
||||||
|
import com.cloudera.sqoop.lib.BlobRef;
|
||||||
|
import com.cloudera.sqoop.lib.ClobRef;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.mapreduce.AutoProgressMapper;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Imports records by transforming them to Avro records in an Avro data file.
|
||||||
|
*/
|
||||||
|
public class AvroImportMapper
|
||||||
|
extends AutoProgressMapper<LongWritable, SqoopRecord,
|
||||||
|
AvroWrapper<GenericRecord>, NullWritable> {
|
||||||
|
|
||||||
|
private final AvroWrapper<GenericRecord> wrapper =
|
||||||
|
new AvroWrapper<GenericRecord>();
|
||||||
|
private Schema schema;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void setup(Context context) {
|
||||||
|
schema = AvroJob.getMapOutputSchema(context.getConfiguration());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void map(LongWritable key, SqoopRecord val, Context context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
wrapper.datum(toGenericRecord(val));
|
||||||
|
context.write(wrapper, NullWritable.get());
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private GenericRecord toGenericRecord(SqoopRecord val) {
|
||||||
|
Map<String, Object> fieldMap = val.getFieldMap();
|
||||||
|
GenericRecord record = new GenericData.Record(schema);
|
||||||
|
for (Map.Entry<String, Object> entry : fieldMap.entrySet()) {
|
||||||
|
record.put(entry.getKey(), toAvro(entry.getValue()));
|
||||||
|
}
|
||||||
|
return record;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convert the Avro representation of a Java type (that has already been
|
||||||
|
* converted from the SQL equivalent).
|
||||||
|
* @param o
|
||||||
|
* @return
|
||||||
|
*/
|
||||||
|
private Object toAvro(Object o) {
|
||||||
|
if (o instanceof BigDecimal) {
|
||||||
|
return o.toString();
|
||||||
|
} else if (o instanceof Date) {
|
||||||
|
return ((Date) o).getTime();
|
||||||
|
} else if (o instanceof Time) {
|
||||||
|
return ((Time) o).getTime();
|
||||||
|
} else if (o instanceof Timestamp) {
|
||||||
|
return ((Timestamp) o).getTime();
|
||||||
|
} else if (o instanceof BytesWritable) {
|
||||||
|
BytesWritable bw = (BytesWritable) o;
|
||||||
|
return ByteBuffer.wrap(bw.getBytes(), 0, bw.getLength());
|
||||||
|
} else if (o instanceof ClobRef) {
|
||||||
|
throw new UnsupportedOperationException("ClobRef not suported");
|
||||||
|
} else if (o instanceof BlobRef) {
|
||||||
|
throw new UnsupportedOperationException("BlobRef not suported");
|
||||||
|
}
|
||||||
|
// primitive types (Integer, etc) are left unchanged
|
||||||
|
return o;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
}
|
58
src/java/org/apache/sqoop/mapreduce/AvroInputFormat.java
Normal file
58
src/java/org/apache/sqoop/mapreduce/AvroInputFormat.java
Normal file
@ -0,0 +1,58 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import org.apache.avro.mapred.AvroWrapper;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
import org.apache.hadoop.io.NullWritable;
|
||||||
|
import org.apache.hadoop.mapreduce.InputSplit;
|
||||||
|
import org.apache.hadoop.mapreduce.JobContext;
|
||||||
|
import org.apache.hadoop.mapreduce.RecordReader;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
|
||||||
|
|
||||||
|
/** An {@link org.apache.hadoop.mapred.InputFormat} for Avro data files. */
|
||||||
|
public class AvroInputFormat<T>
|
||||||
|
extends FileInputFormat<AvroWrapper<T>, NullWritable> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected List<FileStatus> listStatus(JobContext job) throws IOException {
|
||||||
|
List<FileStatus> result = new ArrayList<FileStatus>();
|
||||||
|
for (FileStatus file : super.listStatus(job)) {
|
||||||
|
if (file.getPath().getName().endsWith(
|
||||||
|
org.apache.avro.mapred.AvroOutputFormat.EXT)) {
|
||||||
|
result.add(file);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RecordReader<AvroWrapper<T>, NullWritable> createRecordReader(
|
||||||
|
InputSplit split, TaskAttemptContext context) throws IOException,
|
||||||
|
InterruptedException {
|
||||||
|
context.setStatus(split.toString());
|
||||||
|
return new AvroRecordReader<T>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
41
src/java/org/apache/sqoop/mapreduce/AvroJob.java
Normal file
41
src/java/org/apache/sqoop/mapreduce/AvroJob.java
Normal file
@ -0,0 +1,41 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import org.apache.avro.Schema;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper class for setting up an Avro MapReduce job.
|
||||||
|
*/
|
||||||
|
public final class AvroJob {
|
||||||
|
public static final String MAP_OUTPUT_SCHEMA = "avro.map.output.schema";
|
||||||
|
|
||||||
|
private AvroJob() {
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void setMapOutputSchema(Configuration job, Schema s) {
|
||||||
|
job.set(MAP_OUTPUT_SCHEMA, s.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Return a job's map output key schema. */
|
||||||
|
public static Schema getMapOutputSchema(Configuration job) {
|
||||||
|
return Schema.parse(job.get(MAP_OUTPUT_SCHEMA));
|
||||||
|
}
|
||||||
|
}
|
64
src/java/org/apache/sqoop/mapreduce/AvroOutputFormat.java
Normal file
64
src/java/org/apache/sqoop/mapreduce/AvroOutputFormat.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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.avro.Schema;
|
||||||
|
import org.apache.avro.file.DataFileWriter;
|
||||||
|
import org.apache.avro.generic.GenericDatumWriter;
|
||||||
|
import org.apache.avro.mapred.AvroWrapper;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.io.NullWritable;
|
||||||
|
import org.apache.hadoop.mapreduce.RecordWriter;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
||||||
|
|
||||||
|
/** An {@link org.apache.hadoop.mapred.OutputFormat} for Avro data files. */
|
||||||
|
public class AvroOutputFormat<T>
|
||||||
|
extends FileOutputFormat<AvroWrapper<T>, NullWritable> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RecordWriter<AvroWrapper<T>, NullWritable> getRecordWriter(
|
||||||
|
TaskAttemptContext context) throws IOException, InterruptedException {
|
||||||
|
|
||||||
|
Schema schema = AvroJob.getMapOutputSchema(context.getConfiguration());
|
||||||
|
|
||||||
|
final DataFileWriter<T> WRITER =
|
||||||
|
new DataFileWriter<T>(new GenericDatumWriter<T>());
|
||||||
|
|
||||||
|
Path path = getDefaultWorkFile(context,
|
||||||
|
org.apache.avro.mapred.AvroOutputFormat.EXT);
|
||||||
|
WRITER.create(schema,
|
||||||
|
path.getFileSystem(context.getConfiguration()).create(path));
|
||||||
|
|
||||||
|
return new RecordWriter<AvroWrapper<T>, NullWritable>() {
|
||||||
|
@Override
|
||||||
|
public void write(AvroWrapper<T> wrapper, NullWritable ignore)
|
||||||
|
throws IOException {
|
||||||
|
WRITER.append(wrapper.datum());
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
public void close(TaskAttemptContext context) throws IOException,
|
||||||
|
InterruptedException {
|
||||||
|
WRITER.close();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
104
src/java/org/apache/sqoop/mapreduce/AvroRecordReader.java
Normal file
104
src/java/org/apache/sqoop/mapreduce/AvroRecordReader.java
Normal file
@ -0,0 +1,104 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.avro.file.DataFileReader;
|
||||||
|
import org.apache.avro.file.FileReader;
|
||||||
|
import org.apache.avro.file.SeekableInput;
|
||||||
|
import org.apache.avro.generic.GenericDatumReader;
|
||||||
|
import org.apache.avro.io.DatumReader;
|
||||||
|
import org.apache.avro.mapred.AvroWrapper;
|
||||||
|
import org.apache.avro.mapred.FsInput;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.io.NullWritable;
|
||||||
|
import org.apache.hadoop.mapreduce.InputSplit;
|
||||||
|
import org.apache.hadoop.mapreduce.RecordReader;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.input.FileSplit;
|
||||||
|
|
||||||
|
/** An {@link RecordReader} for Avro data files. */
|
||||||
|
public class AvroRecordReader<T>
|
||||||
|
extends RecordReader<AvroWrapper<T>, NullWritable> {
|
||||||
|
|
||||||
|
private FileReader<T> reader;
|
||||||
|
private long start;
|
||||||
|
private long end;
|
||||||
|
private AvroWrapper<T> key;
|
||||||
|
private NullWritable value;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void initialize(InputSplit genericSplit, TaskAttemptContext context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
FileSplit split = (FileSplit) genericSplit;
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
SeekableInput in = new FsInput(split.getPath(), conf);
|
||||||
|
DatumReader<T> datumReader = new GenericDatumReader<T>();
|
||||||
|
this.reader = DataFileReader.openReader(in, datumReader);
|
||||||
|
reader.sync(split.getStart()); // sync to start
|
||||||
|
this.start = reader.tell();
|
||||||
|
this.end = split.getStart() + split.getLength();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean nextKeyValue() throws IOException, InterruptedException {
|
||||||
|
if (!reader.hasNext() || reader.pastSync(end)) {
|
||||||
|
key = null;
|
||||||
|
value = null;
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (key == null) {
|
||||||
|
key = new AvroWrapper<T>();
|
||||||
|
}
|
||||||
|
if (value == null) {
|
||||||
|
value = NullWritable.get();
|
||||||
|
}
|
||||||
|
key.datum(reader.next(key.datum()));
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public AvroWrapper<T> getCurrentKey() throws IOException,
|
||||||
|
InterruptedException {
|
||||||
|
return key;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public NullWritable getCurrentValue()
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
return value;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public float getProgress() throws IOException {
|
||||||
|
if (end == start) {
|
||||||
|
return 0.0f;
|
||||||
|
} else {
|
||||||
|
return Math.min(1.0f, (getPos() - start) / (float)(end - start));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getPos() throws IOException {
|
||||||
|
return reader.tell();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException { reader.close(); }
|
||||||
|
}
|
||||||
|
|
131
src/java/org/apache/sqoop/mapreduce/CombineShimRecordReader.java
Normal file
131
src/java/org/apache/sqoop/mapreduce/CombineShimRecordReader.java
Normal file
@ -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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.io.LongWritable;
|
||||||
|
import org.apache.hadoop.mapreduce.InputSplit;
|
||||||
|
import org.apache.hadoop.mapreduce.RecordReader;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.input.FileSplit;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.input.LineRecordReader;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.input.SequenceFileRecordReader;
|
||||||
|
import org.apache.hadoop.util.ReflectionUtils;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* RecordReader that CombineFileRecordReader can instantiate, which itself
|
||||||
|
* translates a CombineFileSplit into a FileSplit.
|
||||||
|
*/
|
||||||
|
public class CombineShimRecordReader
|
||||||
|
extends RecordReader<LongWritable, Object> {
|
||||||
|
|
||||||
|
public static final Log LOG =
|
||||||
|
LogFactory.getLog(CombineShimRecordReader.class.getName());
|
||||||
|
|
||||||
|
private CombineFileSplit split;
|
||||||
|
private TaskAttemptContext context;
|
||||||
|
private int index;
|
||||||
|
private RecordReader<LongWritable, Object> rr;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructor invoked by CombineFileRecordReader that identifies part of a
|
||||||
|
* CombineFileSplit to use.
|
||||||
|
*/
|
||||||
|
public CombineShimRecordReader(CombineFileSplit split,
|
||||||
|
TaskAttemptContext context, Integer index)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
this.index = index;
|
||||||
|
this.split = (CombineFileSplit) split;
|
||||||
|
this.context = context;
|
||||||
|
|
||||||
|
createChildReader();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void initialize(InputSplit curSplit, TaskAttemptContext curContext)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
this.split = (CombineFileSplit) curSplit;
|
||||||
|
this.context = curContext;
|
||||||
|
|
||||||
|
if (null == rr) {
|
||||||
|
createChildReader();
|
||||||
|
}
|
||||||
|
|
||||||
|
FileSplit fileSplit = new FileSplit(this.split.getPath(index),
|
||||||
|
this.split.getOffset(index), this.split.getLength(index),
|
||||||
|
this.split.getLocations());
|
||||||
|
this.rr.initialize(fileSplit, this.context);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public float getProgress() throws IOException, InterruptedException {
|
||||||
|
return rr.getProgress();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException {
|
||||||
|
if (null != rr) {
|
||||||
|
rr.close();
|
||||||
|
rr = null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public LongWritable getCurrentKey()
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
return rr.getCurrentKey();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object getCurrentValue()
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
return rr.getCurrentValue();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean nextKeyValue() throws IOException, InterruptedException {
|
||||||
|
return rr.nextKeyValue();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Actually instantiate the user's chosen RecordReader implementation.
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
private void createChildReader() throws IOException, InterruptedException {
|
||||||
|
LOG.debug("ChildSplit operates on: " + split.getPath(index));
|
||||||
|
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
|
||||||
|
// Determine the file format we're reading.
|
||||||
|
Class rrClass;
|
||||||
|
if (ExportJobBase.isSequenceFiles(conf, split.getPath(index))) {
|
||||||
|
rrClass = SequenceFileRecordReader.class;
|
||||||
|
} else {
|
||||||
|
rrClass = LineRecordReader.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Create the appropriate record reader.
|
||||||
|
this.rr = (RecordReader<LongWritable, Object>)
|
||||||
|
ReflectionUtils.newInstance(rrClass, conf);
|
||||||
|
}
|
||||||
|
}
|
206
src/java/org/apache/sqoop/mapreduce/DataDrivenImportJob.java
Normal file
206
src/java/org/apache/sqoop/mapreduce/DataDrivenImportJob.java
Normal file
@ -0,0 +1,206 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import org.apache.avro.Schema;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.io.NullWritable;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.mapreduce.InputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
|
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.db.DBWritable;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
|
||||||
|
import com.cloudera.sqoop.SqoopOptions;
|
||||||
|
import com.cloudera.sqoop.config.ConfigurationHelper;
|
||||||
|
import com.cloudera.sqoop.lib.LargeObjectLoader;
|
||||||
|
import com.cloudera.sqoop.manager.ConnManager;
|
||||||
|
import com.cloudera.sqoop.manager.ImportJobContext;
|
||||||
|
import com.cloudera.sqoop.mapreduce.ImportJobBase;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DataDrivenDBInputFormat;
|
||||||
|
import com.cloudera.sqoop.orm.AvroSchemaGenerator;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Actually runs a jdbc import job using the ORM files generated by the
|
||||||
|
* sqoop.orm package. Uses DataDrivenDBInputFormat.
|
||||||
|
*/
|
||||||
|
public class DataDrivenImportJob extends ImportJobBase {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(
|
||||||
|
DataDrivenImportJob.class.getName());
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public DataDrivenImportJob(final SqoopOptions opts) {
|
||||||
|
super(opts, null, DataDrivenDBInputFormat.class, null, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public DataDrivenImportJob(final SqoopOptions opts,
|
||||||
|
final Class<? extends InputFormat> inputFormatClass,
|
||||||
|
ImportJobContext context) {
|
||||||
|
super(opts, null, inputFormatClass, null, context);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void configureMapper(Job job, String tableName,
|
||||||
|
String tableClassName) throws IOException {
|
||||||
|
if (options.getFileLayout() == SqoopOptions.FileLayout.TextFile) {
|
||||||
|
// For text files, specify these as the output types; for
|
||||||
|
// other types, we just use the defaults.
|
||||||
|
job.setOutputKeyClass(Text.class);
|
||||||
|
job.setOutputValueClass(NullWritable.class);
|
||||||
|
} else if (options.getFileLayout()
|
||||||
|
== SqoopOptions.FileLayout.AvroDataFile) {
|
||||||
|
ConnManager connManager = getContext().getConnManager();
|
||||||
|
AvroSchemaGenerator generator = new AvroSchemaGenerator(options,
|
||||||
|
connManager, tableName);
|
||||||
|
Schema schema = generator.generate();
|
||||||
|
AvroJob.setMapOutputSchema(job.getConfiguration(), schema);
|
||||||
|
}
|
||||||
|
|
||||||
|
job.setMapperClass(getMapperClass());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Class<? extends Mapper> getMapperClass() {
|
||||||
|
if (options.getFileLayout() == SqoopOptions.FileLayout.TextFile) {
|
||||||
|
return TextImportMapper.class;
|
||||||
|
} else if (options.getFileLayout()
|
||||||
|
== SqoopOptions.FileLayout.SequenceFile) {
|
||||||
|
return SequenceFileImportMapper.class;
|
||||||
|
} else if (options.getFileLayout()
|
||||||
|
== SqoopOptions.FileLayout.AvroDataFile) {
|
||||||
|
return AvroImportMapper.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Class<? extends OutputFormat> getOutputFormatClass()
|
||||||
|
throws ClassNotFoundException {
|
||||||
|
if (options.getFileLayout() == SqoopOptions.FileLayout.TextFile) {
|
||||||
|
return RawKeyTextOutputFormat.class;
|
||||||
|
} else if (options.getFileLayout()
|
||||||
|
== SqoopOptions.FileLayout.SequenceFile) {
|
||||||
|
return SequenceFileOutputFormat.class;
|
||||||
|
} else if (options.getFileLayout()
|
||||||
|
== SqoopOptions.FileLayout.AvroDataFile) {
|
||||||
|
return AvroOutputFormat.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void configureInputFormat(Job job, String tableName,
|
||||||
|
String tableClassName, String splitByCol) throws IOException {
|
||||||
|
ConnManager mgr = getContext().getConnManager();
|
||||||
|
try {
|
||||||
|
String username = options.getUsername();
|
||||||
|
if (null == username || username.length() == 0) {
|
||||||
|
DBConfiguration.configureDB(job.getConfiguration(),
|
||||||
|
mgr.getDriverClass(), options.getConnectString(),
|
||||||
|
options.getFetchSize());
|
||||||
|
} else {
|
||||||
|
DBConfiguration.configureDB(job.getConfiguration(),
|
||||||
|
mgr.getDriverClass(), options.getConnectString(),
|
||||||
|
username, options.getPassword(), options.getFetchSize());
|
||||||
|
}
|
||||||
|
|
||||||
|
if (null != tableName) {
|
||||||
|
// Import a table.
|
||||||
|
String [] colNames = options.getColumns();
|
||||||
|
if (null == colNames) {
|
||||||
|
colNames = mgr.getColumnNames(tableName);
|
||||||
|
}
|
||||||
|
|
||||||
|
String [] sqlColNames = null;
|
||||||
|
if (null != colNames) {
|
||||||
|
sqlColNames = new String[colNames.length];
|
||||||
|
for (int i = 0; i < colNames.length; i++) {
|
||||||
|
sqlColNames[i] = mgr.escapeColName(colNames[i]);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// It's ok if the where clause is null in DBInputFormat.setInput.
|
||||||
|
String whereClause = options.getWhereClause();
|
||||||
|
|
||||||
|
// We can't set the class properly in here, because we may not have the
|
||||||
|
// jar loaded in this JVM. So we start by calling setInput() with
|
||||||
|
// DBWritable and then overriding the string manually.
|
||||||
|
DataDrivenDBInputFormat.setInput(job, DBWritable.class,
|
||||||
|
mgr.escapeTableName(tableName), whereClause,
|
||||||
|
mgr.escapeColName(splitByCol), sqlColNames);
|
||||||
|
|
||||||
|
// If user specified boundary query on the command line propagate it to
|
||||||
|
// the job
|
||||||
|
if(options.getBoundaryQuery() != null) {
|
||||||
|
DataDrivenDBInputFormat.setBoundingQuery(job.getConfiguration(),
|
||||||
|
options.getBoundaryQuery());
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// Import a free-form query.
|
||||||
|
String inputQuery = options.getSqlQuery();
|
||||||
|
String sanitizedQuery = inputQuery.replace(
|
||||||
|
DataDrivenDBInputFormat.SUBSTITUTE_TOKEN, " (1 = 1) ");
|
||||||
|
|
||||||
|
String inputBoundingQuery = options.getBoundaryQuery();
|
||||||
|
|
||||||
|
if(inputBoundingQuery == null) {
|
||||||
|
inputBoundingQuery =
|
||||||
|
mgr.getInputBoundsQuery(splitByCol, sanitizedQuery);
|
||||||
|
if (inputBoundingQuery == null) {
|
||||||
|
if (splitByCol != null) {
|
||||||
|
inputBoundingQuery = "SELECT MIN(" + splitByCol + "), MAX("
|
||||||
|
+ splitByCol + ") FROM (" + sanitizedQuery + ") AS t1";
|
||||||
|
} else {
|
||||||
|
inputBoundingQuery = "";
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
DataDrivenDBInputFormat.setInput(job, DBWritable.class,
|
||||||
|
inputQuery, inputBoundingQuery);
|
||||||
|
new DBConfiguration(job.getConfiguration()).setInputOrderBy(
|
||||||
|
splitByCol);
|
||||||
|
}
|
||||||
|
|
||||||
|
LOG.debug("Using table class: " + tableClassName);
|
||||||
|
job.getConfiguration().set(ConfigurationHelper.getDbInputClassProperty(),
|
||||||
|
tableClassName);
|
||||||
|
|
||||||
|
job.getConfiguration().setLong(LargeObjectLoader.MAX_INLINE_LOB_LEN_KEY,
|
||||||
|
options.getInlineLobLimit());
|
||||||
|
|
||||||
|
LOG.debug("Using InputFormat: " + inputFormatClass);
|
||||||
|
job.setInputFormatClass(inputFormatClass);
|
||||||
|
} finally {
|
||||||
|
try {
|
||||||
|
mgr.close();
|
||||||
|
} catch (SQLException sqlE) {
|
||||||
|
LOG.warn("Error closing connection: " + sqlE);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
134
src/java/org/apache/sqoop/mapreduce/DelegatingOutputFormat.java
Normal file
134
src/java/org/apache/sqoop/mapreduce/DelegatingOutputFormat.java
Normal file
@ -0,0 +1,134 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.mapreduce.JobContext;
|
||||||
|
import org.apache.hadoop.mapreduce.OutputCommitter;
|
||||||
|
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.RecordWriter;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
import org.apache.hadoop.util.ReflectionUtils;
|
||||||
|
import com.cloudera.sqoop.lib.FieldMappable;
|
||||||
|
import com.cloudera.sqoop.lib.FieldMapProcessor;
|
||||||
|
import com.cloudera.sqoop.lib.ProcessingException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* OutputFormat that produces a RecordReader which instantiates
|
||||||
|
* a FieldMapProcessor which will process FieldMappable
|
||||||
|
* output keys.
|
||||||
|
*
|
||||||
|
* <p>The output value is ignored.</p>
|
||||||
|
*
|
||||||
|
* <p>The FieldMapProcessor implementation may do any arbitrary
|
||||||
|
* processing on the object. For example, it may write an object
|
||||||
|
* to HBase, etc.</p>
|
||||||
|
*
|
||||||
|
* <p>If the FieldMapProcessor implementation also implements
|
||||||
|
* Closeable, it will be close()'d in the RecordReader's close()
|
||||||
|
* method.</p>
|
||||||
|
*
|
||||||
|
* <p>If the FMP implements Configurable, it will be configured
|
||||||
|
* correctly via ReflectionUtils.</p>
|
||||||
|
*/
|
||||||
|
public class DelegatingOutputFormat<K extends FieldMappable, V>
|
||||||
|
extends OutputFormat<K, V> {
|
||||||
|
|
||||||
|
/** conf key: the FieldMapProcessor class to instantiate. */
|
||||||
|
public static final String DELEGATE_CLASS_KEY =
|
||||||
|
"sqoop.output.delegate.field.map.processor.class";
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public void checkOutputSpecs(JobContext context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
|
||||||
|
if (null == conf.get(DELEGATE_CLASS_KEY)) {
|
||||||
|
throw new IOException("Delegate FieldMapProcessor class is not set.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public OutputCommitter getOutputCommitter(TaskAttemptContext context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
return new NullOutputCommitter();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
||||||
|
throws IOException {
|
||||||
|
try {
|
||||||
|
return new DelegatingRecordWriter(context);
|
||||||
|
} catch (ClassNotFoundException cnfe) {
|
||||||
|
throw new IOException(cnfe);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* RecordWriter to write the output to a row in a database table.
|
||||||
|
* The actual database updates are executed in a second thread.
|
||||||
|
*/
|
||||||
|
public class DelegatingRecordWriter extends RecordWriter<K, V> {
|
||||||
|
|
||||||
|
private Configuration conf;
|
||||||
|
|
||||||
|
private FieldMapProcessor mapProcessor;
|
||||||
|
|
||||||
|
public DelegatingRecordWriter(TaskAttemptContext context)
|
||||||
|
throws ClassNotFoundException {
|
||||||
|
|
||||||
|
this.conf = context.getConfiguration();
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
Class<? extends FieldMapProcessor> procClass =
|
||||||
|
(Class<? extends FieldMapProcessor>)
|
||||||
|
conf.getClass(DELEGATE_CLASS_KEY, null);
|
||||||
|
this.mapProcessor = ReflectionUtils.newInstance(procClass, this.conf);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected Configuration getConf() {
|
||||||
|
return this.conf;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public void close(TaskAttemptContext context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
if (mapProcessor instanceof Closeable) {
|
||||||
|
((Closeable) mapProcessor).close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public void write(K key, V value)
|
||||||
|
throws InterruptedException, IOException {
|
||||||
|
try {
|
||||||
|
mapProcessor.accept(key);
|
||||||
|
} catch (ProcessingException pe) {
|
||||||
|
throw new IOException(pe);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
139
src/java/org/apache/sqoop/mapreduce/ExportBatchOutputFormat.java
Normal file
139
src/java/org/apache/sqoop/mapreduce/ExportBatchOutputFormat.java
Normal file
@ -0,0 +1,139 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.Connection;
|
||||||
|
import java.sql.PreparedStatement;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.util.List;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.mapreduce.RecordWriter;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.mapreduce.ExportOutputFormat;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This class uses batch mode to execute underlying statements instead of
|
||||||
|
* using a single multirow insert statement as its superclass.
|
||||||
|
*/
|
||||||
|
public class ExportBatchOutputFormat<K extends SqoopRecord, V>
|
||||||
|
extends ExportOutputFormat<K, V> {
|
||||||
|
|
||||||
|
private static final Log LOG =
|
||||||
|
LogFactory.getLog(ExportBatchOutputFormat.class);
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
||||||
|
throws IOException {
|
||||||
|
try {
|
||||||
|
return new ExportBatchRecordWriter(context);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new IOException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* RecordWriter to write the output to a row in a database table.
|
||||||
|
* The actual database updates are executed in a second thread.
|
||||||
|
*/
|
||||||
|
public class ExportBatchRecordWriter extends ExportRecordWriter {
|
||||||
|
|
||||||
|
public ExportBatchRecordWriter(TaskAttemptContext context)
|
||||||
|
throws ClassNotFoundException, SQLException {
|
||||||
|
super(context);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
protected boolean isBatchExec() {
|
||||||
|
// We use batches here.
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
protected PreparedStatement getPreparedStatement(
|
||||||
|
List<SqoopRecord> userRecords) throws SQLException {
|
||||||
|
|
||||||
|
PreparedStatement stmt = null;
|
||||||
|
|
||||||
|
// Synchronize on connection to ensure this does not conflict
|
||||||
|
// with the operations in the update thread.
|
||||||
|
Connection conn = getConnection();
|
||||||
|
synchronized (conn) {
|
||||||
|
stmt = conn.prepareStatement(getInsertStatement(userRecords.size()));
|
||||||
|
}
|
||||||
|
|
||||||
|
// Inject the record parameters into the VALUES clauses.
|
||||||
|
for (SqoopRecord record : userRecords) {
|
||||||
|
record.write(stmt, 0);
|
||||||
|
stmt.addBatch();
|
||||||
|
}
|
||||||
|
|
||||||
|
return stmt;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return an INSERT statement.
|
||||||
|
*/
|
||||||
|
protected String getInsertStatement(int numRows) {
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
|
||||||
|
sb.append("INSERT INTO " + tableName + " ");
|
||||||
|
|
||||||
|
int numSlots;
|
||||||
|
if (this.columnNames != null) {
|
||||||
|
numSlots = this.columnNames.length;
|
||||||
|
|
||||||
|
sb.append("(");
|
||||||
|
boolean first = true;
|
||||||
|
for (String col : columnNames) {
|
||||||
|
if (!first) {
|
||||||
|
sb.append(", ");
|
||||||
|
}
|
||||||
|
|
||||||
|
sb.append(col);
|
||||||
|
first = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
sb.append(") ");
|
||||||
|
} else {
|
||||||
|
numSlots = this.columnCount; // set if columnNames is null.
|
||||||
|
}
|
||||||
|
|
||||||
|
sb.append("VALUES ");
|
||||||
|
|
||||||
|
// generates the (?, ?, ?...).
|
||||||
|
sb.append("(");
|
||||||
|
for (int i = 0; i < numSlots; i++) {
|
||||||
|
if (i != 0) {
|
||||||
|
sb.append(", ");
|
||||||
|
}
|
||||||
|
|
||||||
|
sb.append("?");
|
||||||
|
}
|
||||||
|
sb.append(")");
|
||||||
|
|
||||||
|
return sb.toString();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
119
src/java/org/apache/sqoop/mapreduce/ExportInputFormat.java
Normal file
119
src/java/org/apache/sqoop/mapreduce/ExportInputFormat.java
Normal file
@ -0,0 +1,119 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
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 org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* InputFormat that generates a user-defined number of splits to inject data
|
||||||
|
* into the database.
|
||||||
|
*/
|
||||||
|
public class ExportInputFormat
|
||||||
|
extends CombineFileInputFormat<LongWritable, Object> {
|
||||||
|
|
||||||
|
public static final Log LOG =
|
||||||
|
LogFactory.getLog(ExportInputFormat.class.getName());
|
||||||
|
|
||||||
|
public static final int DEFAULT_NUM_MAP_TASKS = 4;
|
||||||
|
|
||||||
|
public ExportInputFormat() {
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the number of bytes across all files in the job.
|
||||||
|
*/
|
||||||
|
private long getJobSize(JobContext job) throws IOException {
|
||||||
|
List<FileStatus> stats = listStatus(job);
|
||||||
|
long count = 0;
|
||||||
|
for (FileStatus stat : stats) {
|
||||||
|
count += stat.getLen();
|
||||||
|
}
|
||||||
|
|
||||||
|
return count;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<InputSplit> getSplits(JobContext job) throws IOException {
|
||||||
|
// Set the max split size based on the number of map tasks we want.
|
||||||
|
long numTasks = getNumMapTasks(job);
|
||||||
|
long numFileBytes = getJobSize(job);
|
||||||
|
long maxSplitSize = numFileBytes / numTasks;
|
||||||
|
|
||||||
|
setMaxSplitSize(maxSplitSize);
|
||||||
|
|
||||||
|
LOG.debug("Target numMapTasks=" + numTasks);
|
||||||
|
LOG.debug("Total input bytes=" + numFileBytes);
|
||||||
|
LOG.debug("maxSplitSize=" + maxSplitSize);
|
||||||
|
|
||||||
|
List<InputSplit> splits = super.getSplits(job);
|
||||||
|
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Generated splits:");
|
||||||
|
for (InputSplit split : splits) {
|
||||||
|
LOG.debug(" " + split);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return splits;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public RecordReader createRecordReader(
|
||||||
|
InputSplit split, TaskAttemptContext context) throws IOException {
|
||||||
|
|
||||||
|
CombineFileSplit combineSplit = (CombineFileSplit) split;
|
||||||
|
|
||||||
|
// Use CombineFileRecordReader since this can handle CombineFileSplits
|
||||||
|
// and instantiate another RecordReader in a loop; do this with the
|
||||||
|
// CombineShimRecordReader.
|
||||||
|
RecordReader rr = new CombineFileRecordReader(combineSplit, context,
|
||||||
|
CombineShimRecordReader.class);
|
||||||
|
|
||||||
|
return rr;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Allows the user to control the number of map tasks used for this
|
||||||
|
* export job.
|
||||||
|
*/
|
||||||
|
public static void setNumMapTasks(JobContext job, int numTasks) {
|
||||||
|
job.getConfiguration().setInt(ExportJobBase.EXPORT_MAP_TASKS_KEY, numTasks);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the number of map tasks to use in this export job.
|
||||||
|
*/
|
||||||
|
public static int getNumMapTasks(JobContext job) {
|
||||||
|
return job.getConfiguration().getInt(ExportJobBase.EXPORT_MAP_TASKS_KEY,
|
||||||
|
DEFAULT_NUM_MAP_TASKS);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
410
src/java/org/apache/sqoop/mapreduce/ExportJobBase.java
Normal file
410
src/java/org/apache/sqoop/mapreduce/ExportJobBase.java
Normal file
@ -0,0 +1,410 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.FileNotFoundException;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.io.NullWritable;
|
||||||
|
import org.apache.hadoop.mapreduce.Counters;
|
||||||
|
import org.apache.hadoop.mapreduce.InputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
|
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
|
||||||
|
import org.apache.sqoop.util.PerfCounters;
|
||||||
|
import com.cloudera.sqoop.SqoopOptions;
|
||||||
|
import com.cloudera.sqoop.config.ConfigurationHelper;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.manager.ConnManager;
|
||||||
|
import com.cloudera.sqoop.manager.ExportJobContext;
|
||||||
|
import com.cloudera.sqoop.orm.TableClassName;
|
||||||
|
import com.cloudera.sqoop.mapreduce.JobBase;
|
||||||
|
import com.cloudera.sqoop.util.ExportException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Base class for running an export MapReduce job.
|
||||||
|
*/
|
||||||
|
public class ExportJobBase extends JobBase {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The (inferred) type of a file or group of files.
|
||||||
|
*/
|
||||||
|
public enum FileType {
|
||||||
|
SEQUENCE_FILE, AVRO_DATA_FILE, UNKNOWN
|
||||||
|
}
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(
|
||||||
|
ExportJobBase.class.getName());
|
||||||
|
|
||||||
|
/** What SqoopRecord class to use to read a record for export. */
|
||||||
|
public static final String SQOOP_EXPORT_TABLE_CLASS_KEY =
|
||||||
|
"sqoop.mapreduce.export.table.class";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* What column of the table to use for the WHERE clause of
|
||||||
|
* an updating export.
|
||||||
|
*/
|
||||||
|
public static final String SQOOP_EXPORT_UPDATE_COL_KEY =
|
||||||
|
"sqoop.mapreduce.export.update.col";
|
||||||
|
|
||||||
|
/** Number of map tasks to use for an export. */
|
||||||
|
public static final String EXPORT_MAP_TASKS_KEY =
|
||||||
|
"sqoop.mapreduce.export.map.tasks";
|
||||||
|
|
||||||
|
protected ExportJobContext context;
|
||||||
|
|
||||||
|
public ExportJobBase(final ExportJobContext ctxt) {
|
||||||
|
this(ctxt, null, null, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public ExportJobBase(final ExportJobContext ctxt,
|
||||||
|
final Class<? extends Mapper> mapperClass,
|
||||||
|
final Class<? extends InputFormat> inputFormatClass,
|
||||||
|
final Class<? extends OutputFormat> outputFormatClass) {
|
||||||
|
super(ctxt.getOptions(), mapperClass, inputFormatClass, outputFormatClass);
|
||||||
|
this.context = ctxt;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return true if p is a SequenceFile, or a directory containing
|
||||||
|
* SequenceFiles.
|
||||||
|
*/
|
||||||
|
public static boolean isSequenceFiles(Configuration conf, Path p)
|
||||||
|
throws IOException {
|
||||||
|
return getFileType(conf, p) == FileType.SEQUENCE_FILE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the type of the file represented by p (or the files in p, if a
|
||||||
|
* directory)
|
||||||
|
*/
|
||||||
|
public static FileType getFileType(Configuration conf, Path p)
|
||||||
|
throws IOException {
|
||||||
|
FileSystem fs = p.getFileSystem(conf);
|
||||||
|
|
||||||
|
try {
|
||||||
|
FileStatus stat = fs.getFileStatus(p);
|
||||||
|
|
||||||
|
if (null == stat) {
|
||||||
|
// Couldn't get the item.
|
||||||
|
LOG.warn("Input path " + p + " does not exist");
|
||||||
|
return FileType.UNKNOWN;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (stat.isDir()) {
|
||||||
|
FileStatus [] subitems = fs.listStatus(p);
|
||||||
|
if (subitems == null || subitems.length == 0) {
|
||||||
|
LOG.warn("Input path " + p + " contains no files");
|
||||||
|
return FileType.UNKNOWN; // empty dir.
|
||||||
|
}
|
||||||
|
|
||||||
|
// Pick a child entry to examine instead.
|
||||||
|
boolean foundChild = false;
|
||||||
|
for (int i = 0; i < subitems.length; i++) {
|
||||||
|
stat = subitems[i];
|
||||||
|
if (!stat.isDir() && !stat.getPath().getName().startsWith("_")) {
|
||||||
|
foundChild = true;
|
||||||
|
break; // This item is a visible file. Check it.
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!foundChild) {
|
||||||
|
stat = null; // Couldn't find a reasonable candidate.
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (null == stat) {
|
||||||
|
LOG.warn("null FileStatus object in isSequenceFiles(); "
|
||||||
|
+ "assuming false.");
|
||||||
|
return FileType.UNKNOWN;
|
||||||
|
}
|
||||||
|
|
||||||
|
Path target = stat.getPath();
|
||||||
|
return fromMagicNumber(target, conf);
|
||||||
|
} catch (FileNotFoundException fnfe) {
|
||||||
|
LOG.warn("Input path " + p + " does not exist");
|
||||||
|
return FileType.UNKNOWN; // doesn't exist!
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param file a file to test.
|
||||||
|
* @return true if 'file' refers to a SequenceFile.
|
||||||
|
*/
|
||||||
|
private static FileType fromMagicNumber(Path file, Configuration conf) {
|
||||||
|
// Test target's header to see if it contains magic numbers indicating its
|
||||||
|
// file type
|
||||||
|
byte [] header = new byte[3];
|
||||||
|
FSDataInputStream is = null;
|
||||||
|
try {
|
||||||
|
FileSystem fs = file.getFileSystem(conf);
|
||||||
|
is = fs.open(file);
|
||||||
|
is.readFully(header);
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
// Error reading header or EOF; assume unknown
|
||||||
|
LOG.warn("IOException checking input file header: " + ioe);
|
||||||
|
return FileType.UNKNOWN;
|
||||||
|
} finally {
|
||||||
|
try {
|
||||||
|
if (null != is) {
|
||||||
|
is.close();
|
||||||
|
}
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
// ignore; closing.
|
||||||
|
LOG.warn("IOException closing input stream: " + ioe + "; ignoring.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (header[0] == 'S' && header[1] == 'E' && header[2] == 'Q') {
|
||||||
|
return FileType.SEQUENCE_FILE;
|
||||||
|
}
|
||||||
|
if (header[0] == 'O' && header[1] == 'b' && header[2] == 'j') {
|
||||||
|
return FileType.AVRO_DATA_FILE;
|
||||||
|
}
|
||||||
|
return FileType.UNKNOWN;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the Path to the files we are going to export to the db.
|
||||||
|
*/
|
||||||
|
protected Path getInputPath() throws IOException {
|
||||||
|
Path inputPath = new Path(context.getOptions().getExportDir());
|
||||||
|
Configuration conf = options.getConf();
|
||||||
|
inputPath = inputPath.makeQualified(FileSystem.get(conf));
|
||||||
|
return inputPath;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void configureInputFormat(Job job, String tableName,
|
||||||
|
String tableClassName, String splitByCol)
|
||||||
|
throws ClassNotFoundException, IOException {
|
||||||
|
|
||||||
|
super.configureInputFormat(job, tableName, tableClassName, splitByCol);
|
||||||
|
FileInputFormat.addInputPath(job, getInputPath());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Class<? extends InputFormat> getInputFormatClass()
|
||||||
|
throws ClassNotFoundException {
|
||||||
|
Class<? extends InputFormat> configuredIF = super.getInputFormatClass();
|
||||||
|
if (null == configuredIF) {
|
||||||
|
return ExportInputFormat.class;
|
||||||
|
} else {
|
||||||
|
return configuredIF;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Class<? extends OutputFormat> getOutputFormatClass()
|
||||||
|
throws ClassNotFoundException {
|
||||||
|
Class<? extends OutputFormat> configuredOF = super.getOutputFormatClass();
|
||||||
|
if (null == configuredOF) {
|
||||||
|
if (!options.isBatchMode()) {
|
||||||
|
return ExportOutputFormat.class;
|
||||||
|
} else {
|
||||||
|
return ExportBatchOutputFormat.class;
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
return configuredOF;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void configureMapper(Job job, String tableName,
|
||||||
|
String tableClassName) throws ClassNotFoundException, IOException {
|
||||||
|
|
||||||
|
job.setMapperClass(getMapperClass());
|
||||||
|
|
||||||
|
// Concurrent writes of the same records would be problematic.
|
||||||
|
ConfigurationHelper.setJobMapSpeculativeExecution(job, false);
|
||||||
|
|
||||||
|
job.setMapOutputKeyClass(SqoopRecord.class);
|
||||||
|
job.setMapOutputValueClass(NullWritable.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected int configureNumTasks(Job job) throws IOException {
|
||||||
|
int numMaps = super.configureNumTasks(job);
|
||||||
|
job.getConfiguration().setInt(EXPORT_MAP_TASKS_KEY, numMaps);
|
||||||
|
return numMaps;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected boolean runJob(Job job) throws ClassNotFoundException, IOException,
|
||||||
|
InterruptedException {
|
||||||
|
|
||||||
|
PerfCounters perfCounters = new PerfCounters();
|
||||||
|
perfCounters.startClock();
|
||||||
|
|
||||||
|
boolean success = job.waitForCompletion(true);
|
||||||
|
perfCounters.stopClock();
|
||||||
|
|
||||||
|
Counters jobCounters = job.getCounters();
|
||||||
|
// If the job has been retired, these may be unavailable.
|
||||||
|
if (null == jobCounters) {
|
||||||
|
displayRetiredJobNotice(LOG);
|
||||||
|
} else {
|
||||||
|
perfCounters.addBytes(jobCounters.getGroup("FileSystemCounters")
|
||||||
|
.findCounter("HDFS_BYTES_READ").getValue());
|
||||||
|
LOG.info("Transferred " + perfCounters.toString());
|
||||||
|
long numRecords = ConfigurationHelper.getNumMapInputRecords(job);
|
||||||
|
LOG.info("Exported " + numRecords + " records.");
|
||||||
|
}
|
||||||
|
|
||||||
|
return success;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Run an export job to dump a table from HDFS to a database. If a staging
|
||||||
|
* table is specified and the connection manager supports staging of data,
|
||||||
|
* the export will first populate the staging table and then migrate the
|
||||||
|
* data to the target table.
|
||||||
|
* @throws IOException if the export job encounters an IO error
|
||||||
|
* @throws ExportException if the job fails unexpectedly or is misconfigured.
|
||||||
|
*/
|
||||||
|
public void runExport() throws ExportException, IOException {
|
||||||
|
|
||||||
|
ConnManager cmgr = context.getConnManager();
|
||||||
|
SqoopOptions options = context.getOptions();
|
||||||
|
Configuration conf = options.getConf();
|
||||||
|
|
||||||
|
String outputTableName = context.getTableName();
|
||||||
|
String stagingTableName = context.getOptions().getStagingTableName();
|
||||||
|
|
||||||
|
String tableName = outputTableName;
|
||||||
|
boolean stagingEnabled = false;
|
||||||
|
if (stagingTableName != null) { // user has specified the staging table
|
||||||
|
if (cmgr.supportsStagingForExport()) {
|
||||||
|
LOG.info("Data will be staged in the table: " + stagingTableName);
|
||||||
|
tableName = stagingTableName;
|
||||||
|
stagingEnabled = true;
|
||||||
|
} else {
|
||||||
|
throw new ExportException("The active connection manager ("
|
||||||
|
+ cmgr.getClass().getCanonicalName()
|
||||||
|
+ ") does not support staging of data for export. "
|
||||||
|
+ "Please retry without specifying the --staging-table option.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
String tableClassName =
|
||||||
|
new TableClassName(options).getClassForTable(outputTableName);
|
||||||
|
String ormJarFile = context.getJarFile();
|
||||||
|
|
||||||
|
LOG.info("Beginning export of " + outputTableName);
|
||||||
|
loadJars(conf, ormJarFile, tableClassName);
|
||||||
|
|
||||||
|
if (stagingEnabled) {
|
||||||
|
// Prepare the staging table
|
||||||
|
if (options.doClearStagingTable()) {
|
||||||
|
try {
|
||||||
|
// Delete all records from staging table
|
||||||
|
cmgr.deleteAllRecords(stagingTableName);
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
throw new ExportException(
|
||||||
|
"Failed to empty staging table before export run", ex);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// User has not explicitly specified the clear staging table option.
|
||||||
|
// Assert that the staging table is empty.
|
||||||
|
try {
|
||||||
|
long rowCount = cmgr.getTableRowCount(stagingTableName);
|
||||||
|
if (rowCount != 0L) {
|
||||||
|
throw new ExportException("The specified staging table ("
|
||||||
|
+ stagingTableName + ") is not empty. To force deletion of "
|
||||||
|
+ "its data, please retry with --clear-staging-table option.");
|
||||||
|
}
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
throw new ExportException(
|
||||||
|
"Failed to count data rows in staging table: "
|
||||||
|
+ stagingTableName, ex);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
Job job = new Job(conf);
|
||||||
|
|
||||||
|
// Set the external jar to use for the job.
|
||||||
|
job.getConfiguration().set("mapred.jar", ormJarFile);
|
||||||
|
|
||||||
|
configureInputFormat(job, tableName, tableClassName, null);
|
||||||
|
configureOutputFormat(job, tableName, tableClassName);
|
||||||
|
configureMapper(job, tableName, tableClassName);
|
||||||
|
configureNumTasks(job);
|
||||||
|
cacheJars(job, context.getConnManager());
|
||||||
|
setJob(job);
|
||||||
|
boolean success = runJob(job);
|
||||||
|
if (!success) {
|
||||||
|
throw new ExportException("Export job failed!");
|
||||||
|
}
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
throw new IOException(ie);
|
||||||
|
} catch (ClassNotFoundException cnfe) {
|
||||||
|
throw new IOException(cnfe);
|
||||||
|
} finally {
|
||||||
|
unloadJars();
|
||||||
|
}
|
||||||
|
|
||||||
|
// Unstage the data if needed
|
||||||
|
if (stagingEnabled) {
|
||||||
|
// Migrate data from staging table to the output table
|
||||||
|
try {
|
||||||
|
LOG.info("Starting to migrate data from staging table to destination.");
|
||||||
|
cmgr.migrateData(stagingTableName, outputTableName);
|
||||||
|
} catch (SQLException ex) {
|
||||||
|
LOG.error("Failed to move data from staging table ("
|
||||||
|
+ stagingTableName + ") to target table ("
|
||||||
|
+ outputTableName + ")", ex);
|
||||||
|
throw new ExportException(
|
||||||
|
"Failed to move data from staging table", ex);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return true if the input directory contains SequenceFiles.
|
||||||
|
* @deprecated use {@link #getInputFileType()} instead
|
||||||
|
*/
|
||||||
|
@Deprecated
|
||||||
|
protected boolean inputIsSequenceFiles() {
|
||||||
|
try {
|
||||||
|
return isSequenceFiles(
|
||||||
|
context.getOptions().getConf(), getInputPath());
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
LOG.warn("Could not check file format for export; assuming text");
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected FileType getInputFileType() {
|
||||||
|
try {
|
||||||
|
return getFileType(context.getOptions().getConf(), getInputPath());
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
return FileType.UNKNOWN;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
207
src/java/org/apache/sqoop/mapreduce/ExportOutputFormat.java
Normal file
207
src/java/org/apache/sqoop/mapreduce/ExportOutputFormat.java
Normal file
@ -0,0 +1,207 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.Connection;
|
||||||
|
import java.sql.PreparedStatement;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.List;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.mapreduce.JobContext;
|
||||||
|
import org.apache.hadoop.mapreduce.RecordWriter;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
||||||
|
import com.cloudera.sqoop.mapreduce.AsyncSqlOutputFormat;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Insert the emitted keys as records into a database table.
|
||||||
|
* This supports a configurable "spill threshold" at which
|
||||||
|
* point intermediate transactions are committed.
|
||||||
|
*
|
||||||
|
* Record objects are buffered before actually performing the INSERT
|
||||||
|
* statements; this requires that the key implement the
|
||||||
|
* SqoopRecord interface.
|
||||||
|
*
|
||||||
|
* Uses DBOutputFormat/DBConfiguration for configuring the output.
|
||||||
|
*/
|
||||||
|
public class ExportOutputFormat<K extends SqoopRecord, V>
|
||||||
|
extends AsyncSqlOutputFormat<K, V> {
|
||||||
|
|
||||||
|
private static final Log LOG = LogFactory.getLog(ExportOutputFormat.class);
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public void checkOutputSpecs(JobContext context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
DBConfiguration dbConf = new DBConfiguration(conf);
|
||||||
|
|
||||||
|
// Sanity check all the configuration values we need.
|
||||||
|
if (null == conf.get(DBConfiguration.URL_PROPERTY)) {
|
||||||
|
throw new IOException("Database connection URL is not set.");
|
||||||
|
} else if (null == dbConf.getOutputTableName()) {
|
||||||
|
throw new IOException("Table name is not set for export");
|
||||||
|
} else if (null == dbConf.getOutputFieldNames()
|
||||||
|
&& 0 == dbConf.getOutputFieldCount()) {
|
||||||
|
throw new IOException(
|
||||||
|
"Output field names are null and zero output field count set.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
||||||
|
throws IOException {
|
||||||
|
try {
|
||||||
|
return new ExportRecordWriter(context);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new IOException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* RecordWriter to write the output to a row in a database table.
|
||||||
|
* The actual database updates are executed in a second thread.
|
||||||
|
*/
|
||||||
|
public class ExportRecordWriter extends AsyncSqlRecordWriter<K, V> {
|
||||||
|
|
||||||
|
protected String tableName;
|
||||||
|
protected String [] columnNames; // The columns to insert into.
|
||||||
|
protected int columnCount; // If columnNames is null, tells ## of cols.
|
||||||
|
|
||||||
|
public ExportRecordWriter(TaskAttemptContext context)
|
||||||
|
throws ClassNotFoundException, SQLException {
|
||||||
|
super(context);
|
||||||
|
|
||||||
|
Configuration conf = getConf();
|
||||||
|
|
||||||
|
DBConfiguration dbConf = new DBConfiguration(conf);
|
||||||
|
tableName = dbConf.getOutputTableName();
|
||||||
|
columnNames = dbConf.getOutputFieldNames();
|
||||||
|
columnCount = dbConf.getOutputFieldCount();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the name of the table we are inserting into.
|
||||||
|
*/
|
||||||
|
protected final String getTableName() {
|
||||||
|
return tableName;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the list of columns we are updating.
|
||||||
|
*/
|
||||||
|
protected final String [] getColumnNames() {
|
||||||
|
if (null == columnNames) {
|
||||||
|
return null;
|
||||||
|
} else {
|
||||||
|
return Arrays.copyOf(columnNames, columnNames.length);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the number of columns we are updating.
|
||||||
|
*/
|
||||||
|
protected final int getColumnCount() {
|
||||||
|
return columnCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
protected PreparedStatement getPreparedStatement(
|
||||||
|
List<SqoopRecord> userRecords) throws SQLException {
|
||||||
|
|
||||||
|
PreparedStatement stmt = null;
|
||||||
|
|
||||||
|
// Synchronize on connection to ensure this does not conflict
|
||||||
|
// with the operations in the update thread.
|
||||||
|
Connection conn = getConnection();
|
||||||
|
synchronized (conn) {
|
||||||
|
stmt = conn.prepareStatement(getInsertStatement(userRecords.size()));
|
||||||
|
}
|
||||||
|
|
||||||
|
// Inject the record parameters into the VALUES clauses.
|
||||||
|
int position = 0;
|
||||||
|
for (SqoopRecord record : userRecords) {
|
||||||
|
position += record.write(stmt, position);
|
||||||
|
}
|
||||||
|
|
||||||
|
return stmt;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return an INSERT statement suitable for inserting 'numRows' rows.
|
||||||
|
*/
|
||||||
|
protected String getInsertStatement(int numRows) {
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
|
||||||
|
sb.append("INSERT INTO " + tableName + " ");
|
||||||
|
|
||||||
|
int numSlots;
|
||||||
|
if (this.columnNames != null) {
|
||||||
|
numSlots = this.columnNames.length;
|
||||||
|
|
||||||
|
sb.append("(");
|
||||||
|
boolean first = true;
|
||||||
|
for (String col : columnNames) {
|
||||||
|
if (!first) {
|
||||||
|
sb.append(", ");
|
||||||
|
}
|
||||||
|
|
||||||
|
sb.append(col);
|
||||||
|
first = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
sb.append(") ");
|
||||||
|
} else {
|
||||||
|
numSlots = this.columnCount; // set if columnNames is null.
|
||||||
|
}
|
||||||
|
|
||||||
|
sb.append("VALUES ");
|
||||||
|
|
||||||
|
// generates the (?, ?, ?...) used for each row.
|
||||||
|
StringBuilder sbRow = new StringBuilder();
|
||||||
|
sbRow.append("(");
|
||||||
|
for (int i = 0; i < numSlots; i++) {
|
||||||
|
if (i != 0) {
|
||||||
|
sbRow.append(", ");
|
||||||
|
}
|
||||||
|
|
||||||
|
sbRow.append("?");
|
||||||
|
}
|
||||||
|
sbRow.append(")");
|
||||||
|
|
||||||
|
// Now append that numRows times.
|
||||||
|
for (int i = 0; i < numRows; i++) {
|
||||||
|
if (i != 0) {
|
||||||
|
sb.append(", ");
|
||||||
|
}
|
||||||
|
|
||||||
|
sb.append(sbRow);
|
||||||
|
}
|
||||||
|
|
||||||
|
return sb.toString();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
178
src/java/org/apache/sqoop/mapreduce/HBaseImportJob.java
Normal file
178
src/java/org/apache/sqoop/mapreduce/HBaseImportJob.java
Normal file
@ -0,0 +1,178 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||||
|
import org.apache.hadoop.hbase.client.HTable;
|
||||||
|
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.io.NullWritable;
|
||||||
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
|
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||||
|
import com.cloudera.sqoop.SqoopOptions;
|
||||||
|
import com.cloudera.sqoop.hbase.HBasePutProcessor;
|
||||||
|
import com.cloudera.sqoop.lib.FieldMapProcessor;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.manager.ConnManager;
|
||||||
|
import com.cloudera.sqoop.manager.ImportJobContext;
|
||||||
|
import com.cloudera.sqoop.mapreduce.DataDrivenImportJob;
|
||||||
|
import com.cloudera.sqoop.util.ImportException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Runs an HBase import via DataDrivenDBInputFormat to the HBasePutProcessor
|
||||||
|
* in the DelegatingOutputFormat.
|
||||||
|
*/
|
||||||
|
public class HBaseImportJob extends DataDrivenImportJob {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(
|
||||||
|
HBaseImportJob.class.getName());
|
||||||
|
|
||||||
|
public HBaseImportJob(final SqoopOptions opts,
|
||||||
|
final ImportJobContext importContext) {
|
||||||
|
super(opts, importContext.getInputFormat(), importContext);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void configureMapper(Job job, String tableName,
|
||||||
|
String tableClassName) throws IOException {
|
||||||
|
job.setOutputKeyClass(SqoopRecord.class);
|
||||||
|
job.setOutputValueClass(NullWritable.class);
|
||||||
|
job.setMapperClass(getMapperClass());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Class<? extends Mapper> getMapperClass() {
|
||||||
|
return HBaseImportMapper.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Class<? extends OutputFormat> getOutputFormatClass()
|
||||||
|
throws ClassNotFoundException {
|
||||||
|
return DelegatingOutputFormat.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void configureOutputFormat(Job job, String tableName,
|
||||||
|
String tableClassName) throws ClassNotFoundException, IOException {
|
||||||
|
|
||||||
|
// Use the DelegatingOutputFormat with the HBasePutProcessor.
|
||||||
|
job.setOutputFormatClass(getOutputFormatClass());
|
||||||
|
|
||||||
|
Configuration conf = job.getConfiguration();
|
||||||
|
conf.setClass("sqoop.output.delegate.field.map.processor.class",
|
||||||
|
HBasePutProcessor.class,
|
||||||
|
FieldMapProcessor.class);
|
||||||
|
|
||||||
|
// Set the HBase parameters (table, column family, row key):
|
||||||
|
conf.set(HBasePutProcessor.TABLE_NAME_KEY, options.getHBaseTable());
|
||||||
|
conf.set(HBasePutProcessor.COL_FAMILY_KEY, options.getHBaseColFamily());
|
||||||
|
|
||||||
|
// What column of the input becomes the row key?
|
||||||
|
String rowKeyCol = options.getHBaseRowKeyColumn();
|
||||||
|
if (null == rowKeyCol) {
|
||||||
|
// User didn't explicitly set one. If there's a split-by column set,
|
||||||
|
// use that.
|
||||||
|
rowKeyCol = options.getSplitByCol();
|
||||||
|
}
|
||||||
|
|
||||||
|
if (null == rowKeyCol) {
|
||||||
|
// No split-by column is explicitly set.
|
||||||
|
// If the table has a primary key, use that.
|
||||||
|
ConnManager manager = getContext().getConnManager();
|
||||||
|
rowKeyCol = manager.getPrimaryKey(tableName);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (null == rowKeyCol) {
|
||||||
|
// Give up here if this is still unset.
|
||||||
|
throw new IOException("Could not determine the row-key column. "
|
||||||
|
+ "Use --hbase-row-key to specify the input column that "
|
||||||
|
+ "names each row.");
|
||||||
|
}
|
||||||
|
|
||||||
|
conf.set(HBasePutProcessor.ROW_KEY_COLUMN_KEY, rowKeyCol);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** Create the target HBase table before running the job. */
|
||||||
|
protected void jobSetup(Job job) throws IOException, ImportException {
|
||||||
|
Configuration conf = job.getConfiguration();
|
||||||
|
String tableName = conf.get(HBasePutProcessor.TABLE_NAME_KEY);
|
||||||
|
String familyName = conf.get(HBasePutProcessor.COL_FAMILY_KEY);
|
||||||
|
|
||||||
|
if (null == tableName) {
|
||||||
|
throw new ImportException(
|
||||||
|
"Import to HBase error: Table name not specified");
|
||||||
|
}
|
||||||
|
|
||||||
|
if (null == familyName) {
|
||||||
|
throw new ImportException(
|
||||||
|
"Import to HBase error: Column family not specified");
|
||||||
|
}
|
||||||
|
|
||||||
|
// Add HBase configuration files to this conf object.
|
||||||
|
HBaseConfiguration.addHbaseResources(conf);
|
||||||
|
|
||||||
|
HBaseAdmin admin = new HBaseAdmin(conf);
|
||||||
|
|
||||||
|
// Check to see if the table exists.
|
||||||
|
HTableDescriptor tableDesc = new HTableDescriptor(tableName);
|
||||||
|
byte [] familyBytes = Bytes.toBytes(familyName);
|
||||||
|
HColumnDescriptor colDesc = new HColumnDescriptor(familyBytes);
|
||||||
|
if (!admin.tableExists(tableName)) {
|
||||||
|
if (options.getCreateHBaseTable()) {
|
||||||
|
// Create the table.
|
||||||
|
LOG.info("Creating missing HBase table " + tableName);
|
||||||
|
tableDesc.addFamily(colDesc);
|
||||||
|
admin.createTable(tableDesc);
|
||||||
|
} else {
|
||||||
|
LOG.warn("Could not find HBase table " + tableName);
|
||||||
|
LOG.warn("This job may fail. Either explicitly create the table,");
|
||||||
|
LOG.warn("or re-run with --hbase-create-table.");
|
||||||
|
}
|
||||||
|
} else if (!tableDesc.hasFamily(familyBytes)) {
|
||||||
|
if (options.getCreateHBaseTable()) {
|
||||||
|
// Create the column family.
|
||||||
|
LOG.info("Creating missing column family " + familyName);
|
||||||
|
admin.disableTable(tableName);
|
||||||
|
admin.addColumn(tableName, colDesc);
|
||||||
|
admin.enableTable(tableName);
|
||||||
|
} else {
|
||||||
|
LOG.warn("Could not find column family " + familyName + " in table "
|
||||||
|
+ tableName);
|
||||||
|
LOG.warn("This job may fail. Either create the column family,");
|
||||||
|
LOG.warn("or re-run with --hbase-create-table.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Make sure HBase libraries are shipped as part of the job.
|
||||||
|
TableMapReduceUtil.addDependencyJars(job);
|
||||||
|
TableMapReduceUtil.addDependencyJars(conf, HTable.class);
|
||||||
|
|
||||||
|
super.jobSetup(job);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
41
src/java/org/apache/sqoop/mapreduce/HBaseImportMapper.java
Normal file
41
src/java/org/apache/sqoop/mapreduce/HBaseImportMapper.java
Normal file
@ -0,0 +1,41 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.hadoop.io.LongWritable;
|
||||||
|
import org.apache.hadoop.io.NullWritable;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.mapreduce.AutoProgressMapper;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Imports records by writing them to HBase via the DelegatingOutputFormat
|
||||||
|
* and the HBasePutProcessor.
|
||||||
|
*/
|
||||||
|
public class HBaseImportMapper
|
||||||
|
extends AutoProgressMapper
|
||||||
|
<LongWritable, SqoopRecord, SqoopRecord, NullWritable> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void map(LongWritable key, SqoopRecord val, Context context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
context.write(val, NullWritable.get());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
203
src/java/org/apache/sqoop/mapreduce/ImportJobBase.java
Normal file
203
src/java/org/apache/sqoop/mapreduce/ImportJobBase.java
Normal file
@ -0,0 +1,203 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.io.SequenceFile.CompressionType;
|
||||||
|
import org.apache.hadoop.io.compress.CompressionCodec;
|
||||||
|
import org.apache.hadoop.io.compress.GzipCodec;
|
||||||
|
import org.apache.hadoop.mapreduce.Counters;
|
||||||
|
import org.apache.hadoop.mapreduce.InputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
|
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
|
||||||
|
import org.apache.sqoop.util.PerfCounters;
|
||||||
|
import com.cloudera.sqoop.SqoopOptions;
|
||||||
|
import com.cloudera.sqoop.config.ConfigurationHelper;
|
||||||
|
import com.cloudera.sqoop.io.CodecMap;
|
||||||
|
import com.cloudera.sqoop.manager.ImportJobContext;
|
||||||
|
import com.cloudera.sqoop.mapreduce.JobBase;
|
||||||
|
import com.cloudera.sqoop.orm.TableClassName;
|
||||||
|
import com.cloudera.sqoop.util.ImportException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Base class for running an import MapReduce job.
|
||||||
|
* Allows dependency injection, etc, for easy customization of import job types.
|
||||||
|
*/
|
||||||
|
public class ImportJobBase extends JobBase {
|
||||||
|
|
||||||
|
private ImportJobContext context;
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(
|
||||||
|
ImportJobBase.class.getName());
|
||||||
|
|
||||||
|
public ImportJobBase() {
|
||||||
|
this(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public ImportJobBase(final SqoopOptions opts) {
|
||||||
|
this(opts, null, null, null, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public ImportJobBase(final SqoopOptions opts,
|
||||||
|
final Class<? extends Mapper> mapperClass,
|
||||||
|
final Class<? extends InputFormat> inputFormatClass,
|
||||||
|
final Class<? extends OutputFormat> outputFormatClass,
|
||||||
|
final ImportJobContext context) {
|
||||||
|
super(opts, mapperClass, inputFormatClass, outputFormatClass);
|
||||||
|
this.context = context;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Configure the output format to use for the job.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
protected void configureOutputFormat(Job job, String tableName,
|
||||||
|
String tableClassName) throws ClassNotFoundException, IOException {
|
||||||
|
|
||||||
|
job.setOutputFormatClass(getOutputFormatClass());
|
||||||
|
|
||||||
|
if (options.getFileLayout() == SqoopOptions.FileLayout.SequenceFile) {
|
||||||
|
job.getConfiguration().set("mapred.output.value.class", tableClassName);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (options.shouldUseCompression()) {
|
||||||
|
FileOutputFormat.setCompressOutput(job, true);
|
||||||
|
|
||||||
|
String codecName = options.getCompressionCodec();
|
||||||
|
Class<? extends CompressionCodec> codecClass;
|
||||||
|
if (codecName == null) {
|
||||||
|
codecClass = GzipCodec.class;
|
||||||
|
} else {
|
||||||
|
Configuration conf = job.getConfiguration();
|
||||||
|
codecClass = CodecMap.getCodec(codecName, conf).getClass();
|
||||||
|
}
|
||||||
|
FileOutputFormat.setOutputCompressorClass(job, codecClass);
|
||||||
|
|
||||||
|
if (options.getFileLayout() == SqoopOptions.FileLayout.SequenceFile) {
|
||||||
|
SequenceFileOutputFormat.setOutputCompressionType(job,
|
||||||
|
CompressionType.BLOCK);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
Path outputPath = context.getDestination();
|
||||||
|
FileOutputFormat.setOutputPath(job, outputPath);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Actually run the MapReduce job.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
protected boolean runJob(Job job) throws ClassNotFoundException, IOException,
|
||||||
|
InterruptedException {
|
||||||
|
|
||||||
|
PerfCounters perfCounters = new PerfCounters();
|
||||||
|
perfCounters.startClock();
|
||||||
|
|
||||||
|
boolean success = job.waitForCompletion(true);
|
||||||
|
perfCounters.stopClock();
|
||||||
|
|
||||||
|
Counters jobCounters = job.getCounters();
|
||||||
|
// If the job has been retired, these may be unavailable.
|
||||||
|
if (null == jobCounters) {
|
||||||
|
displayRetiredJobNotice(LOG);
|
||||||
|
} else {
|
||||||
|
perfCounters.addBytes(jobCounters.getGroup("FileSystemCounters")
|
||||||
|
.findCounter("HDFS_BYTES_WRITTEN").getValue());
|
||||||
|
LOG.info("Transferred " + perfCounters.toString());
|
||||||
|
long numRecords = ConfigurationHelper.getNumMapOutputRecords(job);
|
||||||
|
LOG.info("Retrieved " + numRecords + " records.");
|
||||||
|
}
|
||||||
|
return success;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Run an import job to read a table in to HDFS.
|
||||||
|
*
|
||||||
|
* @param tableName the database table to read; may be null if a free-form
|
||||||
|
* query is specified in the SqoopOptions, and the ImportJobBase subclass
|
||||||
|
* supports free-form queries.
|
||||||
|
* @param ormJarFile the Jar file to insert into the dcache classpath.
|
||||||
|
* (may be null)
|
||||||
|
* @param splitByCol the column of the database table to use to split
|
||||||
|
* the import
|
||||||
|
* @param conf A fresh Hadoop Configuration to use to build an MR job.
|
||||||
|
* @throws IOException if the job encountered an IO problem
|
||||||
|
* @throws ImportException if the job failed unexpectedly or was
|
||||||
|
* misconfigured.
|
||||||
|
*/
|
||||||
|
public void runImport(String tableName, String ormJarFile, String splitByCol,
|
||||||
|
Configuration conf) throws IOException, ImportException {
|
||||||
|
|
||||||
|
if (null != tableName) {
|
||||||
|
LOG.info("Beginning import of " + tableName);
|
||||||
|
} else {
|
||||||
|
LOG.info("Beginning query import.");
|
||||||
|
}
|
||||||
|
|
||||||
|
String tableClassName =
|
||||||
|
new TableClassName(options).getClassForTable(tableName);
|
||||||
|
loadJars(conf, ormJarFile, tableClassName);
|
||||||
|
|
||||||
|
try {
|
||||||
|
Job job = new Job(conf);
|
||||||
|
|
||||||
|
// Set the external jar to use for the job.
|
||||||
|
job.getConfiguration().set("mapred.jar", ormJarFile);
|
||||||
|
|
||||||
|
configureInputFormat(job, tableName, tableClassName, splitByCol);
|
||||||
|
configureOutputFormat(job, tableName, tableClassName);
|
||||||
|
configureMapper(job, tableName, tableClassName);
|
||||||
|
configureNumTasks(job);
|
||||||
|
cacheJars(job, getContext().getConnManager());
|
||||||
|
|
||||||
|
jobSetup(job);
|
||||||
|
setJob(job);
|
||||||
|
boolean success = runJob(job);
|
||||||
|
if (!success) {
|
||||||
|
throw new ImportException("Import job failed!");
|
||||||
|
}
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
throw new IOException(ie);
|
||||||
|
} catch (ClassNotFoundException cnfe) {
|
||||||
|
throw new IOException(cnfe);
|
||||||
|
} finally {
|
||||||
|
unloadJars();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Open-ended "setup" routine that is called after the job is configured
|
||||||
|
* but just before it is submitted to MapReduce. Subclasses may override
|
||||||
|
* if necessary.
|
||||||
|
*/
|
||||||
|
protected void jobSetup(Job job) throws IOException, ImportException {
|
||||||
|
}
|
||||||
|
|
||||||
|
protected ImportJobContext getContext() {
|
||||||
|
return context;
|
||||||
|
}
|
||||||
|
}
|
139
src/java/org/apache/sqoop/mapreduce/JdbcExportJob.java
Normal file
139
src/java/org/apache/sqoop/mapreduce/JdbcExportJob.java
Normal file
@ -0,0 +1,139 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Map;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.io.DefaultStringifier;
|
||||||
|
import org.apache.hadoop.io.MapWritable;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.mapreduce.InputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
|
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||||
|
import com.cloudera.sqoop.manager.ConnManager;
|
||||||
|
import com.cloudera.sqoop.manager.ExportJobContext;
|
||||||
|
import com.cloudera.sqoop.mapreduce.ExportJobBase;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBOutputFormat;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Run an export using JDBC (JDBC-based ExportOutputFormat).
|
||||||
|
*/
|
||||||
|
public class JdbcExportJob extends ExportJobBase {
|
||||||
|
|
||||||
|
private FileType fileType;
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(
|
||||||
|
JdbcExportJob.class.getName());
|
||||||
|
|
||||||
|
public JdbcExportJob(final ExportJobContext context) {
|
||||||
|
super(context);
|
||||||
|
}
|
||||||
|
|
||||||
|
public JdbcExportJob(final ExportJobContext ctxt,
|
||||||
|
final Class<? extends Mapper> mapperClass,
|
||||||
|
final Class<? extends InputFormat> inputFormatClass,
|
||||||
|
final Class<? extends OutputFormat> outputFormatClass) {
|
||||||
|
super(ctxt, mapperClass, inputFormatClass, outputFormatClass);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void configureInputFormat(Job job, String tableName,
|
||||||
|
String tableClassName, String splitByCol)
|
||||||
|
throws ClassNotFoundException, IOException {
|
||||||
|
|
||||||
|
fileType = getInputFileType();
|
||||||
|
|
||||||
|
super.configureInputFormat(job, tableName, tableClassName, splitByCol);
|
||||||
|
|
||||||
|
if (fileType == FileType.AVRO_DATA_FILE) {
|
||||||
|
LOG.debug("Configuring for Avro export");
|
||||||
|
ConnManager connManager = context.getConnManager();
|
||||||
|
Map<String, Integer> columnTypeInts =
|
||||||
|
connManager.getColumnTypes(tableName, options.getSqlQuery());
|
||||||
|
MapWritable columnTypes = new MapWritable();
|
||||||
|
for (Map.Entry<String, Integer> e : columnTypeInts.entrySet()) {
|
||||||
|
Text columnName = new Text(e.getKey());
|
||||||
|
Text columnText = new Text(connManager.toJavaType(e.getValue()));
|
||||||
|
columnTypes.put(columnName, columnText);
|
||||||
|
}
|
||||||
|
DefaultStringifier.store(job.getConfiguration(), columnTypes,
|
||||||
|
AvroExportMapper.AVRO_COLUMN_TYPES_MAP);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Class<? extends InputFormat> getInputFormatClass()
|
||||||
|
throws ClassNotFoundException {
|
||||||
|
if (fileType == FileType.AVRO_DATA_FILE) {
|
||||||
|
return AvroInputFormat.class;
|
||||||
|
}
|
||||||
|
return super.getInputFormatClass();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Class<? extends Mapper> getMapperClass() {
|
||||||
|
switch (fileType) {
|
||||||
|
case SEQUENCE_FILE:
|
||||||
|
return SequenceFileExportMapper.class;
|
||||||
|
case AVRO_DATA_FILE:
|
||||||
|
return AvroExportMapper.class;
|
||||||
|
case UNKNOWN:
|
||||||
|
default:
|
||||||
|
return TextExportMapper.class;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void configureOutputFormat(Job job, String tableName,
|
||||||
|
String tableClassName) throws IOException {
|
||||||
|
|
||||||
|
ConnManager mgr = context.getConnManager();
|
||||||
|
try {
|
||||||
|
String username = options.getUsername();
|
||||||
|
if (null == username || username.length() == 0) {
|
||||||
|
DBConfiguration.configureDB(job.getConfiguration(),
|
||||||
|
mgr.getDriverClass(),
|
||||||
|
options.getConnectString());
|
||||||
|
} else {
|
||||||
|
DBConfiguration.configureDB(job.getConfiguration(),
|
||||||
|
mgr.getDriverClass(),
|
||||||
|
options.getConnectString(),
|
||||||
|
username, options.getPassword());
|
||||||
|
}
|
||||||
|
|
||||||
|
String [] colNames = options.getColumns();
|
||||||
|
if (null == colNames) {
|
||||||
|
colNames = mgr.getColumnNames(tableName);
|
||||||
|
}
|
||||||
|
DBOutputFormat.setOutput(job, tableName, colNames);
|
||||||
|
|
||||||
|
job.setOutputFormatClass(getOutputFormatClass());
|
||||||
|
job.getConfiguration().set(SQOOP_EXPORT_TABLE_CLASS_KEY, tableClassName);
|
||||||
|
} catch (ClassNotFoundException cnfe) {
|
||||||
|
throw new IOException("Could not load OutputFormat", cnfe);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
145
src/java/org/apache/sqoop/mapreduce/JdbcUpdateExportJob.java
Normal file
145
src/java/org/apache/sqoop/mapreduce/JdbcUpdateExportJob.java
Normal file
@ -0,0 +1,145 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.LinkedHashSet;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.StringTokenizer;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.mapreduce.InputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
|
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||||
|
import com.cloudera.sqoop.manager.ConnManager;
|
||||||
|
import com.cloudera.sqoop.manager.ExportJobContext;
|
||||||
|
import com.cloudera.sqoop.mapreduce.ExportJobBase;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBOutputFormat;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Run an update-based export using JDBC (JDBC-based UpdateOutputFormat).
|
||||||
|
*/
|
||||||
|
public class JdbcUpdateExportJob extends ExportJobBase {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(
|
||||||
|
JdbcUpdateExportJob.class.getName());
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return an instance of the UpdateOutputFormat class object loaded
|
||||||
|
* from the shim jar.
|
||||||
|
*/
|
||||||
|
private static Class<? extends OutputFormat> getUpdateOutputFormat()
|
||||||
|
throws IOException {
|
||||||
|
return UpdateOutputFormat.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
public JdbcUpdateExportJob(final ExportJobContext context)
|
||||||
|
throws IOException {
|
||||||
|
super(context, null, null, getUpdateOutputFormat());
|
||||||
|
}
|
||||||
|
|
||||||
|
public JdbcUpdateExportJob(final ExportJobContext ctxt,
|
||||||
|
final Class<? extends Mapper> mapperClass,
|
||||||
|
final Class<? extends InputFormat> inputFormatClass,
|
||||||
|
final Class<? extends OutputFormat> outputFormatClass) {
|
||||||
|
super(ctxt, mapperClass, inputFormatClass, outputFormatClass);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Class<? extends Mapper> getMapperClass() {
|
||||||
|
if (inputIsSequenceFiles()) {
|
||||||
|
return SequenceFileExportMapper.class;
|
||||||
|
} else {
|
||||||
|
return TextExportMapper.class;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void configureOutputFormat(Job job, String tableName,
|
||||||
|
String tableClassName) throws IOException {
|
||||||
|
|
||||||
|
ConnManager mgr = context.getConnManager();
|
||||||
|
try {
|
||||||
|
String username = options.getUsername();
|
||||||
|
if (null == username || username.length() == 0) {
|
||||||
|
DBConfiguration.configureDB(job.getConfiguration(),
|
||||||
|
mgr.getDriverClass(),
|
||||||
|
options.getConnectString());
|
||||||
|
} else {
|
||||||
|
DBConfiguration.configureDB(job.getConfiguration(),
|
||||||
|
mgr.getDriverClass(),
|
||||||
|
options.getConnectString(),
|
||||||
|
username, options.getPassword());
|
||||||
|
}
|
||||||
|
|
||||||
|
String [] colNames = options.getColumns();
|
||||||
|
if (null == colNames) {
|
||||||
|
colNames = mgr.getColumnNames(tableName);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (null == colNames) {
|
||||||
|
throw new IOException(
|
||||||
|
"Export column names could not be determined for " + tableName);
|
||||||
|
}
|
||||||
|
|
||||||
|
String updateKeyColumns = options.getUpdateKeyCol();
|
||||||
|
if (null == updateKeyColumns) {
|
||||||
|
throw new IOException("Update key column not set in export job");
|
||||||
|
}
|
||||||
|
// Update key columns lookup and removal
|
||||||
|
Set<String> updateKeys = new LinkedHashSet<String>();
|
||||||
|
Set<String> updateKeysUppercase = new HashSet<String>();
|
||||||
|
StringTokenizer stok = new StringTokenizer(updateKeyColumns, ",");
|
||||||
|
while (stok.hasMoreTokens()) {
|
||||||
|
String nextUpdateKey = stok.nextToken().trim();
|
||||||
|
if (nextUpdateKey.length() > 0) {
|
||||||
|
updateKeys.add(nextUpdateKey);
|
||||||
|
updateKeysUppercase.add(nextUpdateKey.toUpperCase());
|
||||||
|
} else {
|
||||||
|
throw new RuntimeException("Invalid update key column value specified"
|
||||||
|
+ ": '" + updateKeyColumns + "'");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (updateKeys.size() == 0) {
|
||||||
|
throw new IOException("Unpdate key columns not valid in export job");
|
||||||
|
}
|
||||||
|
|
||||||
|
// Make sure we strip out the key column from this list.
|
||||||
|
String [] outColNames = new String[colNames.length - updateKeys.size()];
|
||||||
|
int j = 0;
|
||||||
|
for (int i = 0; i < colNames.length; i++) {
|
||||||
|
if (!updateKeysUppercase.contains(colNames[i].toUpperCase())) {
|
||||||
|
outColNames[j++] = colNames[i];
|
||||||
|
}
|
||||||
|
}
|
||||||
|
DBOutputFormat.setOutput(job, tableName, outColNames);
|
||||||
|
|
||||||
|
job.setOutputFormatClass(getOutputFormatClass());
|
||||||
|
job.getConfiguration().set(SQOOP_EXPORT_TABLE_CLASS_KEY, tableClassName);
|
||||||
|
job.getConfiguration().set(SQOOP_EXPORT_UPDATE_COL_KEY, updateKeyColumns);
|
||||||
|
} catch (ClassNotFoundException cnfe) {
|
||||||
|
throw new IOException("Could not load OutputFormat", cnfe);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
106
src/java/org/apache/sqoop/mapreduce/JdbcUpsertExportJob.java
Normal file
106
src/java/org/apache/sqoop/mapreduce/JdbcUpsertExportJob.java
Normal file
@ -0,0 +1,106 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.LinkedHashSet;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.StringTokenizer;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
|
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||||
|
import com.cloudera.sqoop.manager.ConnManager;
|
||||||
|
import com.cloudera.sqoop.manager.ExportJobContext;
|
||||||
|
import com.cloudera.sqoop.mapreduce.JdbcUpdateExportJob;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBOutputFormat;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Run an update/insert export using JDBC (JDBC-based UpsertOutputFormat).
|
||||||
|
*/
|
||||||
|
public class JdbcUpsertExportJob extends JdbcUpdateExportJob {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(
|
||||||
|
JdbcUpsertExportJob.class.getName());
|
||||||
|
|
||||||
|
public JdbcUpsertExportJob(final ExportJobContext context,
|
||||||
|
final Class<? extends OutputFormat> outputFormatClass)
|
||||||
|
throws IOException {
|
||||||
|
super(context, null, null, outputFormatClass);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void configureOutputFormat(Job job, String tableName,
|
||||||
|
String tableClassName) throws IOException {
|
||||||
|
|
||||||
|
ConnManager mgr = context.getConnManager();
|
||||||
|
try {
|
||||||
|
String username = options.getUsername();
|
||||||
|
if (null == username || username.length() == 0) {
|
||||||
|
DBConfiguration.configureDB(job.getConfiguration(),
|
||||||
|
mgr.getDriverClass(),
|
||||||
|
options.getConnectString());
|
||||||
|
} else {
|
||||||
|
DBConfiguration.configureDB(job.getConfiguration(),
|
||||||
|
mgr.getDriverClass(),
|
||||||
|
options.getConnectString(),
|
||||||
|
username, options.getPassword());
|
||||||
|
}
|
||||||
|
|
||||||
|
String [] colNames = options.getColumns();
|
||||||
|
if (null == colNames) {
|
||||||
|
colNames = mgr.getColumnNames(tableName);
|
||||||
|
}
|
||||||
|
if (null == colNames) {
|
||||||
|
throw new IOException(
|
||||||
|
"Export column names could not be determined for " + tableName);
|
||||||
|
}
|
||||||
|
DBOutputFormat.setOutput(job, tableName, colNames);
|
||||||
|
|
||||||
|
String updateKeyColumns = options.getUpdateKeyCol();
|
||||||
|
if (null == updateKeyColumns) {
|
||||||
|
throw new IOException("Update key column not set in export job");
|
||||||
|
}
|
||||||
|
// Update key columns lookup and removal
|
||||||
|
Set<String> updateKeys = new LinkedHashSet<String>();
|
||||||
|
StringTokenizer stok = new StringTokenizer(updateKeyColumns, ",");
|
||||||
|
while (stok.hasMoreTokens()) {
|
||||||
|
String nextUpdateKey = stok.nextToken().trim();
|
||||||
|
if (nextUpdateKey.length() > 0) {
|
||||||
|
updateKeys.add(nextUpdateKey);
|
||||||
|
} else {
|
||||||
|
throw new RuntimeException("Invalid update key column value specified"
|
||||||
|
+ ": '" + updateKeyColumns + "'");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (updateKeys.size() == 0) {
|
||||||
|
throw new IOException("Unpdate key columns not valid in export job");
|
||||||
|
}
|
||||||
|
|
||||||
|
job.setOutputFormatClass(getOutputFormatClass());
|
||||||
|
job.getConfiguration().set(SQOOP_EXPORT_TABLE_CLASS_KEY, tableClassName);
|
||||||
|
job.getConfiguration().set(SQOOP_EXPORT_UPDATE_COL_KEY, updateKeyColumns);
|
||||||
|
} catch (ClassNotFoundException cnfe) {
|
||||||
|
throw new IOException("Could not load OutputFormat", cnfe);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
325
src/java/org/apache/sqoop/mapreduce/JobBase.java
Normal file
325
src/java/org/apache/sqoop/mapreduce/JobBase.java
Normal file
@ -0,0 +1,325 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.mapreduce.InputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
|
import org.apache.hadoop.mapreduce.OutputFormat;
|
||||||
|
import org.apache.hadoop.util.StringUtils;
|
||||||
|
import com.cloudera.sqoop.SqoopOptions;
|
||||||
|
import com.cloudera.sqoop.config.ConfigurationHelper;
|
||||||
|
import com.cloudera.sqoop.manager.ConnManager;
|
||||||
|
import com.cloudera.sqoop.tool.SqoopTool;
|
||||||
|
import com.cloudera.sqoop.util.ClassLoaderStack;
|
||||||
|
import com.cloudera.sqoop.util.Jars;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Base class for configuring and running a MapReduce job.
|
||||||
|
* Allows dependency injection, etc, for easy customization of import job types.
|
||||||
|
*/
|
||||||
|
public class JobBase {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(JobBase.class.getName());
|
||||||
|
|
||||||
|
protected SqoopOptions options;
|
||||||
|
protected Class<? extends Mapper> mapperClass;
|
||||||
|
protected Class<? extends InputFormat> inputFormatClass;
|
||||||
|
protected Class<? extends OutputFormat> outputFormatClass;
|
||||||
|
|
||||||
|
private Job mrJob;
|
||||||
|
|
||||||
|
private ClassLoader prevClassLoader = null;
|
||||||
|
|
||||||
|
public JobBase() {
|
||||||
|
this(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public JobBase(final SqoopOptions opts) {
|
||||||
|
this(opts, null, null, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public JobBase(final SqoopOptions opts,
|
||||||
|
final Class<? extends Mapper> mapperClass,
|
||||||
|
final Class<? extends InputFormat> inputFormatClass,
|
||||||
|
final Class<? extends OutputFormat> outputFormatClass) {
|
||||||
|
|
||||||
|
this.options = opts;
|
||||||
|
this.mapperClass = mapperClass;
|
||||||
|
this.inputFormatClass = inputFormatClass;
|
||||||
|
this.outputFormatClass = outputFormatClass;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the mapper class to use for the job.
|
||||||
|
*/
|
||||||
|
protected Class<? extends Mapper> getMapperClass()
|
||||||
|
throws ClassNotFoundException {
|
||||||
|
return this.mapperClass;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the inputformat class to use for the job.
|
||||||
|
*/
|
||||||
|
protected Class<? extends InputFormat> getInputFormatClass()
|
||||||
|
throws ClassNotFoundException {
|
||||||
|
return this.inputFormatClass;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the outputformat class to use for the job.
|
||||||
|
*/
|
||||||
|
protected Class<? extends OutputFormat> getOutputFormatClass()
|
||||||
|
throws ClassNotFoundException {
|
||||||
|
return this.outputFormatClass;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Set the OutputFormat class to use for this job. */
|
||||||
|
public void setOutputFormatClass(Class<? extends OutputFormat> cls) {
|
||||||
|
this.outputFormatClass = cls;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Set the InputFormat class to use for this job. */
|
||||||
|
public void setInputFormatClass(Class<? extends InputFormat> cls) {
|
||||||
|
this.inputFormatClass = cls;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Set the Mapper class to use for this job. */
|
||||||
|
public void setMapperClass(Class<? extends Mapper> cls) {
|
||||||
|
this.mapperClass = cls;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set the SqoopOptions configuring this job.
|
||||||
|
*/
|
||||||
|
public void setOptions(SqoopOptions opts) {
|
||||||
|
this.options = opts;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Put jar files required by Sqoop into the DistributedCache.
|
||||||
|
* @param job the Job being submitted.
|
||||||
|
* @param mgr the ConnManager to use.
|
||||||
|
*/
|
||||||
|
protected void cacheJars(Job job, ConnManager mgr)
|
||||||
|
throws IOException {
|
||||||
|
|
||||||
|
Configuration conf = job.getConfiguration();
|
||||||
|
FileSystem fs = FileSystem.getLocal(conf);
|
||||||
|
Set<String> localUrls = new HashSet<String>();
|
||||||
|
|
||||||
|
addToCache(Jars.getSqoopJarPath(), fs, localUrls);
|
||||||
|
if (null != mgr) {
|
||||||
|
addToCache(Jars.getDriverClassJar(mgr), fs, localUrls);
|
||||||
|
addToCache(Jars.getJarPathForClass(mgr.getClass()), fs, localUrls);
|
||||||
|
}
|
||||||
|
|
||||||
|
SqoopTool tool = this.options.getActiveSqoopTool();
|
||||||
|
if (null != tool) {
|
||||||
|
// Make sure the jar for the tool itself is on the classpath. (In case
|
||||||
|
// this is a third-party plugin tool.)
|
||||||
|
addToCache(Jars.getJarPathForClass(tool.getClass()), fs, localUrls);
|
||||||
|
List<String> toolDeps = tool.getDependencyJars();
|
||||||
|
if (null != toolDeps) {
|
||||||
|
for (String depFile : toolDeps) {
|
||||||
|
addToCache(depFile, fs, localUrls);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// If the user specified a particular jar file name,
|
||||||
|
|
||||||
|
// Add anything in $SQOOP_HOME/lib, if this is set.
|
||||||
|
String sqoopHome = System.getenv("SQOOP_HOME");
|
||||||
|
if (null != sqoopHome) {
|
||||||
|
File sqoopHomeFile = new File(sqoopHome);
|
||||||
|
File sqoopLibFile = new File(sqoopHomeFile, "lib");
|
||||||
|
if (sqoopLibFile.exists()) {
|
||||||
|
addDirToCache(sqoopLibFile, fs, localUrls);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
LOG.warn("SQOOP_HOME is unset. May not be able to find "
|
||||||
|
+ "all job dependencies.");
|
||||||
|
}
|
||||||
|
|
||||||
|
// If we didn't put anything in our set, then there's nothing to cache.
|
||||||
|
if (localUrls.isEmpty()) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Add these to the 'tmpjars' array, which the MR JobSubmitter
|
||||||
|
// will upload to HDFS and put in the DistributedCache libjars.
|
||||||
|
String tmpjars = conf.get("tmpjars");
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
if (null != tmpjars) {
|
||||||
|
sb.append(tmpjars);
|
||||||
|
sb.append(",");
|
||||||
|
}
|
||||||
|
sb.append(StringUtils.arrayToString(localUrls.toArray(new String[0])));
|
||||||
|
conf.set("tmpjars", sb.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
private void addToCache(String file, FileSystem fs, Set<String> localUrls) {
|
||||||
|
if (null == file) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
Path p = new Path(file);
|
||||||
|
String qualified = p.makeQualified(fs).toString();
|
||||||
|
LOG.debug("Adding to job classpath: " + qualified);
|
||||||
|
localUrls.add(qualified);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add the .jar elements of a directory to the DCache classpath,
|
||||||
|
* nonrecursively.
|
||||||
|
*/
|
||||||
|
private void addDirToCache(File dir, FileSystem fs, Set<String> localUrls) {
|
||||||
|
if (null == dir) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
for (File libfile : dir.listFiles()) {
|
||||||
|
if (libfile.exists() && !libfile.isDirectory()
|
||||||
|
&& libfile.getName().endsWith("jar")) {
|
||||||
|
addToCache(libfile.toString(), fs, localUrls);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* If jars must be loaded into the local environment, do so here.
|
||||||
|
*/
|
||||||
|
protected void loadJars(Configuration conf, String ormJarFile,
|
||||||
|
String tableClassName) throws IOException {
|
||||||
|
boolean isLocal = "local".equals(conf.get("mapreduce.jobtracker.address"))
|
||||||
|
|| "local".equals(conf.get("mapred.job.tracker"));
|
||||||
|
if (isLocal) {
|
||||||
|
// If we're using the LocalJobRunner, then instead of using the compiled
|
||||||
|
// jar file as the job source, we're running in the current thread. Push
|
||||||
|
// on another classloader that loads from that jar in addition to
|
||||||
|
// everything currently on the classpath.
|
||||||
|
this.prevClassLoader = ClassLoaderStack.addJarFile(ormJarFile,
|
||||||
|
tableClassName);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* If any classloader was invoked by loadJars, free it here.
|
||||||
|
*/
|
||||||
|
protected void unloadJars() {
|
||||||
|
if (null != this.prevClassLoader) {
|
||||||
|
// unload the special classloader for this jar.
|
||||||
|
ClassLoaderStack.setCurrentClassLoader(this.prevClassLoader);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Configure the inputformat to use for the job.
|
||||||
|
*/
|
||||||
|
protected void configureInputFormat(Job job, String tableName,
|
||||||
|
String tableClassName, String splitByCol)
|
||||||
|
throws ClassNotFoundException, IOException {
|
||||||
|
//TODO: 'splitByCol' is import-job specific; lift it out of this API.
|
||||||
|
Class<? extends InputFormat> ifClass = getInputFormatClass();
|
||||||
|
LOG.debug("Using InputFormat: " + ifClass);
|
||||||
|
job.setInputFormatClass(ifClass);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Configure the output format to use for the job.
|
||||||
|
*/
|
||||||
|
protected void configureOutputFormat(Job job, String tableName,
|
||||||
|
String tableClassName) throws ClassNotFoundException, IOException {
|
||||||
|
Class<? extends OutputFormat> ofClass = getOutputFormatClass();
|
||||||
|
LOG.debug("Using OutputFormat: " + ofClass);
|
||||||
|
job.setOutputFormatClass(ofClass);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set the mapper class implementation to use in the job,
|
||||||
|
* as well as any related configuration (e.g., map output types).
|
||||||
|
*/
|
||||||
|
protected void configureMapper(Job job, String tableName,
|
||||||
|
String tableClassName) throws ClassNotFoundException, IOException {
|
||||||
|
job.setMapperClass(getMapperClass());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Configure the number of map/reduce tasks to use in the job.
|
||||||
|
*/
|
||||||
|
protected int configureNumTasks(Job job) throws IOException {
|
||||||
|
int numMapTasks = options.getNumMappers();
|
||||||
|
if (numMapTasks < 1) {
|
||||||
|
numMapTasks = SqoopOptions.DEFAULT_NUM_MAPPERS;
|
||||||
|
LOG.warn("Invalid mapper count; using " + numMapTasks + " mappers.");
|
||||||
|
}
|
||||||
|
|
||||||
|
ConfigurationHelper.setJobNumMaps(job, numMapTasks);
|
||||||
|
job.setNumReduceTasks(0);
|
||||||
|
return numMapTasks;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Set the main job that will be run. */
|
||||||
|
protected void setJob(Job job) {
|
||||||
|
mrJob = job;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the main MapReduce job that is being run, or null if no
|
||||||
|
* job has started.
|
||||||
|
*/
|
||||||
|
public Job getJob() {
|
||||||
|
return mrJob;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Actually run the MapReduce job.
|
||||||
|
*/
|
||||||
|
protected boolean runJob(Job job) throws ClassNotFoundException, IOException,
|
||||||
|
InterruptedException {
|
||||||
|
return job.waitForCompletion(true);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Display a notice on the log that the current MapReduce job has
|
||||||
|
* been retired, and thus Counters are unavailable.
|
||||||
|
* @param log the Log to display the info to.
|
||||||
|
*/
|
||||||
|
protected void displayRetiredJobNotice(Log log) {
|
||||||
|
log.info("The MapReduce job has already been retired. Performance");
|
||||||
|
log.info("counters are unavailable. To get this information, ");
|
||||||
|
log.info("you will need to enable the completed job store on ");
|
||||||
|
log.info("the jobtracker with:");
|
||||||
|
log.info("mapreduce.jobtracker.persist.jobstatus.active = true");
|
||||||
|
log.info("mapreduce.jobtracker.persist.jobstatus.hours = 1");
|
||||||
|
log.info("A jobtracker restart is required for these settings");
|
||||||
|
log.info("to take effect.");
|
||||||
|
}
|
||||||
|
}
|
145
src/java/org/apache/sqoop/mapreduce/MergeJob.java
Normal file
145
src/java/org/apache/sqoop/mapreduce/MergeJob.java
Normal file
@ -0,0 +1,145 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.io.NullWritable;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
|
||||||
|
import org.apache.sqoop.util.Jars;
|
||||||
|
import com.cloudera.sqoop.SqoopOptions;
|
||||||
|
import com.cloudera.sqoop.mapreduce.JobBase;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Run a MapReduce job that merges two datasets.
|
||||||
|
*/
|
||||||
|
public class MergeJob extends JobBase {
|
||||||
|
|
||||||
|
/** Configuration key specifying the path to the "old" dataset. */
|
||||||
|
public static final String MERGE_OLD_PATH_KEY = "sqoop.merge.old.path";
|
||||||
|
|
||||||
|
/** Configuration key specifying the path to the "new" dataset. */
|
||||||
|
public static final String MERGE_NEW_PATH_KEY = "sqoop.merge.new.path";
|
||||||
|
|
||||||
|
/** Configuration key specifying the name of the key column for joins. */
|
||||||
|
public static final String MERGE_KEY_COL_KEY = "sqoop.merge.key.col";
|
||||||
|
|
||||||
|
/** Configuration key specifying the SqoopRecord class name for
|
||||||
|
* the records we are merging.
|
||||||
|
*/
|
||||||
|
public static final String MERGE_SQOOP_RECORD_KEY = "sqoop.merge.class";
|
||||||
|
|
||||||
|
public MergeJob(final SqoopOptions opts) {
|
||||||
|
super(opts, null, null, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean runMergeJob() throws IOException {
|
||||||
|
Configuration conf = options.getConf();
|
||||||
|
Job job = new Job(conf);
|
||||||
|
|
||||||
|
String userClassName = options.getClassName();
|
||||||
|
if (null == userClassName) {
|
||||||
|
// Shouldn't get here.
|
||||||
|
throw new IOException("Record class name not specified with "
|
||||||
|
+ "--class-name.");
|
||||||
|
}
|
||||||
|
|
||||||
|
// Set the external jar to use for the job.
|
||||||
|
String existingJar = options.getExistingJarName();
|
||||||
|
if (existingJar != null) {
|
||||||
|
// User explicitly identified a jar path.
|
||||||
|
LOG.debug("Setting job jar to user-specified jar: " + existingJar);
|
||||||
|
job.getConfiguration().set("mapred.jar", existingJar);
|
||||||
|
} else {
|
||||||
|
// Infer it from the location of the specified class, if it's on the
|
||||||
|
// classpath.
|
||||||
|
try {
|
||||||
|
Class<? extends Object> userClass = conf.getClassByName(userClassName);
|
||||||
|
if (null != userClass) {
|
||||||
|
String userJar = Jars.getJarPathForClass(userClass);
|
||||||
|
LOG.debug("Setting job jar based on user class " + userClassName
|
||||||
|
+ ": " + userJar);
|
||||||
|
job.getConfiguration().set("mapred.jar", userJar);
|
||||||
|
} else {
|
||||||
|
LOG.warn("Specified class " + userClassName + " is not in a jar. "
|
||||||
|
+ "MapReduce may not find the class");
|
||||||
|
}
|
||||||
|
} catch (ClassNotFoundException cnfe) {
|
||||||
|
throw new IOException(cnfe);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
Path oldPath = new Path(options.getMergeOldPath());
|
||||||
|
Path newPath = new Path(options.getMergeNewPath());
|
||||||
|
|
||||||
|
Configuration jobConf = job.getConfiguration();
|
||||||
|
FileSystem fs = FileSystem.get(jobConf);
|
||||||
|
oldPath = oldPath.makeQualified(fs);
|
||||||
|
newPath = newPath.makeQualified(fs);
|
||||||
|
|
||||||
|
FileInputFormat.addInputPath(job, oldPath);
|
||||||
|
FileInputFormat.addInputPath(job, newPath);
|
||||||
|
|
||||||
|
jobConf.set(MERGE_OLD_PATH_KEY, oldPath.toString());
|
||||||
|
jobConf.set(MERGE_NEW_PATH_KEY, newPath.toString());
|
||||||
|
jobConf.set(MERGE_KEY_COL_KEY, options.getMergeKeyCol());
|
||||||
|
jobConf.set(MERGE_SQOOP_RECORD_KEY, userClassName);
|
||||||
|
|
||||||
|
FileOutputFormat.setOutputPath(job, new Path(options.getTargetDir()));
|
||||||
|
|
||||||
|
if (ExportJobBase.isSequenceFiles(jobConf, newPath)) {
|
||||||
|
job.setInputFormatClass(SequenceFileInputFormat.class);
|
||||||
|
job.setOutputFormatClass(SequenceFileOutputFormat.class);
|
||||||
|
job.setMapperClass(MergeRecordMapper.class);
|
||||||
|
} else {
|
||||||
|
job.setMapperClass(MergeTextMapper.class);
|
||||||
|
job.setOutputFormatClass(RawKeyTextOutputFormat.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
jobConf.set("mapred.output.key.class", userClassName);
|
||||||
|
job.setOutputValueClass(NullWritable.class);
|
||||||
|
|
||||||
|
job.setReducerClass(MergeReducer.class);
|
||||||
|
|
||||||
|
// Set the intermediate data types.
|
||||||
|
job.setMapOutputKeyClass(Text.class);
|
||||||
|
job.setMapOutputValueClass(MergeRecord.class);
|
||||||
|
|
||||||
|
// Make sure Sqoop and anything else we need is on the classpath.
|
||||||
|
cacheJars(job, null);
|
||||||
|
setJob(job);
|
||||||
|
return this.runJob(job);
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
throw new IOException(ie);
|
||||||
|
} catch (ClassNotFoundException cnfe) {
|
||||||
|
throw new IOException(cnfe);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
85
src/java/org/apache/sqoop/mapreduce/MergeMapperBase.java
Normal file
85
src/java/org/apache/sqoop/mapreduce/MergeMapperBase.java
Normal file
@ -0,0 +1,85 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Map;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.mapreduce.InputSplit;
|
||||||
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.input.FileSplit;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Given a set of SqoopRecord instances which are from a "new" dataset
|
||||||
|
* or an "old" dataset, extract a key column from the record and tag
|
||||||
|
* each record with a bit specifying whether it is a new or old record.
|
||||||
|
*/
|
||||||
|
public class MergeMapperBase<INKEY, INVAL>
|
||||||
|
extends Mapper<INKEY, INVAL, Text, MergeRecord> {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(
|
||||||
|
MergeMapperBase.class.getName());
|
||||||
|
|
||||||
|
private String keyColName; // name of the key column.
|
||||||
|
private boolean isNew; // true if this split is from the new dataset.
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void setup(Context context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
keyColName = conf.get(MergeJob.MERGE_KEY_COL_KEY);
|
||||||
|
|
||||||
|
InputSplit is = context.getInputSplit();
|
||||||
|
FileSplit fs = (FileSplit) is;
|
||||||
|
Path splitPath = fs.getPath();
|
||||||
|
|
||||||
|
if (splitPath.toString().startsWith(
|
||||||
|
conf.get(MergeJob.MERGE_NEW_PATH_KEY))) {
|
||||||
|
this.isNew = true;
|
||||||
|
} else if (splitPath.toString().startsWith(
|
||||||
|
conf.get(MergeJob.MERGE_OLD_PATH_KEY))) {
|
||||||
|
this.isNew = false;
|
||||||
|
} else {
|
||||||
|
throw new IOException("File " + splitPath + " is not under new path "
|
||||||
|
+ conf.get(MergeJob.MERGE_NEW_PATH_KEY) + " or old path "
|
||||||
|
+ conf.get(MergeJob.MERGE_OLD_PATH_KEY));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void processRecord(SqoopRecord r, Context c)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
MergeRecord mr = new MergeRecord(r, isNew);
|
||||||
|
Map<String, Object> fieldMap = r.getFieldMap();
|
||||||
|
if (null == fieldMap) {
|
||||||
|
throw new IOException("No field map in record " + r);
|
||||||
|
}
|
||||||
|
Object keyObj = fieldMap.get(keyColName);
|
||||||
|
if (null == keyObj) {
|
||||||
|
throw new IOException("Cannot join values on null key. "
|
||||||
|
+ "Did you specify a key column that exists?");
|
||||||
|
} else {
|
||||||
|
c.write(new Text(keyObj.toString()), mr);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
131
src/java/org/apache/sqoop/mapreduce/MergeRecord.java
Normal file
131
src/java/org/apache/sqoop/mapreduce/MergeRecord.java
Normal file
@ -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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.hadoop.conf.Configurable;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.io.Writable;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Class that holds a record to be merged. This contains a SqoopRecord which
|
||||||
|
* is the "guts" of the item, and a boolean value indicating whether it is a
|
||||||
|
* "new" record or an "old" record. In the Reducer, we prefer to emit a new
|
||||||
|
* record rather than an old one, if a new one is available.
|
||||||
|
*/
|
||||||
|
public class MergeRecord implements Configurable, Writable {
|
||||||
|
private SqoopRecord sqoopRecord;
|
||||||
|
private boolean isNew;
|
||||||
|
private Configuration config;
|
||||||
|
|
||||||
|
/** Construct an empty MergeRecord. */
|
||||||
|
public MergeRecord() {
|
||||||
|
this.sqoopRecord = null;
|
||||||
|
this.isNew = false;
|
||||||
|
this.config = new Configuration();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Construct a MergeRecord with all fields initialized.
|
||||||
|
*/
|
||||||
|
public MergeRecord(SqoopRecord sr, boolean recordIsNew) {
|
||||||
|
this.sqoopRecord = sr;
|
||||||
|
this.isNew = recordIsNew;
|
||||||
|
this.config = new Configuration();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public void setConf(Configuration conf) {
|
||||||
|
this.config = conf;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public Configuration getConf() {
|
||||||
|
return this.config;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** @return true if this record came from the "new" dataset. */
|
||||||
|
public boolean isNewRecord() {
|
||||||
|
return isNew;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set the isNew field to 'newVal'.
|
||||||
|
*/
|
||||||
|
public void setNewRecord(boolean newVal) {
|
||||||
|
this.isNew = newVal;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the underlying SqoopRecord we're shipping.
|
||||||
|
*/
|
||||||
|
public SqoopRecord getSqoopRecord() {
|
||||||
|
return this.sqoopRecord;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set the SqoopRecord instance we should pass from the mapper to the
|
||||||
|
* reducer.
|
||||||
|
*/
|
||||||
|
public void setSqoopRecord(SqoopRecord record) {
|
||||||
|
this.sqoopRecord = record;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/**
|
||||||
|
* {@inheritDoc}
|
||||||
|
*/
|
||||||
|
public void readFields(DataInput in) throws IOException {
|
||||||
|
this.isNew = in.readBoolean();
|
||||||
|
String className = Text.readString(in);
|
||||||
|
if (null == this.sqoopRecord) {
|
||||||
|
// If we haven't already instantiated an inner SqoopRecord, do so here.
|
||||||
|
try {
|
||||||
|
Class<? extends SqoopRecord> recordClass =
|
||||||
|
(Class<? extends SqoopRecord>) config.getClassByName(className);
|
||||||
|
this.sqoopRecord = recordClass.newInstance();
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new IOException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
this.sqoopRecord.readFields(in);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/**
|
||||||
|
* {@inheritDoc}
|
||||||
|
*/
|
||||||
|
public void write(DataOutput out) throws IOException {
|
||||||
|
out.writeBoolean(this.isNew);
|
||||||
|
Text.writeString(out, this.sqoopRecord.getClass().getName());
|
||||||
|
this.sqoopRecord.write(out);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "" + this.sqoopRecord;
|
||||||
|
}
|
||||||
|
}
|
37
src/java/org/apache/sqoop/mapreduce/MergeRecordMapper.java
Normal file
37
src/java/org/apache/sqoop/mapreduce/MergeRecordMapper.java
Normal file
@ -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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.hadoop.io.LongWritable;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.mapreduce.MergeMapperBase;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mapper for the merge program which operates on SequenceFiles.
|
||||||
|
*/
|
||||||
|
public class MergeRecordMapper
|
||||||
|
extends MergeMapperBase<LongWritable, SqoopRecord> {
|
||||||
|
|
||||||
|
public void map(LongWritable key, SqoopRecord val, Context c)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
processRecord(val, c);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
56
src/java/org/apache/sqoop/mapreduce/MergeReducer.java
Normal file
56
src/java/org/apache/sqoop/mapreduce/MergeReducer.java
Normal file
@ -0,0 +1,56 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.hadoop.io.NullWritable;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.mapreduce.Reducer;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Reducer for merge tool. Given records tagged as 'old' or 'new', emit
|
||||||
|
* a new one if possible; otherwise, an old one.
|
||||||
|
*/
|
||||||
|
public class MergeReducer
|
||||||
|
extends Reducer<Text, MergeRecord, SqoopRecord, NullWritable> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void reduce(Text key, Iterable<MergeRecord> vals, Context c)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
SqoopRecord bestRecord = null;
|
||||||
|
try {
|
||||||
|
for (MergeRecord val : vals) {
|
||||||
|
if (null == bestRecord && !val.isNewRecord()) {
|
||||||
|
// Use an old record if we don't have a new record.
|
||||||
|
bestRecord = (SqoopRecord) val.getSqoopRecord().clone();
|
||||||
|
} else if (val.isNewRecord()) {
|
||||||
|
bestRecord = (SqoopRecord) val.getSqoopRecord().clone();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch (CloneNotSupportedException cnse) {
|
||||||
|
throw new IOException(cnse);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (null != bestRecord) {
|
||||||
|
c.write(bestRecord, NullWritable.get());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
60
src/java/org/apache/sqoop/mapreduce/MergeTextMapper.java
Normal file
60
src/java/org/apache/sqoop/mapreduce/MergeTextMapper.java
Normal file
@ -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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.io.LongWritable;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.util.ReflectionUtils;
|
||||||
|
import com.cloudera.sqoop.lib.RecordParser;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.mapreduce.MergeMapperBase;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mapper for the merge program which operates on text files that we need to
|
||||||
|
* parse into SqoopRecord instances.
|
||||||
|
*/
|
||||||
|
public class MergeTextMapper extends MergeMapperBase<LongWritable, Text> {
|
||||||
|
|
||||||
|
private SqoopRecord record;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void setup(Context c) throws IOException, InterruptedException {
|
||||||
|
Configuration conf = c.getConfiguration();
|
||||||
|
|
||||||
|
Class<? extends SqoopRecord> recordClass =
|
||||||
|
(Class<? extends SqoopRecord>) conf.getClass(
|
||||||
|
MergeJob.MERGE_SQOOP_RECORD_KEY, SqoopRecord.class);
|
||||||
|
this.record = ReflectionUtils.newInstance(recordClass, conf);
|
||||||
|
|
||||||
|
super.setup(c);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void map(LongWritable key, Text val, Context c)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
try {
|
||||||
|
this.record.parse(val);
|
||||||
|
} catch (RecordParser.ParseError pe) {
|
||||||
|
throw new IOException(pe);
|
||||||
|
}
|
||||||
|
|
||||||
|
processRecord(this.record, c);
|
||||||
|
}
|
||||||
|
}
|
130
src/java/org/apache/sqoop/mapreduce/MySQLDumpImportJob.java
Normal file
130
src/java/org/apache/sqoop/mapreduce/MySQLDumpImportJob.java
Normal file
@ -0,0 +1,130 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.io.NullWritable;
|
||||||
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.db.DBWritable;
|
||||||
|
import com.cloudera.sqoop.SqoopOptions;
|
||||||
|
import com.cloudera.sqoop.manager.ConnManager;
|
||||||
|
import com.cloudera.sqoop.manager.ImportJobContext;
|
||||||
|
import com.cloudera.sqoop.manager.MySQLUtils;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DataDrivenDBInputFormat;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Class that runs an import job using mysqldump in the mapper.
|
||||||
|
*/
|
||||||
|
public class MySQLDumpImportJob extends ImportJobBase {
|
||||||
|
|
||||||
|
public static final Log LOG =
|
||||||
|
LogFactory.getLog(MySQLDumpImportJob.class.getName());
|
||||||
|
|
||||||
|
public MySQLDumpImportJob(final SqoopOptions opts, ImportJobContext context)
|
||||||
|
throws ClassNotFoundException {
|
||||||
|
super(opts, MySQLDumpMapper.class, MySQLDumpInputFormat.class,
|
||||||
|
RawKeyTextOutputFormat.class, context);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Configure the inputformat to use for the job.
|
||||||
|
*/
|
||||||
|
protected void configureInputFormat(Job job, String tableName,
|
||||||
|
String tableClassName, String splitByCol)
|
||||||
|
throws ClassNotFoundException, IOException {
|
||||||
|
|
||||||
|
if (null == tableName) {
|
||||||
|
LOG.error(
|
||||||
|
"mysqldump-based import cannot support free-form query imports.");
|
||||||
|
LOG.error("Do not use --direct and --query together for MySQL.");
|
||||||
|
throw new IOException("null tableName for MySQLDumpImportJob.");
|
||||||
|
}
|
||||||
|
|
||||||
|
ConnManager mgr = getContext().getConnManager();
|
||||||
|
String username = options.getUsername();
|
||||||
|
if (null == username || username.length() == 0) {
|
||||||
|
DBConfiguration.configureDB(job.getConfiguration(),
|
||||||
|
mgr.getDriverClass(), options.getConnectString());
|
||||||
|
} else {
|
||||||
|
DBConfiguration.configureDB(job.getConfiguration(),
|
||||||
|
mgr.getDriverClass(), options.getConnectString(), username,
|
||||||
|
options.getPassword());
|
||||||
|
}
|
||||||
|
|
||||||
|
String [] colNames = options.getColumns();
|
||||||
|
if (null == colNames) {
|
||||||
|
colNames = mgr.getColumnNames(tableName);
|
||||||
|
}
|
||||||
|
|
||||||
|
String [] sqlColNames = null;
|
||||||
|
if (null != colNames) {
|
||||||
|
sqlColNames = new String[colNames.length];
|
||||||
|
for (int i = 0; i < colNames.length; i++) {
|
||||||
|
sqlColNames[i] = mgr.escapeColName(colNames[i]);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// It's ok if the where clause is null in DBInputFormat.setInput.
|
||||||
|
String whereClause = options.getWhereClause();
|
||||||
|
|
||||||
|
// We can't set the class properly in here, because we may not have the
|
||||||
|
// jar loaded in this JVM. So we start by calling setInput() with
|
||||||
|
// DBWritable and then overriding the string manually.
|
||||||
|
|
||||||
|
// Note that mysqldump also does *not* want a quoted table name.
|
||||||
|
DataDrivenDBInputFormat.setInput(job, DBWritable.class,
|
||||||
|
tableName, whereClause,
|
||||||
|
mgr.escapeColName(splitByCol), sqlColNames);
|
||||||
|
|
||||||
|
Configuration conf = job.getConfiguration();
|
||||||
|
conf.setInt(MySQLUtils.OUTPUT_FIELD_DELIM_KEY,
|
||||||
|
options.getOutputFieldDelim());
|
||||||
|
conf.setInt(MySQLUtils.OUTPUT_RECORD_DELIM_KEY,
|
||||||
|
options.getOutputRecordDelim());
|
||||||
|
conf.setInt(MySQLUtils.OUTPUT_ENCLOSED_BY_KEY,
|
||||||
|
options.getOutputEnclosedBy());
|
||||||
|
conf.setInt(MySQLUtils.OUTPUT_ESCAPED_BY_KEY,
|
||||||
|
options.getOutputEscapedBy());
|
||||||
|
conf.setBoolean(MySQLUtils.OUTPUT_ENCLOSE_REQUIRED_KEY,
|
||||||
|
options.isOutputEncloseRequired());
|
||||||
|
String [] extraArgs = options.getExtraArgs();
|
||||||
|
if (null != extraArgs) {
|
||||||
|
conf.setStrings(MySQLUtils.EXTRA_ARGS_KEY, extraArgs);
|
||||||
|
}
|
||||||
|
|
||||||
|
LOG.debug("Using InputFormat: " + inputFormatClass);
|
||||||
|
job.setInputFormatClass(getInputFormatClass());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set the mapper class implementation to use in the job,
|
||||||
|
* as well as any related configuration (e.g., map output types).
|
||||||
|
*/
|
||||||
|
protected void configureMapper(Job job, String tableName,
|
||||||
|
String tableClassName) throws ClassNotFoundException, IOException {
|
||||||
|
job.setMapperClass(getMapperClass());
|
||||||
|
job.setOutputKeyClass(String.class);
|
||||||
|
job.setOutputValueClass(NullWritable.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,98 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.io.NullWritable;
|
||||||
|
import org.apache.hadoop.mapreduce.InputSplit;
|
||||||
|
import org.apache.hadoop.mapreduce.RecordReader;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
import com.cloudera.sqoop.mapreduce.DataDrivenImportJob;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DataDrivenDBInputFormat;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* InputFormat designed to take data-driven splits and feed them to a mysqldump
|
||||||
|
* invocation running in the mapper.
|
||||||
|
*
|
||||||
|
* The key emitted by this mapper is a WHERE clause to use in the command
|
||||||
|
* to mysqldump.
|
||||||
|
*/
|
||||||
|
public class MySQLDumpInputFormat extends DataDrivenDBInputFormat {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(
|
||||||
|
MySQLDumpInputFormat.class.getName());
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A RecordReader that just takes the WHERE conditions from the DBInputSplit
|
||||||
|
* and relates them to the mapper as a single input record.
|
||||||
|
*/
|
||||||
|
public static class MySQLDumpRecordReader
|
||||||
|
extends RecordReader<String, NullWritable> {
|
||||||
|
|
||||||
|
private boolean delivered;
|
||||||
|
private String clause;
|
||||||
|
|
||||||
|
public MySQLDumpRecordReader(InputSplit split) {
|
||||||
|
initialize(split, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean nextKeyValue() {
|
||||||
|
boolean hasNext = !delivered;
|
||||||
|
delivered = true;
|
||||||
|
return hasNext;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getCurrentKey() {
|
||||||
|
return clause;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public NullWritable getCurrentValue() {
|
||||||
|
return NullWritable.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public float getProgress() {
|
||||||
|
return delivered ? 1.0f : 0.0f;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void initialize(InputSplit split, TaskAttemptContext context) {
|
||||||
|
DataDrivenDBInputFormat.DataDrivenDBInputSplit dbSplit =
|
||||||
|
(DataDrivenDBInputFormat.DataDrivenDBInputSplit) split;
|
||||||
|
|
||||||
|
this.clause = "(" + dbSplit.getLowerClause() + ") AND ("
|
||||||
|
+ dbSplit.getUpperClause() + ")";
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public RecordReader<String, NullWritable> createRecordReader(InputSplit split,
|
||||||
|
TaskAttemptContext context) {
|
||||||
|
return new MySQLDumpRecordReader(split);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
494
src/java/org/apache/sqoop/mapreduce/MySQLDumpMapper.java
Normal file
494
src/java/org/apache/sqoop/mapreduce/MySQLDumpMapper.java
Normal file
@ -0,0 +1,494 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.BufferedReader;
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.InputStream;
|
||||||
|
import java.io.InputStreamReader;
|
||||||
|
import java.nio.CharBuffer;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.io.NullWritable;
|
||||||
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
|
import org.apache.sqoop.util.AsyncSink;
|
||||||
|
import org.apache.sqoop.util.JdbcUrl;
|
||||||
|
import org.apache.sqoop.util.PerfCounters;
|
||||||
|
import com.cloudera.sqoop.lib.DelimiterSet;
|
||||||
|
import com.cloudera.sqoop.lib.FieldFormatter;
|
||||||
|
import com.cloudera.sqoop.lib.RecordParser;
|
||||||
|
import com.cloudera.sqoop.manager.MySQLUtils;
|
||||||
|
import com.cloudera.sqoop.util.ErrorableAsyncSink;
|
||||||
|
import com.cloudera.sqoop.util.ErrorableThread;
|
||||||
|
import com.cloudera.sqoop.util.LoggingAsyncSink;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mapper that opens up a pipe to mysqldump and pulls data directly.
|
||||||
|
*/
|
||||||
|
public class MySQLDumpMapper
|
||||||
|
extends Mapper<String, NullWritable, String, NullWritable> {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(
|
||||||
|
MySQLDumpMapper.class.getName());
|
||||||
|
|
||||||
|
private Configuration conf;
|
||||||
|
|
||||||
|
// AsyncSinks used to import data from mysqldump directly into HDFS.
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Copies data directly from mysqldump into HDFS, after stripping some
|
||||||
|
* header and footer characters that are attached to each line in mysqldump.
|
||||||
|
*/
|
||||||
|
public static class CopyingAsyncSink extends ErrorableAsyncSink {
|
||||||
|
private final MySQLDumpMapper.Context context;
|
||||||
|
private final PerfCounters counters;
|
||||||
|
|
||||||
|
protected CopyingAsyncSink(final MySQLDumpMapper.Context context,
|
||||||
|
final PerfCounters ctrs) {
|
||||||
|
this.context = context;
|
||||||
|
this.counters = ctrs;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void processStream(InputStream is) {
|
||||||
|
child = new CopyingStreamThread(is, context, counters);
|
||||||
|
child.start();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class CopyingStreamThread extends ErrorableThread {
|
||||||
|
public static final Log LOG = LogFactory.getLog(
|
||||||
|
CopyingStreamThread.class.getName());
|
||||||
|
|
||||||
|
private final MySQLDumpMapper.Context context;
|
||||||
|
private final InputStream stream;
|
||||||
|
private final PerfCounters counters;
|
||||||
|
|
||||||
|
CopyingStreamThread(final InputStream is,
|
||||||
|
final Context c, final PerfCounters ctrs) {
|
||||||
|
this.context = c;
|
||||||
|
this.stream = is;
|
||||||
|
this.counters = ctrs;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void run() {
|
||||||
|
BufferedReader r = null;
|
||||||
|
|
||||||
|
try {
|
||||||
|
r = new BufferedReader(new InputStreamReader(this.stream));
|
||||||
|
|
||||||
|
// Actually do the read/write transfer loop here.
|
||||||
|
int preambleLen = -1; // set to this for "undefined"
|
||||||
|
while (true) {
|
||||||
|
String inLine = r.readLine();
|
||||||
|
if (null == inLine) {
|
||||||
|
break; // EOF.
|
||||||
|
}
|
||||||
|
|
||||||
|
// this line is of the form "INSERT .. VALUES ( actual value text
|
||||||
|
// );" strip the leading preamble up to the '(' and the trailing
|
||||||
|
// ');'.
|
||||||
|
if (preambleLen == -1) {
|
||||||
|
// we haven't determined how long the preamble is. It's constant
|
||||||
|
// across all lines, so just figure this out once.
|
||||||
|
String recordStartMark = "VALUES (";
|
||||||
|
preambleLen = inLine.indexOf(recordStartMark)
|
||||||
|
+ recordStartMark.length();
|
||||||
|
}
|
||||||
|
|
||||||
|
// chop off the leading and trailing text as we write the
|
||||||
|
// output to HDFS.
|
||||||
|
int len = inLine.length() - 2 - preambleLen;
|
||||||
|
context.write(inLine.substring(preambleLen, inLine.length() - 2),
|
||||||
|
null);
|
||||||
|
context.write("\n", null);
|
||||||
|
counters.addBytes(1 + len);
|
||||||
|
}
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
LOG.error("IOException reading from mysqldump: " + ioe.toString());
|
||||||
|
// flag this error so we get an error status back in the caller.
|
||||||
|
setError();
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
LOG.error("InterruptedException reading from mysqldump: "
|
||||||
|
+ ie.toString());
|
||||||
|
// flag this error so we get an error status back in the caller.
|
||||||
|
setError();
|
||||||
|
} finally {
|
||||||
|
if (null != r) {
|
||||||
|
try {
|
||||||
|
r.close();
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
LOG.info("Error closing FIFO stream: " + ioe.toString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The ReparsingAsyncSink will instantiate a RecordParser to read mysqldump's
|
||||||
|
* output, and re-emit the text in the user's specified output format.
|
||||||
|
*/
|
||||||
|
public static class ReparsingAsyncSink extends ErrorableAsyncSink {
|
||||||
|
private final MySQLDumpMapper.Context context;
|
||||||
|
private final Configuration conf;
|
||||||
|
private final PerfCounters counters;
|
||||||
|
|
||||||
|
protected ReparsingAsyncSink(final MySQLDumpMapper.Context c,
|
||||||
|
final Configuration conf, final PerfCounters ctrs) {
|
||||||
|
this.context = c;
|
||||||
|
this.conf = conf;
|
||||||
|
this.counters = ctrs;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void processStream(InputStream is) {
|
||||||
|
child = new ReparsingStreamThread(is, context, conf, counters);
|
||||||
|
child.start();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class ReparsingStreamThread extends ErrorableThread {
|
||||||
|
public static final Log LOG = LogFactory.getLog(
|
||||||
|
ReparsingStreamThread.class.getName());
|
||||||
|
|
||||||
|
private final MySQLDumpMapper.Context context;
|
||||||
|
private final Configuration conf;
|
||||||
|
private final InputStream stream;
|
||||||
|
private final PerfCounters counters;
|
||||||
|
|
||||||
|
ReparsingStreamThread(final InputStream is,
|
||||||
|
final MySQLDumpMapper.Context c, Configuration conf,
|
||||||
|
final PerfCounters ctrs) {
|
||||||
|
this.context = c;
|
||||||
|
this.conf = conf;
|
||||||
|
this.stream = is;
|
||||||
|
this.counters = ctrs;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static final char MYSQL_FIELD_DELIM = ',';
|
||||||
|
private static final char MYSQL_RECORD_DELIM = '\n';
|
||||||
|
private static final char MYSQL_ENCLOSE_CHAR = '\'';
|
||||||
|
private static final char MYSQL_ESCAPE_CHAR = '\\';
|
||||||
|
private static final boolean MYSQL_ENCLOSE_REQUIRED = false;
|
||||||
|
|
||||||
|
private static final RecordParser MYSQLDUMP_PARSER;
|
||||||
|
|
||||||
|
static {
|
||||||
|
// build a record parser for mysqldump's format
|
||||||
|
MYSQLDUMP_PARSER = new RecordParser(DelimiterSet.MYSQL_DELIMITERS);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void run() {
|
||||||
|
BufferedReader r = null;
|
||||||
|
|
||||||
|
try {
|
||||||
|
r = new BufferedReader(new InputStreamReader(this.stream));
|
||||||
|
|
||||||
|
// Configure the output with the user's delimiters.
|
||||||
|
char outputFieldDelim = (char) conf.getInt(
|
||||||
|
MySQLUtils.OUTPUT_FIELD_DELIM_KEY,
|
||||||
|
DelimiterSet.NULL_CHAR);
|
||||||
|
String outputFieldDelimStr = "" + outputFieldDelim;
|
||||||
|
char outputRecordDelim = (char) conf.getInt(
|
||||||
|
MySQLUtils.OUTPUT_RECORD_DELIM_KEY,
|
||||||
|
DelimiterSet.NULL_CHAR);
|
||||||
|
String outputRecordDelimStr = "" + outputRecordDelim;
|
||||||
|
char outputEnclose = (char) conf.getInt(
|
||||||
|
MySQLUtils.OUTPUT_ENCLOSED_BY_KEY,
|
||||||
|
DelimiterSet.NULL_CHAR);
|
||||||
|
char outputEscape = (char) conf.getInt(
|
||||||
|
MySQLUtils.OUTPUT_ESCAPED_BY_KEY,
|
||||||
|
DelimiterSet.NULL_CHAR);
|
||||||
|
boolean outputEncloseRequired = conf.getBoolean(
|
||||||
|
MySQLUtils.OUTPUT_ENCLOSE_REQUIRED_KEY, false);
|
||||||
|
|
||||||
|
DelimiterSet delimiters = new DelimiterSet(
|
||||||
|
outputFieldDelim,
|
||||||
|
outputRecordDelim,
|
||||||
|
outputEnclose,
|
||||||
|
outputEscape,
|
||||||
|
outputEncloseRequired);
|
||||||
|
|
||||||
|
// Actually do the read/write transfer loop here.
|
||||||
|
int preambleLen = -1; // set to this for "undefined"
|
||||||
|
while (true) {
|
||||||
|
String inLine = r.readLine();
|
||||||
|
if (null == inLine) {
|
||||||
|
break; // EOF.
|
||||||
|
}
|
||||||
|
|
||||||
|
// this line is of the form "INSERT .. VALUES ( actual value text
|
||||||
|
// );" strip the leading preamble up to the '(' and the trailing
|
||||||
|
// ');'.
|
||||||
|
if (preambleLen == -1) {
|
||||||
|
// we haven't determined how long the preamble is. It's constant
|
||||||
|
// across all lines, so just figure this out once.
|
||||||
|
String recordStartMark = "VALUES (";
|
||||||
|
preambleLen = inLine.indexOf(recordStartMark)
|
||||||
|
+ recordStartMark.length();
|
||||||
|
}
|
||||||
|
|
||||||
|
// Wrap the input string in a char buffer that ignores the leading
|
||||||
|
// and trailing text.
|
||||||
|
CharBuffer charbuf = CharBuffer.wrap(inLine, preambleLen,
|
||||||
|
inLine.length() - 2);
|
||||||
|
|
||||||
|
// Pass this along to the parser
|
||||||
|
List<String> fields = null;
|
||||||
|
try {
|
||||||
|
fields = MYSQLDUMP_PARSER.parseRecord(charbuf);
|
||||||
|
} catch (RecordParser.ParseError pe) {
|
||||||
|
LOG.warn("ParseError reading from mysqldump: "
|
||||||
|
+ pe.toString() + "; record skipped");
|
||||||
|
continue; // Skip emitting this row.
|
||||||
|
}
|
||||||
|
|
||||||
|
// For all of the output fields, emit them using the delimiters
|
||||||
|
// the user chooses.
|
||||||
|
boolean first = true;
|
||||||
|
int recordLen = 1; // for the delimiter.
|
||||||
|
for (String field : fields) {
|
||||||
|
if (!first) {
|
||||||
|
context.write(outputFieldDelimStr, null);
|
||||||
|
} else {
|
||||||
|
first = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
String fieldStr = FieldFormatter.escapeAndEnclose(field,
|
||||||
|
delimiters);
|
||||||
|
context.write(fieldStr, null);
|
||||||
|
recordLen += fieldStr.length();
|
||||||
|
}
|
||||||
|
|
||||||
|
context.write(outputRecordDelimStr, null);
|
||||||
|
counters.addBytes(recordLen);
|
||||||
|
}
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
LOG.error("IOException reading from mysqldump: " + ioe.toString());
|
||||||
|
// flag this error so the parent can handle it appropriately.
|
||||||
|
setError();
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
LOG.error("InterruptedException reading from mysqldump: "
|
||||||
|
+ ie.toString());
|
||||||
|
// flag this error so we get an error status back in the caller.
|
||||||
|
setError();
|
||||||
|
} finally {
|
||||||
|
if (null != r) {
|
||||||
|
try {
|
||||||
|
r.close();
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
LOG.info("Error closing FIFO stream: " + ioe.toString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// TODO(aaron): Refactor this method to be much shorter.
|
||||||
|
// CHECKSTYLE:OFF
|
||||||
|
/**
|
||||||
|
* Import the table into HDFS by using mysqldump to pull out the data from
|
||||||
|
* the database and upload the files directly to HDFS.
|
||||||
|
*/
|
||||||
|
public void map(String splitConditions, NullWritable val, Context context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
|
||||||
|
LOG.info("Beginning mysqldump fast path import");
|
||||||
|
|
||||||
|
ArrayList<String> args = new ArrayList<String>();
|
||||||
|
String tableName = conf.get(MySQLUtils.TABLE_NAME_KEY);
|
||||||
|
|
||||||
|
// We need to parse the connect string URI to determine the database name.
|
||||||
|
// Using java.net.URL directly on the connect string will fail because
|
||||||
|
// Java doesn't respect arbitrary JDBC-based schemes. So we chop off the
|
||||||
|
// scheme (everything before '://') and replace it with 'http', which we
|
||||||
|
// know will work.
|
||||||
|
String connectString = conf.get(MySQLUtils.CONNECT_STRING_KEY);
|
||||||
|
String databaseName = JdbcUrl.getDatabaseName(connectString);
|
||||||
|
String hostname = JdbcUrl.getHostName(connectString);
|
||||||
|
int port = JdbcUrl.getPort(connectString);
|
||||||
|
|
||||||
|
if (null == databaseName) {
|
||||||
|
throw new IOException("Could not determine database name");
|
||||||
|
}
|
||||||
|
|
||||||
|
LOG.info("Performing import of table " + tableName + " from database "
|
||||||
|
+ databaseName);
|
||||||
|
|
||||||
|
args.add(MySQLUtils.MYSQL_DUMP_CMD); // requires that this is on the path.
|
||||||
|
|
||||||
|
String password = conf.get(MySQLUtils.PASSWORD_KEY);
|
||||||
|
String passwordFile = null;
|
||||||
|
|
||||||
|
Process p = null;
|
||||||
|
AsyncSink sink = null;
|
||||||
|
AsyncSink errSink = null;
|
||||||
|
PerfCounters counters = new PerfCounters();
|
||||||
|
try {
|
||||||
|
// --defaults-file must be the first argument.
|
||||||
|
if (null != password && password.length() > 0) {
|
||||||
|
passwordFile = MySQLUtils.writePasswordFile(conf);
|
||||||
|
args.add("--defaults-file=" + passwordFile);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Don't use the --where="<whereClause>" version because spaces in it can
|
||||||
|
// confuse Java, and adding in surrounding quotes confuses Java as well.
|
||||||
|
String whereClause = conf.get(MySQLUtils.WHERE_CLAUSE_KEY, "(1=1)")
|
||||||
|
+ " AND (" + splitConditions + ")";
|
||||||
|
args.add("-w");
|
||||||
|
args.add(whereClause);
|
||||||
|
|
||||||
|
args.add("--host=" + hostname);
|
||||||
|
if (-1 != port) {
|
||||||
|
args.add("--port=" + Integer.toString(port));
|
||||||
|
}
|
||||||
|
args.add("--skip-opt");
|
||||||
|
args.add("--compact");
|
||||||
|
args.add("--no-create-db");
|
||||||
|
args.add("--no-create-info");
|
||||||
|
args.add("--quick"); // no buffering
|
||||||
|
args.add("--single-transaction");
|
||||||
|
|
||||||
|
String username = conf.get(MySQLUtils.USERNAME_KEY);
|
||||||
|
if (null != username) {
|
||||||
|
args.add("--user=" + username);
|
||||||
|
}
|
||||||
|
|
||||||
|
// If the user supplied extra args, add them here.
|
||||||
|
String [] extra = conf.getStrings(MySQLUtils.EXTRA_ARGS_KEY);
|
||||||
|
if (null != extra) {
|
||||||
|
for (String arg : extra) {
|
||||||
|
args.add(arg);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
args.add(databaseName);
|
||||||
|
args.add(tableName);
|
||||||
|
|
||||||
|
// begin the import in an external process.
|
||||||
|
LOG.debug("Starting mysqldump with arguments:");
|
||||||
|
for (String arg : args) {
|
||||||
|
LOG.debug(" " + arg);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Actually start the mysqldump.
|
||||||
|
p = Runtime.getRuntime().exec(args.toArray(new String[0]));
|
||||||
|
|
||||||
|
// read from the stdout pipe into the HDFS writer.
|
||||||
|
InputStream is = p.getInputStream();
|
||||||
|
|
||||||
|
if (MySQLUtils.outputDelimsAreMySQL(conf)) {
|
||||||
|
LOG.debug("Output delimiters conform to mysqldump; "
|
||||||
|
+ "using straight copy");
|
||||||
|
sink = new CopyingAsyncSink(context, counters);
|
||||||
|
} else {
|
||||||
|
LOG.debug("User-specified delimiters; using reparsing import");
|
||||||
|
LOG.info("Converting data to use specified delimiters.");
|
||||||
|
LOG.info("(For the fastest possible import, use");
|
||||||
|
LOG.info("--mysql-delimiters to specify the same field");
|
||||||
|
LOG.info("delimiters as are used by mysqldump.)");
|
||||||
|
sink = new ReparsingAsyncSink(context, conf, counters);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Start an async thread to read and upload the whole stream.
|
||||||
|
counters.startClock();
|
||||||
|
sink.processStream(is);
|
||||||
|
|
||||||
|
// Start an async thread to send stderr to log4j.
|
||||||
|
errSink = new LoggingAsyncSink(LOG);
|
||||||
|
errSink.processStream(p.getErrorStream());
|
||||||
|
} finally {
|
||||||
|
|
||||||
|
// block until the process is done.
|
||||||
|
int result = 0;
|
||||||
|
if (null != p) {
|
||||||
|
while (true) {
|
||||||
|
try {
|
||||||
|
result = p.waitFor();
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
// interrupted; loop around.
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Remove the password file.
|
||||||
|
if (null != passwordFile) {
|
||||||
|
if (!new File(passwordFile).delete()) {
|
||||||
|
LOG.error("Could not remove mysql password file " + passwordFile);
|
||||||
|
LOG.error("You should remove this file to protect your credentials.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// block until the stream sink is done too.
|
||||||
|
int streamResult = 0;
|
||||||
|
if (null != sink) {
|
||||||
|
while (true) {
|
||||||
|
try {
|
||||||
|
streamResult = sink.join();
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
// interrupted; loop around.
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Try to wait for stderr to finish, but regard any errors as advisory.
|
||||||
|
if (null != errSink) {
|
||||||
|
try {
|
||||||
|
if (0 != errSink.join()) {
|
||||||
|
LOG.info("Encountered exception reading stderr stream");
|
||||||
|
}
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
LOG.info("Thread interrupted waiting for stderr to complete: "
|
||||||
|
+ ie.toString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
LOG.info("Transfer loop complete.");
|
||||||
|
|
||||||
|
if (0 != result) {
|
||||||
|
throw new IOException("mysqldump terminated with status "
|
||||||
|
+ Integer.toString(result));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (0 != streamResult) {
|
||||||
|
throw new IOException("Encountered exception in stream sink");
|
||||||
|
}
|
||||||
|
|
||||||
|
counters.stopClock();
|
||||||
|
LOG.info("Transferred " + counters.toString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// CHECKSTYLE:ON
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void setup(Context context) {
|
||||||
|
this.conf = context.getConfiguration();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
114
src/java/org/apache/sqoop/mapreduce/MySQLExportJob.java
Normal file
114
src/java/org/apache/sqoop/mapreduce/MySQLExportJob.java
Normal file
@ -0,0 +1,114 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.db.DBWritable;
|
||||||
|
import com.cloudera.sqoop.manager.ConnManager;
|
||||||
|
import com.cloudera.sqoop.manager.ExportJobContext;
|
||||||
|
import com.cloudera.sqoop.manager.MySQLUtils;
|
||||||
|
import com.cloudera.sqoop.mapreduce.ExportJobBase;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DataDrivenDBInputFormat;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Class that runs an export job using mysqlimport in the mapper.
|
||||||
|
*/
|
||||||
|
public class MySQLExportJob extends ExportJobBase {
|
||||||
|
|
||||||
|
public static final Log LOG =
|
||||||
|
LogFactory.getLog(MySQLExportJob.class.getName());
|
||||||
|
|
||||||
|
public MySQLExportJob(final ExportJobContext context) {
|
||||||
|
super(context, null, null, NullOutputFormat.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/**
|
||||||
|
* Configure the inputformat to use for the job.
|
||||||
|
*/
|
||||||
|
protected void configureInputFormat(Job job, String tableName,
|
||||||
|
String tableClassName, String splitByCol)
|
||||||
|
throws ClassNotFoundException, IOException {
|
||||||
|
|
||||||
|
// Configure the delimiters, etc.
|
||||||
|
Configuration conf = job.getConfiguration();
|
||||||
|
conf.setInt(MySQLUtils.OUTPUT_FIELD_DELIM_KEY,
|
||||||
|
options.getOutputFieldDelim());
|
||||||
|
conf.setInt(MySQLUtils.OUTPUT_RECORD_DELIM_KEY,
|
||||||
|
options.getOutputRecordDelim());
|
||||||
|
conf.setInt(MySQLUtils.OUTPUT_ENCLOSED_BY_KEY,
|
||||||
|
options.getOutputEnclosedBy());
|
||||||
|
conf.setInt(MySQLUtils.OUTPUT_ESCAPED_BY_KEY,
|
||||||
|
options.getOutputEscapedBy());
|
||||||
|
conf.setBoolean(MySQLUtils.OUTPUT_ENCLOSE_REQUIRED_KEY,
|
||||||
|
options.isOutputEncloseRequired());
|
||||||
|
String [] extraArgs = options.getExtraArgs();
|
||||||
|
if (null != extraArgs) {
|
||||||
|
conf.setStrings(MySQLUtils.EXTRA_ARGS_KEY, extraArgs);
|
||||||
|
}
|
||||||
|
|
||||||
|
ConnManager mgr = context.getConnManager();
|
||||||
|
String username = options.getUsername();
|
||||||
|
if (null == username || username.length() == 0) {
|
||||||
|
DBConfiguration.configureDB(job.getConfiguration(),
|
||||||
|
mgr.getDriverClass(), options.getConnectString());
|
||||||
|
} else {
|
||||||
|
DBConfiguration.configureDB(job.getConfiguration(),
|
||||||
|
mgr.getDriverClass(), options.getConnectString(), username,
|
||||||
|
options.getPassword());
|
||||||
|
}
|
||||||
|
|
||||||
|
String [] colNames = options.getColumns();
|
||||||
|
if (null == colNames) {
|
||||||
|
colNames = mgr.getColumnNames(tableName);
|
||||||
|
}
|
||||||
|
|
||||||
|
String [] sqlColNames = null;
|
||||||
|
if (null != colNames) {
|
||||||
|
sqlColNames = new String[colNames.length];
|
||||||
|
for (int i = 0; i < colNames.length; i++) {
|
||||||
|
sqlColNames[i] = mgr.escapeColName(colNames[i]);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Note that mysqldump also does *not* want a quoted table name.
|
||||||
|
DataDrivenDBInputFormat.setInput(job, DBWritable.class,
|
||||||
|
tableName, null, null, sqlColNames);
|
||||||
|
|
||||||
|
// Configure the actual InputFormat to use.
|
||||||
|
super.configureInputFormat(job, tableName, tableClassName, splitByCol);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Class<? extends Mapper> getMapperClass() {
|
||||||
|
if (inputIsSequenceFiles()) {
|
||||||
|
return MySQLRecordExportMapper.class;
|
||||||
|
} else {
|
||||||
|
return MySQLTextExportMapper.class;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
357
src/java/org/apache/sqoop/mapreduce/MySQLExportMapper.java
Normal file
357
src/java/org/apache/sqoop/mapreduce/MySQLExportMapper.java
Normal file
@ -0,0 +1,357 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.BufferedOutputStream;
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.FileOutputStream;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.OutputStream;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.io.NullWritable;
|
||||||
|
import org.apache.hadoop.mapreduce.Mapper;
|
||||||
|
import org.apache.sqoop.util.AsyncSink;
|
||||||
|
import org.apache.sqoop.util.JdbcUrl;
|
||||||
|
import org.apache.sqoop.util.LoggingAsyncSink;
|
||||||
|
import org.apache.sqoop.util.NullAsyncSink;
|
||||||
|
import org.apache.sqoop.util.TaskId;
|
||||||
|
import com.cloudera.sqoop.io.NamedFifo;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
||||||
|
import com.cloudera.sqoop.manager.MySQLUtils;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mapper that starts a 'mysqlimport' process and uses that to export rows from
|
||||||
|
* HDFS to a MySQL database at high speed.
|
||||||
|
*
|
||||||
|
* map() methods are actually provided by subclasses that read from
|
||||||
|
* SequenceFiles (containing existing SqoopRecords) or text files
|
||||||
|
* (containing delimited lines) and deliver these results to the fifo
|
||||||
|
* used to interface with mysqlimport.
|
||||||
|
*/
|
||||||
|
public class MySQLExportMapper<KEYIN, VALIN>
|
||||||
|
extends Mapper<KEYIN, VALIN, NullWritable, NullWritable> {
|
||||||
|
|
||||||
|
public static final Log LOG = LogFactory.getLog(
|
||||||
|
MySQLExportMapper.class.getName());
|
||||||
|
|
||||||
|
/** Configuration key that specifies the number of bytes before which it
|
||||||
|
* commits the current export transaction and opens a new one.
|
||||||
|
* Default is 32 MB; setting this to 0 will use no checkpoints.
|
||||||
|
*/
|
||||||
|
public static final String MYSQL_CHECKPOINT_BYTES_KEY =
|
||||||
|
"sqoop.mysql.export.checkpoint.bytes";
|
||||||
|
|
||||||
|
public static final long DEFAULT_CHECKPOINT_BYTES = 32 * 1024 * 1024;
|
||||||
|
|
||||||
|
// Configured value for MSYQL_CHECKPOINT_BYTES_KEY.
|
||||||
|
protected long checkpointDistInBytes;
|
||||||
|
|
||||||
|
protected Configuration conf;
|
||||||
|
|
||||||
|
/** The FIFO being used to communicate with mysqlimport. */
|
||||||
|
protected File fifoFile;
|
||||||
|
|
||||||
|
/** The process object representing the active connection to mysqlimport. */
|
||||||
|
protected Process mysqlImportProcess;
|
||||||
|
|
||||||
|
/** The stream to write to stdin for mysqlimport. */
|
||||||
|
protected OutputStream importStream;
|
||||||
|
|
||||||
|
// Handlers for stdout and stderr from mysqlimport.
|
||||||
|
protected AsyncSink outSink;
|
||||||
|
protected AsyncSink errSink;
|
||||||
|
|
||||||
|
/** File object where we wrote the user's password to pass to mysqlimport. */
|
||||||
|
protected File passwordFile;
|
||||||
|
|
||||||
|
/** Character set used to write to mysqlimport. */
|
||||||
|
protected String mysqlCharSet;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tally of bytes written to current mysqlimport instance.
|
||||||
|
* We commit an interim tx and open a new mysqlimport after this
|
||||||
|
* gets too big. */
|
||||||
|
private long bytesWritten;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a named FIFO, and start mysqlimport connected to that FIFO.
|
||||||
|
* A File object representing the FIFO is in 'fifoFile'.
|
||||||
|
*/
|
||||||
|
private void initMySQLImportProcess() throws IOException {
|
||||||
|
File taskAttemptDir = TaskId.getLocalWorkPath(conf);
|
||||||
|
|
||||||
|
this.fifoFile = new File(taskAttemptDir,
|
||||||
|
conf.get(MySQLUtils.TABLE_NAME_KEY, "UNKNOWN_TABLE") + ".txt");
|
||||||
|
String filename = fifoFile.toString();
|
||||||
|
|
||||||
|
// Create the FIFO itself.
|
||||||
|
try {
|
||||||
|
new NamedFifo(this.fifoFile).create();
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
// Command failed.
|
||||||
|
LOG.error("Could not mknod " + filename);
|
||||||
|
this.fifoFile = null;
|
||||||
|
throw new IOException(
|
||||||
|
"Could not create FIFO to interface with mysqlimport", ioe);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Now open the connection to mysqlimport.
|
||||||
|
ArrayList<String> args = new ArrayList<String>();
|
||||||
|
|
||||||
|
String connectString = conf.get(MySQLUtils.CONNECT_STRING_KEY);
|
||||||
|
String databaseName = JdbcUrl.getDatabaseName(connectString);
|
||||||
|
String hostname = JdbcUrl.getHostName(connectString);
|
||||||
|
int port = JdbcUrl.getPort(connectString);
|
||||||
|
|
||||||
|
if (null == databaseName) {
|
||||||
|
throw new IOException("Could not determine database name");
|
||||||
|
}
|
||||||
|
|
||||||
|
args.add(MySQLUtils.MYSQL_IMPORT_CMD); // needs to be on the path.
|
||||||
|
String password = conf.get(MySQLUtils.PASSWORD_KEY);
|
||||||
|
|
||||||
|
if (null != password && password.length() > 0) {
|
||||||
|
passwordFile = new File(MySQLUtils.writePasswordFile(conf));
|
||||||
|
args.add("--defaults-file=" + passwordFile);
|
||||||
|
}
|
||||||
|
|
||||||
|
String username = conf.get(MySQLUtils.USERNAME_KEY);
|
||||||
|
if (null != username) {
|
||||||
|
args.add("--user=" + username);
|
||||||
|
}
|
||||||
|
|
||||||
|
args.add("--host=" + hostname);
|
||||||
|
if (-1 != port) {
|
||||||
|
args.add("--port=" + Integer.toString(port));
|
||||||
|
}
|
||||||
|
|
||||||
|
args.add("--compress");
|
||||||
|
args.add("--local");
|
||||||
|
args.add("--silent");
|
||||||
|
|
||||||
|
// Specify the subset of columns we're importing.
|
||||||
|
DBConfiguration dbConf = new DBConfiguration(conf);
|
||||||
|
String [] cols = dbConf.getInputFieldNames();
|
||||||
|
if (null != cols) {
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
boolean first = true;
|
||||||
|
for (String col : cols) {
|
||||||
|
if (!first) {
|
||||||
|
sb.append(",");
|
||||||
|
}
|
||||||
|
sb.append(col);
|
||||||
|
first = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
args.add("--columns=" + sb.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
// Specify the delimiters to use.
|
||||||
|
int outputFieldDelim = conf.getInt(MySQLUtils.OUTPUT_FIELD_DELIM_KEY,
|
||||||
|
(int) ',');
|
||||||
|
int outputRecordDelim = conf.getInt(MySQLUtils.OUTPUT_RECORD_DELIM_KEY,
|
||||||
|
(int) '\n');
|
||||||
|
int enclosedBy = conf.getInt(MySQLUtils.OUTPUT_ENCLOSED_BY_KEY, 0);
|
||||||
|
int escapedBy = conf.getInt(MySQLUtils.OUTPUT_ESCAPED_BY_KEY, 0);
|
||||||
|
boolean encloseRequired = conf.getBoolean(
|
||||||
|
MySQLUtils.OUTPUT_ENCLOSE_REQUIRED_KEY, false);
|
||||||
|
|
||||||
|
args.add("--fields-terminated-by=0x"
|
||||||
|
+ Integer.toString(outputFieldDelim, 16));
|
||||||
|
args.add("--lines-terminated-by=0x"
|
||||||
|
+ Integer.toString(outputRecordDelim, 16));
|
||||||
|
if (0 != enclosedBy) {
|
||||||
|
if (encloseRequired) {
|
||||||
|
args.add("--fields-enclosed-by=0x" + Integer.toString(enclosedBy, 16));
|
||||||
|
} else {
|
||||||
|
args.add("--fields-optionally-enclosed-by=0x"
|
||||||
|
+ Integer.toString(enclosedBy, 16));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (0 != escapedBy) {
|
||||||
|
args.add("--escaped-by=0x" + Integer.toString(escapedBy, 16));
|
||||||
|
}
|
||||||
|
|
||||||
|
// These two arguments are positional and must be last.
|
||||||
|
args.add(databaseName);
|
||||||
|
args.add(filename);
|
||||||
|
|
||||||
|
// Begin the export in an external process.
|
||||||
|
LOG.debug("Starting mysqlimport with arguments:");
|
||||||
|
for (String arg : args) {
|
||||||
|
LOG.debug(" " + arg);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Actually start mysqlimport.
|
||||||
|
mysqlImportProcess = Runtime.getRuntime().exec(args.toArray(new String[0]));
|
||||||
|
|
||||||
|
// Log everything it writes to stderr.
|
||||||
|
// Ignore anything on stdout.
|
||||||
|
this.outSink = new NullAsyncSink();
|
||||||
|
this.outSink.processStream(mysqlImportProcess.getInputStream());
|
||||||
|
|
||||||
|
this.errSink = new LoggingAsyncSink(LOG);
|
||||||
|
this.errSink.processStream(mysqlImportProcess.getErrorStream());
|
||||||
|
|
||||||
|
// Open the named FIFO after starting mysqlimport.
|
||||||
|
this.importStream = new BufferedOutputStream(
|
||||||
|
new FileOutputStream(fifoFile));
|
||||||
|
|
||||||
|
// At this point, mysqlimport is running and hooked up to our FIFO.
|
||||||
|
// The mapper just needs to populate it with data.
|
||||||
|
|
||||||
|
this.bytesWritten = 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run(Context context) throws IOException, InterruptedException {
|
||||||
|
this.conf = context.getConfiguration();
|
||||||
|
setup(context);
|
||||||
|
initMySQLImportProcess();
|
||||||
|
try {
|
||||||
|
while (context.nextKeyValue()) {
|
||||||
|
map(context.getCurrentKey(), context.getCurrentValue(), context);
|
||||||
|
}
|
||||||
|
cleanup(context);
|
||||||
|
} finally {
|
||||||
|
// Shut down the mysqlimport process.
|
||||||
|
closeExportHandles();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void closeExportHandles() throws IOException, InterruptedException {
|
||||||
|
int ret = 0;
|
||||||
|
if (null != this.importStream) {
|
||||||
|
// Close the stream that writes to mysqlimport's stdin first.
|
||||||
|
LOG.debug("Closing import stream");
|
||||||
|
this.importStream.close();
|
||||||
|
this.importStream = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (null != this.mysqlImportProcess) {
|
||||||
|
// We started mysqlimport; wait for it to finish.
|
||||||
|
LOG.info("Waiting for mysqlimport to complete");
|
||||||
|
ret = this.mysqlImportProcess.waitFor();
|
||||||
|
LOG.info("mysqlimport closed connection");
|
||||||
|
this.mysqlImportProcess = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (null != this.passwordFile && this.passwordFile.exists()) {
|
||||||
|
if (!this.passwordFile.delete()) {
|
||||||
|
LOG.error("Could not remove mysql password file " + passwordFile);
|
||||||
|
LOG.error("You should remove this file to protect your credentials.");
|
||||||
|
}
|
||||||
|
|
||||||
|
this.passwordFile = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Finish processing any output from mysqlimport.
|
||||||
|
// This is informational only, so we don't care about return codes.
|
||||||
|
if (null != outSink) {
|
||||||
|
LOG.debug("Waiting for any additional stdout from mysqlimport");
|
||||||
|
outSink.join();
|
||||||
|
outSink = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (null != errSink) {
|
||||||
|
LOG.debug("Waiting for any additional stderr from mysqlimport");
|
||||||
|
errSink.join();
|
||||||
|
errSink = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (this.fifoFile != null && this.fifoFile.exists()) {
|
||||||
|
// Clean up the resources we created.
|
||||||
|
LOG.debug("Removing fifo file");
|
||||||
|
if (!this.fifoFile.delete()) {
|
||||||
|
LOG.error("Could not clean up named FIFO after completing mapper");
|
||||||
|
}
|
||||||
|
|
||||||
|
// We put the FIFO file in a one-off subdir. Remove that.
|
||||||
|
File fifoParentDir = this.fifoFile.getParentFile();
|
||||||
|
LOG.debug("Removing task attempt tmpdir");
|
||||||
|
if (!fifoParentDir.delete()) {
|
||||||
|
LOG.error("Could not clean up task dir after completing mapper");
|
||||||
|
}
|
||||||
|
|
||||||
|
this.fifoFile = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (0 != ret) {
|
||||||
|
// Don't mark the task as successful if mysqlimport returns an error.
|
||||||
|
throw new IOException("mysqlimport terminated with error code " + ret);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void setup(Context context) {
|
||||||
|
this.conf = context.getConfiguration();
|
||||||
|
|
||||||
|
// TODO: Support additional encodings.
|
||||||
|
this.mysqlCharSet = MySQLUtils.MYSQL_DEFAULT_CHARSET;
|
||||||
|
|
||||||
|
this.checkpointDistInBytes = conf.getLong(
|
||||||
|
MYSQL_CHECKPOINT_BYTES_KEY, DEFAULT_CHECKPOINT_BYTES);
|
||||||
|
if (this.checkpointDistInBytes < 0) {
|
||||||
|
LOG.warn("Invalid value for " + MYSQL_CHECKPOINT_BYTES_KEY);
|
||||||
|
this.checkpointDistInBytes = DEFAULT_CHECKPOINT_BYTES;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Takes a delimited text record (e.g., the output of a 'Text' object),
|
||||||
|
* re-encodes it for consumption by mysqlimport, and writes it to the pipe.
|
||||||
|
* @param record A delimited text representation of one record.
|
||||||
|
* @param terminator an optional string that contains delimiters that
|
||||||
|
* terminate the record (if not included in 'record' itself).
|
||||||
|
*/
|
||||||
|
protected void writeRecord(String record, String terminator)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
|
||||||
|
// We've already set up mysqlimport to accept the same delimiters,
|
||||||
|
// so we don't need to convert those. But our input text is UTF8
|
||||||
|
// encoded; mysql allows configurable encoding, but defaults to
|
||||||
|
// latin-1 (ISO8859_1). We'll convert to latin-1 for now.
|
||||||
|
// TODO: Support user-configurable encodings.
|
||||||
|
|
||||||
|
byte [] mysqlBytes = record.getBytes(this.mysqlCharSet);
|
||||||
|
this.importStream.write(mysqlBytes, 0, mysqlBytes.length);
|
||||||
|
this.bytesWritten += mysqlBytes.length;
|
||||||
|
|
||||||
|
if (null != terminator) {
|
||||||
|
byte [] termBytes = terminator.getBytes(this.mysqlCharSet);
|
||||||
|
this.importStream.write(termBytes, 0, termBytes.length);
|
||||||
|
this.bytesWritten += termBytes.length;
|
||||||
|
}
|
||||||
|
|
||||||
|
// If bytesWritten is too big, then we should start a new tx by closing
|
||||||
|
// mysqlimport and opening a new instance of the process.
|
||||||
|
if (this.checkpointDistInBytes != 0
|
||||||
|
&& this.bytesWritten > this.checkpointDistInBytes) {
|
||||||
|
LOG.info("Checkpointing current export.");
|
||||||
|
closeExportHandles();
|
||||||
|
initMySQLImportProcess();
|
||||||
|
this.bytesWritten = 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,49 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.hadoop.io.LongWritable;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.mapreduce.MySQLExportMapper;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* mysqlimport-based exporter which accepts SqoopRecords (e.g., from
|
||||||
|
* SequenceFiles) to emit to the database.
|
||||||
|
*/
|
||||||
|
public class MySQLRecordExportMapper
|
||||||
|
extends MySQLExportMapper<LongWritable, SqoopRecord> {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Export the table to MySQL by using mysqlimport to write the data to the
|
||||||
|
* database.
|
||||||
|
*
|
||||||
|
* Expects one SqoopRecord as the value. Ignores the key.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void map(LongWritable key, SqoopRecord val, Context context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
|
||||||
|
writeRecord(val.toString(), null);
|
||||||
|
|
||||||
|
// We don't emit anything to the OutputCollector because we wrote
|
||||||
|
// straight to mysql. Send a progress indicator to prevent a timeout.
|
||||||
|
context.progress();
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,63 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.hadoop.io.LongWritable;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import com.cloudera.sqoop.manager.MySQLUtils;
|
||||||
|
import com.cloudera.sqoop.mapreduce.MySQLExportMapper;;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* mysqlimport-based exporter which accepts lines of text from files
|
||||||
|
* in HDFS to emit to the database.
|
||||||
|
*/
|
||||||
|
public class MySQLTextExportMapper
|
||||||
|
extends MySQLExportMapper<LongWritable, Text> {
|
||||||
|
|
||||||
|
// End-of-record delimiter.
|
||||||
|
private String recordEndStr;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void setup(Context context) {
|
||||||
|
super.setup(context);
|
||||||
|
|
||||||
|
char recordDelim = (char) conf.getInt(MySQLUtils.OUTPUT_RECORD_DELIM_KEY,
|
||||||
|
(int) '\n');
|
||||||
|
this.recordEndStr = "" + recordDelim;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Export the table to MySQL by using mysqlimport to write the data to the
|
||||||
|
* database.
|
||||||
|
*
|
||||||
|
* Expects one delimited text record as the 'val'; ignores the key.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void map(LongWritable key, Text val, Context context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
|
||||||
|
writeRecord(val.toString(), this.recordEndStr);
|
||||||
|
|
||||||
|
// We don't emit anything to the OutputCollector because we wrote
|
||||||
|
// straight to mysql. Send a progress indicator to prevent a timeout.
|
||||||
|
context.progress();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
45
src/java/org/apache/sqoop/mapreduce/NullOutputCommitter.java
Normal file
45
src/java/org/apache/sqoop/mapreduce/NullOutputCommitter.java
Normal file
@ -0,0 +1,45 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import org.apache.hadoop.mapreduce.JobContext;
|
||||||
|
import org.apache.hadoop.mapreduce.OutputCommitter;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* OutputCommitter instance that does nothing.
|
||||||
|
*/
|
||||||
|
public class NullOutputCommitter extends OutputCommitter {
|
||||||
|
|
||||||
|
public void abortTask(TaskAttemptContext taskContext) { }
|
||||||
|
|
||||||
|
public void cleanupJob(JobContext jobContext) { }
|
||||||
|
|
||||||
|
public void commitTask(TaskAttemptContext taskContext) { }
|
||||||
|
|
||||||
|
public boolean needsTaskCommit(TaskAttemptContext taskContext) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setupJob(JobContext jobContext) { }
|
||||||
|
|
||||||
|
public void setupTask(TaskAttemptContext taskContext) { }
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,110 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import org.apache.hadoop.mapreduce.RecordWriter;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.mapreduce.ExportOutputFormat;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Oracle-specific SQL formatting overrides default ExportOutputFormat's.
|
||||||
|
*/
|
||||||
|
public class OracleExportOutputFormat<K extends SqoopRecord, V>
|
||||||
|
extends ExportOutputFormat<K, V> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
||||||
|
throws IOException {
|
||||||
|
try {
|
||||||
|
return new OracleExportRecordWriter(context);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new IOException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* RecordWriter to write the output to a row in a database table.
|
||||||
|
* The actual database updates are executed in a second thread.
|
||||||
|
*/
|
||||||
|
public class OracleExportRecordWriter extends ExportRecordWriter {
|
||||||
|
|
||||||
|
public OracleExportRecordWriter(TaskAttemptContext context)
|
||||||
|
throws ClassNotFoundException, SQLException {
|
||||||
|
super(context);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/**
|
||||||
|
* @return an INSERT statement suitable for inserting 'numRows' rows.
|
||||||
|
*/
|
||||||
|
protected String getInsertStatement(int numRows) {
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
|
||||||
|
sb.append("INSERT INTO " + getTableName() + " ");
|
||||||
|
|
||||||
|
int numSlots;
|
||||||
|
String [] colNames = getColumnNames();
|
||||||
|
if (colNames != null) {
|
||||||
|
numSlots = colNames.length;
|
||||||
|
|
||||||
|
sb.append("(");
|
||||||
|
boolean first = true;
|
||||||
|
for (String col : colNames) {
|
||||||
|
if (!first) {
|
||||||
|
sb.append(", ");
|
||||||
|
}
|
||||||
|
|
||||||
|
sb.append(col);
|
||||||
|
first = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
sb.append(") ");
|
||||||
|
} else {
|
||||||
|
numSlots = getColumnCount(); // set if columnNames is null.
|
||||||
|
}
|
||||||
|
|
||||||
|
// generates the (?, ?, ?...) used for each row.
|
||||||
|
StringBuilder sbRow = new StringBuilder();
|
||||||
|
sbRow.append("SELECT ");
|
||||||
|
for (int i = 0; i < numSlots; i++) {
|
||||||
|
if (i != 0) {
|
||||||
|
sbRow.append(", ");
|
||||||
|
}
|
||||||
|
|
||||||
|
sbRow.append("?");
|
||||||
|
}
|
||||||
|
sbRow.append(" FROM DUAL ");
|
||||||
|
|
||||||
|
// Now append that numRows times.
|
||||||
|
for (int i = 0; i < numRows; i++) {
|
||||||
|
if (i != 0) {
|
||||||
|
sb.append("UNION ALL ");
|
||||||
|
}
|
||||||
|
|
||||||
|
sb.append(sbRow);
|
||||||
|
}
|
||||||
|
|
||||||
|
return sb.toString();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.util.LinkedHashSet;
|
||||||
|
import java.util.Set;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.mapreduce.RecordWriter;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.mapreduce.UpdateOutputFormat;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Update an existing table with new value if the table already
|
||||||
|
* contains the row, or insert the data into the table if the table
|
||||||
|
* does not contain the row yet.
|
||||||
|
*/
|
||||||
|
public class OracleUpsertOutputFormat<K extends SqoopRecord, V>
|
||||||
|
extends UpdateOutputFormat<K, V> {
|
||||||
|
|
||||||
|
private static final Log LOG =
|
||||||
|
LogFactory.getLog(OracleUpsertOutputFormat.class);
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
||||||
|
throws IOException {
|
||||||
|
try {
|
||||||
|
return new OracleUpsertRecordWriter(context);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new IOException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* RecordWriter to write the output to UPDATE/INSERT statements.
|
||||||
|
*/
|
||||||
|
public class OracleUpsertRecordWriter extends UpdateRecordWriter {
|
||||||
|
|
||||||
|
public OracleUpsertRecordWriter(TaskAttemptContext context)
|
||||||
|
throws ClassNotFoundException, SQLException {
|
||||||
|
super(context);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return an UPDATE/INSERT statement that modifies/inserts a row
|
||||||
|
* depending on whether the row already exist in the table or not.
|
||||||
|
*/
|
||||||
|
protected String getUpdateStatement() {
|
||||||
|
boolean first;
|
||||||
|
|
||||||
|
// lookup table for update columns
|
||||||
|
Set<String> updateKeyLookup = new LinkedHashSet<String>();
|
||||||
|
for (String updateKey : updateCols) {
|
||||||
|
updateKeyLookup.add(updateKey);
|
||||||
|
}
|
||||||
|
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
sb.append("MERGE INTO ");
|
||||||
|
sb.append(tableName);
|
||||||
|
sb.append(" USING dual ON ( ");
|
||||||
|
first = true;
|
||||||
|
for (int i = 0; i < updateCols.length; i++) {
|
||||||
|
if (first) {
|
||||||
|
first = false;
|
||||||
|
} else {
|
||||||
|
sb.append(" AND ");
|
||||||
|
}
|
||||||
|
sb.append(updateCols[i]).append(" = ?");
|
||||||
|
}
|
||||||
|
sb.append(" )");
|
||||||
|
|
||||||
|
sb.append(" WHEN MATCHED THEN UPDATE SET ");
|
||||||
|
first = true;
|
||||||
|
for (String col : columnNames) {
|
||||||
|
if (!updateKeyLookup.contains(col)) {
|
||||||
|
if (first) {
|
||||||
|
first = false;
|
||||||
|
} else {
|
||||||
|
sb.append(", ");
|
||||||
|
}
|
||||||
|
sb.append(col);
|
||||||
|
sb.append(" = ?");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
sb.append(" WHEN NOT MATCHED THEN INSERT ( ");
|
||||||
|
first = true;
|
||||||
|
for (String col : columnNames) {
|
||||||
|
if (first) {
|
||||||
|
first = false;
|
||||||
|
} else {
|
||||||
|
sb.append(", ");
|
||||||
|
}
|
||||||
|
sb.append(col);
|
||||||
|
}
|
||||||
|
sb.append(" ) VALUES ( ");
|
||||||
|
first = true;
|
||||||
|
for (String col : columnNames) {
|
||||||
|
if (first) {
|
||||||
|
first = false;
|
||||||
|
} else {
|
||||||
|
sb.append(", ");
|
||||||
|
}
|
||||||
|
sb.append("?");
|
||||||
|
}
|
||||||
|
sb.append(" )");
|
||||||
|
|
||||||
|
return sb.toString();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
105
src/java/org/apache/sqoop/mapreduce/RawKeyTextOutputFormat.java
Normal file
105
src/java/org/apache/sqoop/mapreduce/RawKeyTextOutputFormat.java
Normal file
@ -0,0 +1,105 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.DataOutputStream;
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.io.compress.CompressionCodec;
|
||||||
|
import org.apache.hadoop.io.compress.GzipCodec;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
||||||
|
import org.apache.hadoop.mapreduce.RecordWriter;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
import org.apache.hadoop.util.*;
|
||||||
|
|
||||||
|
/** An {@link OutputFormat} that writes plain text files.
|
||||||
|
* Only writes the key. Does not write any delimiter/newline after the key.
|
||||||
|
*/
|
||||||
|
public class RawKeyTextOutputFormat<K, V> extends FileOutputFormat<K, V> {
|
||||||
|
|
||||||
|
public static class RawKeyRecordWriter<K, V> extends RecordWriter<K, V> {
|
||||||
|
|
||||||
|
private static final String UTF8 = "UTF-8";
|
||||||
|
|
||||||
|
protected DataOutputStream out;
|
||||||
|
|
||||||
|
public RawKeyRecordWriter(DataOutputStream out) {
|
||||||
|
this.out = out;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the object to the byte stream, handling Text as a special
|
||||||
|
* case.
|
||||||
|
* @param o the object to print
|
||||||
|
* @throws IOException if the write throws, we pass it on
|
||||||
|
*/
|
||||||
|
private void writeObject(Object o) throws IOException {
|
||||||
|
if (o instanceof Text) {
|
||||||
|
Text to = (Text) o;
|
||||||
|
out.write(to.getBytes(), 0, to.getLength());
|
||||||
|
} else {
|
||||||
|
out.write(o.toString().getBytes(UTF8));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public synchronized void write(K key, V value) throws IOException {
|
||||||
|
writeObject(key);
|
||||||
|
}
|
||||||
|
|
||||||
|
public synchronized void close(TaskAttemptContext context)
|
||||||
|
throws IOException {
|
||||||
|
out.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
||||||
|
throws IOException {
|
||||||
|
boolean isCompressed = getCompressOutput(context);
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
String ext = "";
|
||||||
|
CompressionCodec codec = null;
|
||||||
|
|
||||||
|
if (isCompressed) {
|
||||||
|
// create the named codec
|
||||||
|
Class<? extends CompressionCodec> codecClass =
|
||||||
|
getOutputCompressorClass(context, GzipCodec.class);
|
||||||
|
codec = ReflectionUtils.newInstance(codecClass, conf);
|
||||||
|
|
||||||
|
ext = codec.getDefaultExtension();
|
||||||
|
}
|
||||||
|
|
||||||
|
Path file = getDefaultWorkFile(context, ext);
|
||||||
|
FileSystem fs = file.getFileSystem(conf);
|
||||||
|
FSDataOutputStream fileOut = fs.create(file, false);
|
||||||
|
DataOutputStream ostream = fileOut;
|
||||||
|
|
||||||
|
if (isCompressed) {
|
||||||
|
ostream = new DataOutputStream(codec.createOutputStream(fileOut));
|
||||||
|
}
|
||||||
|
|
||||||
|
return new RawKeyRecordWriter<K, V>(ostream);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,110 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import org.apache.hadoop.mapreduce.RecordWriter;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.mapreduce.ExportOutputFormat;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* SQLServer-specific SQL formatting overrides default ExportOutputFormat's.
|
||||||
|
*/
|
||||||
|
public class SQLServerExportOutputFormat<K extends SqoopRecord, V>
|
||||||
|
extends ExportOutputFormat<K, V> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
||||||
|
throws IOException {
|
||||||
|
try {
|
||||||
|
return new SQLServerExportRecordWriter(context);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new IOException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* RecordWriter to write the output to a row in a database table.
|
||||||
|
* The actual database updates are executed in a second thread.
|
||||||
|
*/
|
||||||
|
public class SQLServerExportRecordWriter extends ExportRecordWriter {
|
||||||
|
|
||||||
|
public SQLServerExportRecordWriter(TaskAttemptContext context)
|
||||||
|
throws ClassNotFoundException, SQLException {
|
||||||
|
super(context);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/**
|
||||||
|
* @return an INSERT statement suitable for inserting 'numRows' rows.
|
||||||
|
*/
|
||||||
|
protected String getInsertStatement(int numRows) {
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
|
||||||
|
sb.append("INSERT INTO " + getTableName() + " ");
|
||||||
|
|
||||||
|
int numSlots;
|
||||||
|
String [] colNames = getColumnNames();
|
||||||
|
if (colNames != null) {
|
||||||
|
numSlots = colNames.length;
|
||||||
|
|
||||||
|
sb.append("(");
|
||||||
|
boolean first = true;
|
||||||
|
for (String col : colNames) {
|
||||||
|
if (!first) {
|
||||||
|
sb.append(", ");
|
||||||
|
}
|
||||||
|
|
||||||
|
sb.append(col);
|
||||||
|
first = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
sb.append(") ");
|
||||||
|
} else {
|
||||||
|
numSlots = getColumnCount(); // set if columnNames is null.
|
||||||
|
}
|
||||||
|
|
||||||
|
// generates the (?, ?, ?...) used for each row.
|
||||||
|
StringBuilder sbRow = new StringBuilder();
|
||||||
|
sbRow.append("(SELECT ");
|
||||||
|
for (int i = 0; i < numSlots; i++) {
|
||||||
|
if (i != 0) {
|
||||||
|
sbRow.append(", ");
|
||||||
|
}
|
||||||
|
|
||||||
|
sbRow.append("?");
|
||||||
|
}
|
||||||
|
sbRow.append(") ");
|
||||||
|
|
||||||
|
// Now append that numRows times.
|
||||||
|
for (int i = 0; i < numRows; i++) {
|
||||||
|
if (i != 0) {
|
||||||
|
sb.append("UNION ALL ");
|
||||||
|
}
|
||||||
|
|
||||||
|
sb.append(sbRow);
|
||||||
|
}
|
||||||
|
|
||||||
|
return sb.toString();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.hadoop.io.LongWritable;
|
||||||
|
import org.apache.hadoop.io.NullWritable;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.mapreduce.AutoProgressMapper;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Reads a SqoopRecord from the SequenceFile in which it's packed and emits
|
||||||
|
* that DBWritable to the OutputFormat for writeback to the database.
|
||||||
|
*/
|
||||||
|
public class SequenceFileExportMapper
|
||||||
|
extends AutoProgressMapper<LongWritable, SqoopRecord, SqoopRecord,
|
||||||
|
NullWritable> {
|
||||||
|
|
||||||
|
public SequenceFileExportMapper() {
|
||||||
|
}
|
||||||
|
|
||||||
|
public void map(LongWritable key, SqoopRecord val, Context context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
context.write(val, NullWritable.get());
|
||||||
|
}
|
||||||
|
}
|
@ -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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import org.apache.hadoop.io.LongWritable;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
||||||
|
import com.cloudera.sqoop.lib.LargeObjectLoader;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.mapreduce.AutoProgressMapper;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Imports records by writing them to a SequenceFile.
|
||||||
|
*/
|
||||||
|
public class SequenceFileImportMapper
|
||||||
|
extends AutoProgressMapper<LongWritable, SqoopRecord, LongWritable,
|
||||||
|
SqoopRecord> {
|
||||||
|
|
||||||
|
private LargeObjectLoader lobLoader;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void setup(Context context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
this.lobLoader = new LargeObjectLoader(context.getConfiguration(),
|
||||||
|
FileOutputFormat.getWorkOutputPath(context));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void map(LongWritable key, SqoopRecord val, Context context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
|
||||||
|
try {
|
||||||
|
// Loading of LOBs was delayed until we have a Context.
|
||||||
|
val.loadLargeObjects(lobLoader);
|
||||||
|
} catch (SQLException sqlE) {
|
||||||
|
throw new IOException(sqlE);
|
||||||
|
}
|
||||||
|
|
||||||
|
context.write(key, val);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void cleanup(Context context) throws IOException {
|
||||||
|
if (null != lobLoader) {
|
||||||
|
lobLoader.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
83
src/java/org/apache/sqoop/mapreduce/TextExportMapper.java
Normal file
83
src/java/org/apache/sqoop/mapreduce/TextExportMapper.java
Normal file
@ -0,0 +1,83 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.io.LongWritable;
|
||||||
|
import org.apache.hadoop.io.NullWritable;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.util.ReflectionUtils;
|
||||||
|
import com.cloudera.sqoop.lib.RecordParser;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.mapreduce.AutoProgressMapper;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Converts an input record from a string representation to a parsed Sqoop
|
||||||
|
* record and emits that DBWritable to the OutputFormat for writeback to the
|
||||||
|
* database.
|
||||||
|
*/
|
||||||
|
public class TextExportMapper
|
||||||
|
extends AutoProgressMapper<LongWritable, Text, SqoopRecord, NullWritable> {
|
||||||
|
|
||||||
|
private SqoopRecord recordImpl;
|
||||||
|
|
||||||
|
public TextExportMapper() {
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void setup(Context context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
super.setup(context);
|
||||||
|
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
|
||||||
|
// Instantiate a copy of the user's class to hold and parse the record.
|
||||||
|
String recordClassName = conf.get(
|
||||||
|
ExportJobBase.SQOOP_EXPORT_TABLE_CLASS_KEY);
|
||||||
|
if (null == recordClassName) {
|
||||||
|
throw new IOException("Export table class name ("
|
||||||
|
+ ExportJobBase.SQOOP_EXPORT_TABLE_CLASS_KEY
|
||||||
|
+ ") is not set!");
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
Class cls = Class.forName(recordClassName, true,
|
||||||
|
Thread.currentThread().getContextClassLoader());
|
||||||
|
recordImpl = (SqoopRecord) ReflectionUtils.newInstance(cls, conf);
|
||||||
|
} catch (ClassNotFoundException cnfe) {
|
||||||
|
throw new IOException(cnfe);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (null == recordImpl) {
|
||||||
|
throw new IOException("Could not instantiate object of type "
|
||||||
|
+ recordClassName);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
public void map(LongWritable key, Text val, Context context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
try {
|
||||||
|
recordImpl.parse(val);
|
||||||
|
context.write(recordImpl, NullWritable.get());
|
||||||
|
} catch (RecordParser.ParseError pe) {
|
||||||
|
throw new IOException("Could not parse record: " + val, pe);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
74
src/java/org/apache/sqoop/mapreduce/TextImportMapper.java
Normal file
74
src/java/org/apache/sqoop/mapreduce/TextImportMapper.java
Normal file
@ -0,0 +1,74 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
|
||||||
|
import org.apache.hadoop.io.LongWritable;
|
||||||
|
import org.apache.hadoop.io.NullWritable;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
||||||
|
import com.cloudera.sqoop.lib.LargeObjectLoader;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.mapreduce.AutoProgressMapper;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Imports records by transforming them to strings for a plain-text flat file.
|
||||||
|
*/
|
||||||
|
public class TextImportMapper
|
||||||
|
extends AutoProgressMapper<LongWritable, SqoopRecord, Text, NullWritable> {
|
||||||
|
|
||||||
|
private Text outkey;
|
||||||
|
private LargeObjectLoader lobLoader;
|
||||||
|
|
||||||
|
public TextImportMapper() {
|
||||||
|
outkey = new Text();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void setup(Context context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
this.lobLoader = new LargeObjectLoader(context.getConfiguration(),
|
||||||
|
FileOutputFormat.getWorkOutputPath(context));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void map(LongWritable key, SqoopRecord val, Context context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
|
||||||
|
try {
|
||||||
|
// Loading of LOBs was delayed until we have a Context.
|
||||||
|
val.loadLargeObjects(lobLoader);
|
||||||
|
} catch (SQLException sqlE) {
|
||||||
|
throw new IOException(sqlE);
|
||||||
|
}
|
||||||
|
|
||||||
|
outkey.set(val.toString());
|
||||||
|
context.write(outkey, NullWritable.get());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void cleanup(Context context) throws IOException {
|
||||||
|
if (null != lobLoader) {
|
||||||
|
lobLoader.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
212
src/java/org/apache/sqoop/mapreduce/UpdateOutputFormat.java
Normal file
212
src/java/org/apache/sqoop/mapreduce/UpdateOutputFormat.java
Normal file
@ -0,0 +1,212 @@
|
|||||||
|
/**
|
||||||
|
* 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.mapreduce;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.sql.Connection;
|
||||||
|
import java.sql.PreparedStatement;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.LinkedHashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.StringTokenizer;
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.mapreduce.JobContext;
|
||||||
|
import org.apache.hadoop.mapreduce.RecordWriter;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||||
|
import com.cloudera.sqoop.lib.SqoopRecord;
|
||||||
|
import com.cloudera.sqoop.mapreduce.AsyncSqlOutputFormat;
|
||||||
|
import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Update an existing table of data with new value data.
|
||||||
|
* This requires a designated 'key column' for the WHERE clause
|
||||||
|
* of an UPDATE statement.
|
||||||
|
*
|
||||||
|
* Updates are executed en batch in the PreparedStatement.
|
||||||
|
*
|
||||||
|
* Uses DBOutputFormat/DBConfiguration for configuring the output.
|
||||||
|
*/
|
||||||
|
public class UpdateOutputFormat<K extends SqoopRecord, V>
|
||||||
|
extends AsyncSqlOutputFormat<K, V> {
|
||||||
|
|
||||||
|
private static final Log LOG = LogFactory.getLog(UpdateOutputFormat.class);
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public void checkOutputSpecs(JobContext context)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
Configuration conf = context.getConfiguration();
|
||||||
|
DBConfiguration dbConf = new DBConfiguration(conf);
|
||||||
|
|
||||||
|
// Sanity check all the configuration values we need.
|
||||||
|
if (null == conf.get(DBConfiguration.URL_PROPERTY)) {
|
||||||
|
throw new IOException("Database connection URL is not set.");
|
||||||
|
} else if (null == dbConf.getOutputTableName()) {
|
||||||
|
throw new IOException("Table name is not set for export.");
|
||||||
|
} else if (null == dbConf.getOutputFieldNames()) {
|
||||||
|
throw new IOException(
|
||||||
|
"Output field names are null.");
|
||||||
|
} else if (null == conf.get(ExportJobBase.SQOOP_EXPORT_UPDATE_COL_KEY)) {
|
||||||
|
throw new IOException("Update key column is not set for export.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
|
||||||
|
throws IOException {
|
||||||
|
try {
|
||||||
|
return new UpdateRecordWriter(context);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new IOException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* RecordWriter to write the output to UPDATE statements modifying rows
|
||||||
|
* in the database.
|
||||||
|
*/
|
||||||
|
public class UpdateRecordWriter extends AsyncSqlRecordWriter<K, V> {
|
||||||
|
|
||||||
|
protected String tableName;
|
||||||
|
protected String [] columnNames; // The columns to update.
|
||||||
|
protected String [] updateCols; // The columns containing the fixed key.
|
||||||
|
|
||||||
|
public UpdateRecordWriter(TaskAttemptContext context)
|
||||||
|
throws ClassNotFoundException, SQLException {
|
||||||
|
super(context);
|
||||||
|
|
||||||
|
Configuration conf = getConf();
|
||||||
|
|
||||||
|
DBConfiguration dbConf = new DBConfiguration(conf);
|
||||||
|
this.tableName = dbConf.getOutputTableName();
|
||||||
|
this.columnNames = dbConf.getOutputFieldNames();
|
||||||
|
String updateKeyColumns =
|
||||||
|
conf.get(ExportJobBase.SQOOP_EXPORT_UPDATE_COL_KEY);
|
||||||
|
|
||||||
|
Set<String> updateKeys = new LinkedHashSet<String>();
|
||||||
|
StringTokenizer stok = new StringTokenizer(updateKeyColumns, ",");
|
||||||
|
while (stok.hasMoreTokens()) {
|
||||||
|
String nextUpdateKey = stok.nextToken().trim();
|
||||||
|
if (nextUpdateKey.length() > 0) {
|
||||||
|
updateKeys.add(nextUpdateKey);
|
||||||
|
} else {
|
||||||
|
throw new RuntimeException("Invalid update key column value specified"
|
||||||
|
+ ": '" + updateKeyColumns + "'");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
updateCols = updateKeys.toArray(new String[updateKeys.size()]);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
protected boolean isBatchExec() {
|
||||||
|
// We use batches here.
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the name of the table we are inserting into.
|
||||||
|
*/
|
||||||
|
protected final String getTableName() {
|
||||||
|
return tableName;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the list of columns we are updating.
|
||||||
|
*/
|
||||||
|
protected final String [] getColumnNames() {
|
||||||
|
if (null == columnNames) {
|
||||||
|
return null;
|
||||||
|
} else {
|
||||||
|
return Arrays.copyOf(columnNames, columnNames.length);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the column we are using to determine the row to update.
|
||||||
|
*/
|
||||||
|
protected final String[] getUpdateColumns() {
|
||||||
|
return updateCols;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
protected PreparedStatement getPreparedStatement(
|
||||||
|
List<SqoopRecord> userRecords) throws SQLException {
|
||||||
|
|
||||||
|
PreparedStatement stmt = null;
|
||||||
|
|
||||||
|
// Synchronize on connection to ensure this does not conflict
|
||||||
|
// with the operations in the update thread.
|
||||||
|
Connection conn = getConnection();
|
||||||
|
synchronized (conn) {
|
||||||
|
stmt = conn.prepareStatement(getUpdateStatement());
|
||||||
|
}
|
||||||
|
|
||||||
|
// Inject the record parameters into the UPDATE and WHERE clauses. This
|
||||||
|
// assumes that the update key column is the last column serialized in
|
||||||
|
// by the underlying record. Our code auto-gen process for exports was
|
||||||
|
// responsible for taking care of this constraint.
|
||||||
|
for (SqoopRecord record : userRecords) {
|
||||||
|
record.write(stmt, 0);
|
||||||
|
stmt.addBatch();
|
||||||
|
}
|
||||||
|
|
||||||
|
return stmt;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return an UPDATE statement that modifies rows based on a single key
|
||||||
|
* column (with the intent of modifying a single row).
|
||||||
|
*/
|
||||||
|
protected String getUpdateStatement() {
|
||||||
|
StringBuilder sb = new StringBuilder();
|
||||||
|
sb.append("UPDATE " + this.tableName + " SET ");
|
||||||
|
|
||||||
|
boolean first = true;
|
||||||
|
for (String col : this.columnNames) {
|
||||||
|
if (!first) {
|
||||||
|
sb.append(", ");
|
||||||
|
}
|
||||||
|
|
||||||
|
sb.append(col);
|
||||||
|
sb.append("=?");
|
||||||
|
first = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
sb.append(" WHERE ");
|
||||||
|
first = true;
|
||||||
|
for (int i = 0; i < updateCols.length; i++) {
|
||||||
|
if (first) {
|
||||||
|
first = false;
|
||||||
|
} else {
|
||||||
|
sb.append(" AND ");
|
||||||
|
}
|
||||||
|
sb.append(updateCols[i]).append("=?");
|
||||||
|
}
|
||||||
|
return sb.toString();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue
Block a user