From 37831c6125fe41f350b9a1bccd8e475985360445 Mon Sep 17 00:00:00 2001 From: Arvind Prabhakar Date: Fri, 28 Oct 2011 18:22:16 +0000 Subject: [PATCH] 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 --- .../sqoop/mapreduce/AsyncSqlOutputFormat.java | 284 +--------- .../sqoop/mapreduce/AsyncSqlRecordWriter.java | 196 +------ .../sqoop/mapreduce/AutoProgressMapper.java | 187 +------ .../sqoop/mapreduce/AvroExportMapper.java | 185 +------ .../sqoop/mapreduce/AvroImportMapper.java | 82 +-- .../sqoop/mapreduce/AvroInputFormat.java | 42 +- .../com/cloudera/sqoop/mapreduce/AvroJob.java | 17 +- .../sqoop/mapreduce/AvroOutputFormat.java | 49 +- .../sqoop/mapreduce/AvroRecordReader.java | 88 +--- .../mapreduce/CombineShimRecordReader.java | 104 +--- .../sqoop/mapreduce/DataDrivenImportJob.java | 175 +------ .../mapreduce/DelegatingOutputFormat.java | 112 +--- .../mapreduce/ExportBatchOutputFormat.java | 119 +---- .../sqoop/mapreduce/ExportInputFormat.java | 95 +--- .../sqoop/mapreduce/ExportJobBase.java | 368 +------------ .../sqoop/mapreduce/ExportOutputFormat.java | 186 +------ .../sqoop/mapreduce/HBaseImportJob.java | 149 +----- .../sqoop/mapreduce/HBaseImportMapper.java | 20 +- .../sqoop/mapreduce/ImportJobBase.java | 170 +----- .../sqoop/mapreduce/JdbcExportJob.java | 104 +--- .../sqoop/mapreduce/JdbcUpdateExportJob.java | 110 +--- .../sqoop/mapreduce/JdbcUpsertExportJob.java | 79 +-- .../com/cloudera/sqoop/mapreduce/JobBase.java | 300 +---------- .../cloudera/sqoop/mapreduce/MergeJob.java | 127 +---- .../sqoop/mapreduce/MergeMapperBase.java | 69 +-- .../cloudera/sqoop/mapreduce/MergeRecord.java | 109 +--- .../sqoop/mapreduce/MergeRecordMapper.java | 17 +- .../sqoop/mapreduce/MergeReducer.java | 37 +- .../sqoop/mapreduce/MergeTextMapper.java | 43 +- .../sqoop/mapreduce/MySQLDumpImportJob.java | 107 +--- .../sqoop/mapreduce/MySQLDumpInputFormat.java | 79 +-- .../sqoop/mapreduce/MySQLDumpMapper.java | 464 +--------------- .../sqoop/mapreduce/MySQLExportJob.java | 93 +--- .../sqoop/mapreduce/MySQLExportMapper.java | 338 +----------- .../mapreduce/MySQLRecordExportMapper.java | 29 +- .../mapreduce/MySQLTextExportMapper.java | 43 +- .../sqoop/mapreduce/NullOutputCommitter.java | 25 +- .../mapreduce/OracleExportOutputFormat.java | 89 +--- .../mapreduce/OracleUpsertOutputFormat.java | 110 +--- .../mapreduce/RawKeyTextOutputFormat.java | 84 +-- .../SQLServerExportOutputFormat.java | 89 +--- .../mapreduce/SequenceFileExportMapper.java | 24 +- .../mapreduce/SequenceFileImportMapper.java | 46 +- .../sqoop/mapreduce/TextExportMapper.java | 65 +-- .../sqoop/mapreduce/TextImportMapper.java | 53 +- .../sqoop/mapreduce/UpdateOutputFormat.java | 191 +------ .../sqoop/mapreduce/AsyncSqlOutputFormat.java | 304 +++++++++++ .../sqoop/mapreduce/AsyncSqlRecordWriter.java | 223 ++++++++ .../sqoop/mapreduce/AutoProgressMapper.java | 199 +++++++ .../sqoop/mapreduce/AvroExportMapper.java | 204 ++++++++ .../sqoop/mapreduce/AvroImportMapper.java | 101 ++++ .../sqoop/mapreduce/AvroInputFormat.java | 58 ++ .../org/apache/sqoop/mapreduce/AvroJob.java | 41 ++ .../sqoop/mapreduce/AvroOutputFormat.java | 64 +++ .../sqoop/mapreduce/AvroRecordReader.java | 104 ++++ .../mapreduce/CombineShimRecordReader.java | 131 +++++ .../sqoop/mapreduce/DataDrivenImportJob.java | 206 ++++++++ .../mapreduce/DelegatingOutputFormat.java | 134 +++++ .../mapreduce/ExportBatchOutputFormat.java | 139 +++++ .../sqoop/mapreduce/ExportInputFormat.java | 119 +++++ .../apache/sqoop/mapreduce/ExportJobBase.java | 410 +++++++++++++++ .../sqoop/mapreduce/ExportOutputFormat.java | 207 ++++++++ .../sqoop/mapreduce/HBaseImportJob.java | 178 +++++++ .../sqoop/mapreduce/HBaseImportMapper.java | 41 ++ .../apache/sqoop/mapreduce/ImportJobBase.java | 203 +++++++ .../apache/sqoop/mapreduce/JdbcExportJob.java | 139 +++++ .../sqoop/mapreduce/JdbcUpdateExportJob.java | 145 +++++ .../sqoop/mapreduce/JdbcUpsertExportJob.java | 106 ++++ .../org/apache/sqoop/mapreduce/JobBase.java | 325 ++++++++++++ .../org/apache/sqoop/mapreduce/MergeJob.java | 145 +++++ .../sqoop/mapreduce/MergeMapperBase.java | 85 +++ .../apache/sqoop/mapreduce/MergeRecord.java | 131 +++++ .../sqoop/mapreduce/MergeRecordMapper.java | 37 ++ .../apache/sqoop/mapreduce/MergeReducer.java | 56 ++ .../sqoop/mapreduce/MergeTextMapper.java | 60 +++ .../sqoop/mapreduce/MySQLDumpImportJob.java | 130 +++++ .../sqoop/mapreduce/MySQLDumpInputFormat.java | 98 ++++ .../sqoop/mapreduce/MySQLDumpMapper.java | 494 ++++++++++++++++++ .../sqoop/mapreduce/MySQLExportJob.java | 114 ++++ .../sqoop/mapreduce/MySQLExportMapper.java | 357 +++++++++++++ .../mapreduce/MySQLRecordExportMapper.java | 49 ++ .../mapreduce/MySQLTextExportMapper.java | 63 +++ .../sqoop/mapreduce/NullOutputCommitter.java | 45 ++ .../mapreduce/OracleExportOutputFormat.java | 110 ++++ .../mapreduce/OracleUpsertOutputFormat.java | 131 +++++ .../mapreduce/RawKeyTextOutputFormat.java | 105 ++++ .../SQLServerExportOutputFormat.java | 110 ++++ .../mapreduce/SequenceFileExportMapper.java | 42 ++ .../mapreduce/SequenceFileImportMapper.java | 66 +++ .../sqoop/mapreduce/TextExportMapper.java | 83 +++ .../sqoop/mapreduce/TextImportMapper.java | 74 +++ .../sqoop/mapreduce/UpdateOutputFormat.java | 212 ++++++++ 92 files changed, 6797 insertions(+), 5333 deletions(-) create mode 100644 src/java/org/apache/sqoop/mapreduce/AsyncSqlOutputFormat.java create mode 100644 src/java/org/apache/sqoop/mapreduce/AsyncSqlRecordWriter.java create mode 100644 src/java/org/apache/sqoop/mapreduce/AutoProgressMapper.java create mode 100644 src/java/org/apache/sqoop/mapreduce/AvroExportMapper.java create mode 100644 src/java/org/apache/sqoop/mapreduce/AvroImportMapper.java create mode 100644 src/java/org/apache/sqoop/mapreduce/AvroInputFormat.java create mode 100644 src/java/org/apache/sqoop/mapreduce/AvroJob.java create mode 100644 src/java/org/apache/sqoop/mapreduce/AvroOutputFormat.java create mode 100644 src/java/org/apache/sqoop/mapreduce/AvroRecordReader.java create mode 100644 src/java/org/apache/sqoop/mapreduce/CombineShimRecordReader.java create mode 100644 src/java/org/apache/sqoop/mapreduce/DataDrivenImportJob.java create mode 100644 src/java/org/apache/sqoop/mapreduce/DelegatingOutputFormat.java create mode 100644 src/java/org/apache/sqoop/mapreduce/ExportBatchOutputFormat.java create mode 100644 src/java/org/apache/sqoop/mapreduce/ExportInputFormat.java create mode 100644 src/java/org/apache/sqoop/mapreduce/ExportJobBase.java create mode 100644 src/java/org/apache/sqoop/mapreduce/ExportOutputFormat.java create mode 100644 src/java/org/apache/sqoop/mapreduce/HBaseImportJob.java create mode 100644 src/java/org/apache/sqoop/mapreduce/HBaseImportMapper.java create mode 100644 src/java/org/apache/sqoop/mapreduce/ImportJobBase.java create mode 100644 src/java/org/apache/sqoop/mapreduce/JdbcExportJob.java create mode 100644 src/java/org/apache/sqoop/mapreduce/JdbcUpdateExportJob.java create mode 100644 src/java/org/apache/sqoop/mapreduce/JdbcUpsertExportJob.java create mode 100644 src/java/org/apache/sqoop/mapreduce/JobBase.java create mode 100644 src/java/org/apache/sqoop/mapreduce/MergeJob.java create mode 100644 src/java/org/apache/sqoop/mapreduce/MergeMapperBase.java create mode 100644 src/java/org/apache/sqoop/mapreduce/MergeRecord.java create mode 100644 src/java/org/apache/sqoop/mapreduce/MergeRecordMapper.java create mode 100644 src/java/org/apache/sqoop/mapreduce/MergeReducer.java create mode 100644 src/java/org/apache/sqoop/mapreduce/MergeTextMapper.java create mode 100644 src/java/org/apache/sqoop/mapreduce/MySQLDumpImportJob.java create mode 100644 src/java/org/apache/sqoop/mapreduce/MySQLDumpInputFormat.java create mode 100644 src/java/org/apache/sqoop/mapreduce/MySQLDumpMapper.java create mode 100644 src/java/org/apache/sqoop/mapreduce/MySQLExportJob.java create mode 100644 src/java/org/apache/sqoop/mapreduce/MySQLExportMapper.java create mode 100644 src/java/org/apache/sqoop/mapreduce/MySQLRecordExportMapper.java create mode 100644 src/java/org/apache/sqoop/mapreduce/MySQLTextExportMapper.java create mode 100644 src/java/org/apache/sqoop/mapreduce/NullOutputCommitter.java create mode 100644 src/java/org/apache/sqoop/mapreduce/OracleExportOutputFormat.java create mode 100644 src/java/org/apache/sqoop/mapreduce/OracleUpsertOutputFormat.java create mode 100644 src/java/org/apache/sqoop/mapreduce/RawKeyTextOutputFormat.java create mode 100644 src/java/org/apache/sqoop/mapreduce/SQLServerExportOutputFormat.java create mode 100644 src/java/org/apache/sqoop/mapreduce/SequenceFileExportMapper.java create mode 100644 src/java/org/apache/sqoop/mapreduce/SequenceFileImportMapper.java create mode 100644 src/java/org/apache/sqoop/mapreduce/TextExportMapper.java create mode 100644 src/java/org/apache/sqoop/mapreduce/TextImportMapper.java create mode 100644 src/java/org/apache/sqoop/mapreduce/UpdateOutputFormat.java diff --git a/src/java/com/cloudera/sqoop/mapreduce/AsyncSqlOutputFormat.java b/src/java/com/cloudera/sqoop/mapreduce/AsyncSqlOutputFormat.java index e1672ba3..ccd2be5a 100755 --- a/src/java/com/cloudera/sqoop/mapreduce/AsyncSqlOutputFormat.java +++ b/src/java/com/cloudera/sqoop/mapreduce/AsyncSqlOutputFormat.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,289 +18,65 @@ package com.cloudera.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. + * @deprecated Moving to use org.apache.sqoop namespace. */ public abstract class AsyncSqlOutputFormat - extends OutputFormat { + extends org.apache.sqoop.mapreduce.AsyncSqlOutputFormat { - /** conf key: number of rows to export per INSERT statement. */ 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 = - "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 - * other batched update statement. + * @deprecated Moving to use org.apache.sqoop namespace. */ - 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, 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, boolean commit, boolean stopThread) { - this.stmt = s; - this.isBatch = batch; - this.commit = commit; - this.stopThread = stopThread; + super(s, batch, commit, 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. + * @deprecated Moving to use org.apache.sqoop namespace. */ - 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 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(); - this.stmtsPerTx = stmtsPerTx; + super(conn, 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)); - } - } } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/AsyncSqlRecordWriter.java b/src/java/com/cloudera/sqoop/mapreduce/AsyncSqlRecordWriter.java index d757e464..265dc270 100755 --- a/src/java/com/cloudera/sqoop/mapreduce/AsyncSqlRecordWriter.java +++ b/src/java/com/cloudera/sqoop/mapreduce/AsyncSqlRecordWriter.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,207 +18,19 @@ package com.cloudera.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 com.cloudera.sqoop.mapreduce.db.DBConfiguration; -import com.cloudera.sqoop.util.LoggingUtils; 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. + * @deprecated Moving to use org.apache.sqoop namespace. */ public abstract class AsyncSqlRecordWriter - extends RecordWriter { - - 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 records; - - // Background thread to actually perform the updates. - private AsyncSqlOutputFormat.AsyncSqlExecThread execThread; - private boolean startedExecThread; + extends org.apache.sqoop.mapreduce.AsyncSqlRecordWriter { 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(this.rowsPerStmt); - - this.execThread = new AsyncSqlOutputFormat.AsyncSqlExecThread( - connection, stmtsPerTx); - this.execThread.setDaemon(true); - this.startedExecThread = false; + super(context); } - /** - * 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 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); - } - } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/AutoProgressMapper.java b/src/java/com/cloudera/sqoop/mapreduce/AutoProgressMapper.java index 5329c8a7..ee8ab6dd 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/AutoProgressMapper.java +++ b/src/java/com/cloudera/sqoop/mapreduce/AutoProgressMapper.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,186 +18,25 @@ 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 - extends Mapper { - - 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; + extends org.apache.sqoop.mapreduce.AutoProgressMapper + { 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 = - "sqoop.mapred.auto.progress.sleep"; + org.apache.sqoop.mapreduce.AutoProgressMapper.SLEEP_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. - static final int DEFAULT_SLEEP_INTERVAL = 10000; + public static final int DEFAULT_SLEEP_INTERVAL = + 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()); - } - } - } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/AvroExportMapper.java b/src/java/com/cloudera/sqoop/mapreduce/AvroExportMapper.java index 40c314ae..725b9d06 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/AvroExportMapper.java +++ b/src/java/com/cloudera/sqoop/mapreduce/AvroExportMapper.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,188 +18,13 @@ 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 - extends AutoProgressMapper, NullWritable, - SqoopRecord, NullWritable> { + extends org.apache.sqoop.mapreduce.AvroExportMapper { - 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"; - - 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 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 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 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()); - } - } + public static final String AVRO_COLUMN_TYPES_MAP = + org.apache.sqoop.mapreduce.AvroExportMapper.AVRO_COLUMN_TYPES_MAP; } diff --git a/src/java/com/cloudera/sqoop/mapreduce/AvroImportMapper.java b/src/java/com/cloudera/sqoop/mapreduce/AvroImportMapper.java index 14e18d52..08b57889 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/AvroImportMapper.java +++ b/src/java/com/cloudera/sqoop/mapreduce/AvroImportMapper.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,85 +18,9 @@ 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 - extends AutoProgressMapper, NullWritable> { - - private final AvroWrapper wrapper = - new AvroWrapper(); - 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 fieldMap = val.getFieldMap(); - GenericRecord record = new GenericData.Record(schema); - for (Map.Entry 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; - } - - + extends org.apache.sqoop.mapreduce.AvroImportMapper { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/AvroInputFormat.java b/src/java/com/cloudera/sqoop/mapreduce/AvroInputFormat.java index de1c207c..2491012b 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/AvroInputFormat.java +++ b/src/java/com/cloudera/sqoop/mapreduce/AvroInputFormat.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,42 +18,10 @@ package com.cloudera.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. */ +/** + * @deprecated Moving to use org.apache.sqoop namespace. + */ public class AvroInputFormat - extends FileInputFormat, NullWritable> { - - @Override - protected List listStatus(JobContext job) throws IOException { - List result = new ArrayList(); - 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, NullWritable> createRecordReader( - InputSplit split, TaskAttemptContext context) throws IOException, - InterruptedException { - context.setStatus(split.toString()); - return new AvroRecordReader(); - } - + extends org.apache.sqoop.mapreduce.AvroInputFormat { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/AvroJob.java b/src/java/com/cloudera/sqoop/mapreduce/AvroJob.java index 0b72994f..1ba22a5b 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/AvroJob.java +++ b/src/java/com/cloudera/sqoop/mapreduce/AvroJob.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -24,20 +22,21 @@ 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 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) { - 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) { - return Schema.parse(job.get(MAP_OUTPUT_SCHEMA)); + return org.apache.sqoop.mapreduce.AvroJob.getMapOutputSchema(job); } + } diff --git a/src/java/com/cloudera/sqoop/mapreduce/AvroOutputFormat.java b/src/java/com/cloudera/sqoop/mapreduce/AvroOutputFormat.java index b03ab916..40b4368d 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/AvroOutputFormat.java +++ b/src/java/com/cloudera/sqoop/mapreduce/AvroOutputFormat.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,48 +18,9 @@ package com.cloudera.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. */ +/** + * @deprecated Moving to use org.apache.sqoop namespace. + */ public class AvroOutputFormat - extends FileOutputFormat, NullWritable> { - - @Override - public RecordWriter, NullWritable> getRecordWriter( - TaskAttemptContext context) throws IOException, InterruptedException { - - Schema schema = AvroJob.getMapOutputSchema(context.getConfiguration()); - - final DataFileWriter WRITER = - new DataFileWriter(new GenericDatumWriter()); - - Path path = getDefaultWorkFile(context, - org.apache.avro.mapred.AvroOutputFormat.EXT); - WRITER.create(schema, - path.getFileSystem(context.getConfiguration()).create(path)); - - return new RecordWriter, NullWritable>() { - @Override - public void write(AvroWrapper wrapper, NullWritable ignore) - throws IOException { - WRITER.append(wrapper.datum()); - } - @Override - public void close(TaskAttemptContext context) throws IOException, - InterruptedException { - WRITER.close(); - } - }; - } - + extends org.apache.sqoop.mapreduce.AvroOutputFormat { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/AvroRecordReader.java b/src/java/com/cloudera/sqoop/mapreduce/AvroRecordReader.java index 6a1fb8bc..8f341280 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/AvroRecordReader.java +++ b/src/java/com/cloudera/sqoop/mapreduce/AvroRecordReader.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,88 +18,10 @@ package com.cloudera.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. */ +/** + * @deprecated Moving to use org.apache.sqoop namespace. + */ public class AvroRecordReader - extends RecordReader, NullWritable> { - - private FileReader reader; - private long start; - private long end; - private AvroWrapper 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 datumReader = new GenericDatumReader(); - 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(); - } - if (value == null) { - value = NullWritable.get(); - } - key.datum(reader.next(key.datum())); - return true; - } - - @Override - public AvroWrapper 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(); } + extends org.apache.sqoop.mapreduce.AvroRecordReader { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/CombineShimRecordReader.java b/src/java/com/cloudera/sqoop/mapreduce/CombineShimRecordReader.java index 1acd3358..f1cad2a9 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/CombineShimRecordReader.java +++ b/src/java/com/cloudera/sqoop/mapreduce/CombineShimRecordReader.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -21,115 +19,19 @@ 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.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. + * @deprecated Moving to use org.apache.sqoop namespace. */ public class CombineShimRecordReader - extends RecordReader { + 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 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(); + super(split, context, index); } - @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) - ReflectionUtils.newInstance(rrClass, conf); - } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/DataDrivenImportJob.java b/src/java/com/cloudera/sqoop/mapreduce/DataDrivenImportJob.java index d5c66dd2..43a9213e 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/DataDrivenImportJob.java +++ b/src/java/com/cloudera/sqoop/mapreduce/DataDrivenImportJob.java @@ -21,189 +21,24 @@ package com.cloudera.sqoop.mapreduce; 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.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.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 - * sqoop.orm package. Uses DataDrivenDBInputFormat. + * @deprecated Moving to use org.apache.sqoop namespace. */ -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) { - super(opts, null, DataDrivenDBInputFormat.class, null, null); + super(opts); } public DataDrivenImportJob(final SqoopOptions opts, final Class inputFormatClass, 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 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 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); - } - } - } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/DelegatingOutputFormat.java b/src/java/com/cloudera/sqoop/mapreduce/DelegatingOutputFormat.java index a480f73b..575dc7e3 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/DelegatingOutputFormat.java +++ b/src/java/com/cloudera/sqoop/mapreduce/DelegatingOutputFormat.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,119 +18,15 @@ 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.FieldMapProcessor; -import com.cloudera.sqoop.lib.ProcessingException; /** - * OutputFormat that produces a RecordReader which instantiates - * a FieldMapProcessor which will process FieldMappable - * output keys. - * - *

The output value is ignored.

- * - *

The FieldMapProcessor implementation may do any arbitrary - * processing on the object. For example, it may write an object - * to HBase, etc.

- * - *

If the FieldMapProcessor implementation also implements - * Closeable, it will be close()'d in the RecordReader's close() - * method.

- * - *

If the FMP implements Configurable, it will be configured - * correctly via ReflectionUtils.

+ * @deprecated Moving to use org.apache.sqoop namespace. */ public class DelegatingOutputFormat - extends OutputFormat { + extends org.apache.sqoop.mapreduce.DelegatingOutputFormat { - /** conf key: the FieldMapProcessor class to instantiate. */ 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 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 { - - private Configuration conf; - - private FieldMapProcessor mapProcessor; - - public DelegatingRecordWriter(TaskAttemptContext context) - throws ClassNotFoundException { - - this.conf = context.getConfiguration(); - - @SuppressWarnings("unchecked") - Class procClass = - (Class) - 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); - } - } - } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/ExportBatchOutputFormat.java b/src/java/com/cloudera/sqoop/mapreduce/ExportBatchOutputFormat.java index 6f142ac0..f969caa0 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/ExportBatchOutputFormat.java +++ b/src/java/com/cloudera/sqoop/mapreduce/ExportBatchOutputFormat.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,123 +18,12 @@ 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; /** - * This class uses batch mode to execute underlying statements instead of - * using a single multirow insert statement as its superclass. + * @deprecated Moving to use org.apache.sqoop namespace. */ public class ExportBatchOutputFormat - extends ExportOutputFormat { - - private static final Log LOG = - LogFactory.getLog(ExportBatchOutputFormat.class); - - @Override - /** {@inheritDoc} */ - public RecordWriter 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 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(); - } - } + extends org.apache.sqoop.mapreduce.ExportBatchOutputFormat + { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/ExportInputFormat.java b/src/java/com/cloudera/sqoop/mapreduce/ExportInputFormat.java index b210d5ee..e0ad15b6 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/ExportInputFormat.java +++ b/src/java/com/cloudera/sqoop/mapreduce/ExportInputFormat.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,104 +18,23 @@ 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.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. + * @deprecated Moving to use org.apache.sqoop namespace. */ public class ExportInputFormat - extends CombineFileInputFormat { + extends org.apache.sqoop.mapreduce.ExportInputFormat { - public static final Log LOG = - LogFactory.getLog(ExportInputFormat.class.getName()); + public static final int DEFAULT_NUM_MAP_TASKS = + 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 stats = listStatus(job); - long count = 0; - for (FileStatus stat : stats) { - count += stat.getLen(); - } - - return count; - } - - @Override - public List 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 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); + 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) { - return job.getConfiguration().getInt(ExportJobBase.EXPORT_MAP_TASKS_KEY, - DEFAULT_NUM_MAP_TASKS); + return org.apache.sqoop.mapreduce.ExportInputFormat.getNumMapTasks(job); } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/ExportJobBase.java b/src/java/com/cloudera/sqoop/mapreduce/ExportJobBase.java index 31099113..026b5e49 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/ExportJobBase.java +++ b/src/java/com/cloudera/sqoop/mapreduce/ExportJobBase.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,394 +18,48 @@ package com.cloudera.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 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.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 = - "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 = - "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 = - "sqoop.mapreduce.export.map.tasks"; - - protected ExportJobContext context; + org.apache.sqoop.mapreduce.ExportJobBase.EXPORT_MAP_TASKS_KEY; public ExportJobBase(final ExportJobContext ctxt) { - this(ctxt, null, null, null); + super(ctxt); } public ExportJobBase(final ExportJobContext ctxt, final Class mapperClass, final Class inputFormatClass, final Class outputFormatClass) { - super(ctxt.getOptions(), mapperClass, inputFormatClass, outputFormatClass); - this.context = ctxt; + super(ctxt, mapperClass, inputFormatClass, outputFormatClass); } - /** - * @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 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) 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! - } + return org.apache.sqoop.mapreduce.ExportJobBase.getFileType(conf, p); } - /** - * @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 getInputFormatClass() - throws ClassNotFoundException { - Class configuredIF = super.getInputFormatClass(); - if (null == configuredIF) { - return ExportInputFormat.class; - } else { - return configuredIF; - } - } - - @Override - protected Class getOutputFormatClass() - throws ClassNotFoundException { - Class 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; - } - } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/ExportOutputFormat.java b/src/java/com/cloudera/sqoop/mapreduce/ExportOutputFormat.java index f5a4d23f..dba4a496 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/ExportOutputFormat.java +++ b/src/java/com/cloudera/sqoop/mapreduce/ExportOutputFormat.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,191 +18,11 @@ 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; /** - * 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. + * @deprecated Moving to use org.apache.sqoop namespace. */ public class ExportOutputFormat - extends AsyncSqlOutputFormat { - - 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 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 { - - 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 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(); - } - } + extends org.apache.sqoop.mapreduce.ExportOutputFormat { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/HBaseImportJob.java b/src/java/com/cloudera/sqoop/mapreduce/HBaseImportJob.java index e31a1bba..b3d8ec61 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/HBaseImportJob.java +++ b/src/java/com/cloudera/sqoop/mapreduce/HBaseImportJob.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,163 +18,24 @@ 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.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.util.ImportException; /** - * Runs an HBase import via DataDrivenDBInputFormat to the HBasePutProcessor - * in the DelegatingOutputFormat. + * @deprecated Moving to use org.apache.sqoop namespace. */ -public class HBaseImportJob extends DataDrivenImportJob { +public class HBaseImportJob + extends org.apache.sqoop.mapreduce.HBaseImportJob { public static final Log LOG = LogFactory.getLog( HBaseImportJob.class.getName()); public HBaseImportJob(final SqoopOptions opts, 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 getMapperClass() { - return HBaseImportMapper.class; - } - - @Override - protected Class 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); - } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/HBaseImportMapper.java b/src/java/com/cloudera/sqoop/mapreduce/HBaseImportMapper.java index ce74ae97..1fbc3334 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/HBaseImportMapper.java +++ b/src/java/com/cloudera/sqoop/mapreduce/HBaseImportMapper.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,24 +18,10 @@ 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 - * and the HBasePutProcessor. + * @deprecated Moving to use org.apache.sqoop namespace. */ public class HBaseImportMapper - extends AutoProgressMapper { - - @Override - public void map(LongWritable key, SqoopRecord val, Context context) - throws IOException, InterruptedException { - context.write(val, NullWritable.get()); - } + extends org.apache.sqoop.mapreduce.HBaseImportMapper { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/ImportJobBase.java b/src/java/com/cloudera/sqoop/mapreduce/ImportJobBase.java index 29c8eea1..c9fe8e51 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/ImportJobBase.java +++ b/src/java/com/cloudera/sqoop/mapreduce/ImportJobBase.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,49 +18,24 @@ 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.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 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; /** - * Base class for running an import MapReduce job. - * Allows dependency injection, etc, for easy customization of import job types. + * @deprecated Moving to use org.apache.sqoop namespace. */ -public class ImportJobBase extends JobBase { - - private ImportJobContext context; - - public static final Log LOG = LogFactory.getLog( - ImportJobBase.class.getName()); +public class ImportJobBase + extends org.apache.sqoop.mapreduce.ImportJobBase { public ImportJobBase() { - this(null); + super(); } public ImportJobBase(final SqoopOptions opts) { - this(opts, null, null, null, null); + super(opts); } public ImportJobBase(final SqoopOptions opts, @@ -70,138 +43,7 @@ public ImportJobBase(final SqoopOptions opts, final Class inputFormatClass, final Class outputFormatClass, final ImportJobContext context) { - super(opts, mapperClass, inputFormatClass, outputFormatClass); - this.context = context; + super(opts, mapperClass, inputFormatClass, outputFormatClass, 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 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; - } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/JdbcExportJob.java b/src/java/com/cloudera/sqoop/mapreduce/JdbcExportJob.java index a22f6896..d0baeafa 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/JdbcExportJob.java +++ b/src/java/com/cloudera/sqoop/mapreduce/JdbcExportJob.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,33 +18,16 @@ 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.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.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 { - - private FileType fileType; - - public static final Log LOG = LogFactory.getLog( - JdbcExportJob.class.getName()); +public class JdbcExportJob + extends org.apache.sqoop.mapreduce.JdbcExportJob { public JdbcExportJob(final ExportJobContext context) { super(context); @@ -59,84 +40,5 @@ public JdbcExportJob(final ExportJobContext ctxt, 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 columnTypeInts = - connManager.getColumnTypes(tableName, options.getSqlQuery()); - MapWritable columnTypes = new MapWritable(); - for (Map.Entry 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 getInputFormatClass() - throws ClassNotFoundException { - if (fileType == FileType.AVRO_DATA_FILE) { - return AvroInputFormat.class; - } - return super.getInputFormatClass(); - } - - @Override - protected Class 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); - } - } - } diff --git a/src/java/com/cloudera/sqoop/mapreduce/JdbcUpdateExportJob.java b/src/java/com/cloudera/sqoop/mapreduce/JdbcUpdateExportJob.java index 61456653..5d282ba6 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/JdbcUpdateExportJob.java +++ b/src/java/com/cloudera/sqoop/mapreduce/JdbcUpdateExportJob.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -21,43 +19,20 @@ package com.cloudera.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.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 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 getUpdateOutputFormat() - throws IOException { - return UpdateOutputFormat.class; - } +public class JdbcUpdateExportJob + extends org.apache.sqoop.mapreduce.JdbcUpdateExportJob { public JdbcUpdateExportJob(final ExportJobContext context) throws IOException { - super(context, null, null, getUpdateOutputFormat()); + super(context); } public JdbcUpdateExportJob(final ExportJobContext ctxt, @@ -67,82 +42,5 @@ public JdbcUpdateExportJob(final ExportJobContext ctxt, super(ctxt, mapperClass, inputFormatClass, outputFormatClass); } - @Override - protected Class 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 updateKeys = new LinkedHashSet(); - Set updateKeysUppercase = new HashSet(); - 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); - } - } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/JdbcUpsertExportJob.java b/src/java/com/cloudera/sqoop/mapreduce/JdbcUpsertExportJob.java index 826a72bb..c23f2349 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/JdbcUpsertExportJob.java +++ b/src/java/com/cloudera/sqoop/mapreduce/JdbcUpsertExportJob.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -21,89 +19,20 @@ package com.cloudera.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.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 static final Log LOG = LogFactory.getLog( - JdbcUpsertExportJob.class.getName()); +public class JdbcUpsertExportJob + extends org.apache.sqoop.mapreduce.JdbcUpsertExportJob { public JdbcUpsertExportJob(final ExportJobContext context, final Class outputFormatClass) 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 updateKeys = new LinkedHashSet(); - 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); - } - } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/JobBase.java b/src/java/com/cloudera/sqoop/mapreduce/JobBase.java index 8d4290f9..0c750910 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/JobBase.java +++ b/src/java/com/cloudera/sqoop/mapreduce/JobBase.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,316 +18,30 @@ 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.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. + * @deprecated Moving to use org.apache.sqoop namespace. */ -public class JobBase { - - public static final Log LOG = LogFactory.getLog(JobBase.class.getName()); - - protected SqoopOptions options; - protected Class mapperClass; - protected Class inputFormatClass; - protected Class outputFormatClass; - - private Job mrJob; - - private ClassLoader prevClassLoader = null; +public class JobBase + extends org.apache.sqoop.mapreduce.JobBase { public JobBase() { - this(null); + super(); } public JobBase(final SqoopOptions opts) { - this(opts, null, null, null); + super(opts); } public JobBase(final SqoopOptions opts, final Class mapperClass, final Class inputFormatClass, final Class outputFormatClass) { - - this.options = opts; - this.mapperClass = mapperClass; - this.inputFormatClass = inputFormatClass; - this.outputFormatClass = outputFormatClass; + super(opts, mapperClass, inputFormatClass, outputFormatClass); } - /** - * @return the mapper class to use for the job. - */ - protected Class getMapperClass() - throws ClassNotFoundException { - return this.mapperClass; - } - - /** - * @return the inputformat class to use for the job. - */ - protected Class getInputFormatClass() - throws ClassNotFoundException { - return this.inputFormatClass; - } - - /** - * @return the outputformat class to use for the job. - */ - protected Class getOutputFormatClass() - throws ClassNotFoundException { - return this.outputFormatClass; - } - - /** Set the OutputFormat class to use for this job. */ - public void setOutputFormatClass(Class cls) { - this.outputFormatClass = cls; - } - - /** Set the InputFormat class to use for this job. */ - public void setInputFormatClass(Class cls) { - this.inputFormatClass = cls; - } - - /** Set the Mapper class to use for this job. */ - public void setMapperClass(Class 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 localUrls = new HashSet(); - - 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 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 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 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 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 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."); - } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/MergeJob.java b/src/java/com/cloudera/sqoop/mapreduce/MergeJob.java index 23d31da4..5b213436 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/MergeJob.java +++ b/src/java/com/cloudera/sqoop/mapreduce/MergeJob.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,128 +18,27 @@ 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.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 = "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 static final String MERGE_OLD_PATH_KEY = + org.apache.sqoop.mapreduce.MergeJob.MERGE_OLD_PATH_KEY; + public static final String MERGE_NEW_PATH_KEY = + org.apache.sqoop.mapreduce.MergeJob.MERGE_NEW_PATH_KEY; + public static final String MERGE_KEY_COL_KEY = + org.apache.sqoop.mapreduce.MergeJob.MERGE_KEY_COL_KEY; + public static final String MERGE_SQOOP_RECORD_KEY = + org.apache.sqoop.mapreduce.MergeJob.MERGE_SQOOP_RECORD_KEY; 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 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); - } - } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/MergeMapperBase.java b/src/java/com/cloudera/sqoop/mapreduce/MergeMapperBase.java index 4f0dea36..083c8dc1 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/MergeMapperBase.java +++ b/src/java/com/cloudera/sqoop/mapreduce/MergeMapperBase.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,72 +18,9 @@ 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 - * 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. + * @deprecated Moving to use org.apache.sqoop namespace. */ public class MergeMapperBase - extends Mapper { - - 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 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); - } - } + extends org.apache.sqoop.mapreduce.MergeMapperBase { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/MergeRecord.java b/src/java/com/cloudera/sqoop/mapreduce/MergeRecord.java index 4a083884..0494c055 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/MergeRecord.java +++ b/src/java/com/cloudera/sqoop/mapreduce/MergeRecord.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,117 +18,20 @@ 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; /** - * 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. + * @deprecated Moving to use org.apache.sqoop namespace. */ -public class MergeRecord implements Configurable, Writable { - private SqoopRecord sqoopRecord; - private boolean isNew; - private Configuration config; +public class MergeRecord + extends org.apache.sqoop.mapreduce.MergeRecord { - /** Construct an empty MergeRecord. */ public MergeRecord() { - this.sqoopRecord = null; - this.isNew = false; - this.config = new Configuration(); + super(); } - /** - * Construct a MergeRecord with all fields initialized. - */ public MergeRecord(SqoopRecord sr, boolean recordIsNew) { - this.sqoopRecord = sr; - this.isNew = recordIsNew; - this.config = new Configuration(); + super(sr, recordIsNew); } - @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 recordClass = - (Class) 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; - } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/MergeRecordMapper.java b/src/java/com/cloudera/sqoop/mapreduce/MergeRecordMapper.java index 3869499a..a9acf9e3 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/MergeRecordMapper.java +++ b/src/java/com/cloudera/sqoop/mapreduce/MergeRecordMapper.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,20 +18,9 @@ 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 - extends MergeMapperBase { - - public void map(LongWritable key, SqoopRecord val, Context c) - throws IOException, InterruptedException { - processRecord(val, c); - } + extends org.apache.sqoop.mapreduce.MergeRecordMapper { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/MergeReducer.java b/src/java/com/cloudera/sqoop/mapreduce/MergeReducer.java index b31a9687..e3ecde3b 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/MergeReducer.java +++ b/src/java/com/cloudera/sqoop/mapreduce/MergeReducer.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,41 +18,10 @@ 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 - * a new one if possible; otherwise, an old one. + * @deprecated Moving to use org.apache.sqoop namespace. */ public class MergeReducer - extends Reducer { - - @Override - public void reduce(Text key, Iterable 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()); - } - } + extends org.apache.sqoop.mapreduce.MergeReducer { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/MergeTextMapper.java b/src/java/com/cloudera/sqoop/mapreduce/MergeTextMapper.java index 4a8c35f8..d4d7d025 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/MergeTextMapper.java +++ b/src/java/com/cloudera/sqoop/mapreduce/MergeTextMapper.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,44 +18,9 @@ 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 - * parse into SqoopRecord instances. + * @deprecated Moving to use org.apache.sqoop namespace. */ -public class MergeTextMapper extends MergeMapperBase { - - private SqoopRecord record; - - @Override - protected void setup(Context c) throws IOException, InterruptedException { - Configuration conf = c.getConfiguration(); - - Class recordClass = - (Class) 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); - } +public class MergeTextMapper + extends org.apache.sqoop.mapreduce.MergeTextMapper { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/MySQLDumpImportJob.java b/src/java/com/cloudera/sqoop/mapreduce/MySQLDumpImportJob.java index fca55ee1..a6d0de62 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/MySQLDumpImportJob.java +++ b/src/java/com/cloudera/sqoop/mapreduce/MySQLDumpImportJob.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,115 +18,18 @@ 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.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. + * @deprecated Moving to use org.apache.sqoop namespace. */ -public class MySQLDumpImportJob extends ImportJobBase { - - public static final Log LOG = - LogFactory.getLog(MySQLDumpImportJob.class.getName()); +public class MySQLDumpImportJob + extends org.apache.sqoop.mapreduce.MySQLDumpImportJob { 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); + super(opts, context); } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/MySQLDumpInputFormat.java b/src/java/com/cloudera/sqoop/mapreduce/MySQLDumpInputFormat.java index 1be01c85..066e4e06 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/MySQLDumpInputFormat.java +++ b/src/java/com/cloudera/sqoop/mapreduce/MySQLDumpInputFormat.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,81 +18,10 @@ 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 - * invocation running in the mapper. - * - * The key emitted by this mapper is a WHERE clause to use in the command - * to mysqldump. + * @deprecated Moving to use org.apache.sqoop namespace. */ -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 { - - 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 createRecordReader(InputSplit split, - TaskAttemptContext context) { - return new MySQLDumpRecordReader(split); - } - +public class MySQLDumpInputFormat + extends org.apache.sqoop.mapreduce.MySQLDumpInputFormat { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/MySQLDumpMapper.java b/src/java/com/cloudera/sqoop/mapreduce/MySQLDumpMapper.java index 4763c1a1..b6fe39f8 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/MySQLDumpMapper.java +++ b/src/java/com/cloudera/sqoop/mapreduce/MySQLDumpMapper.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,478 +18,40 @@ 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.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; /** - * Mapper that opens up a pipe to mysqldump and pulls data directly. + * @deprecated Moving to use org.apache.sqoop namespace. */ public class MySQLDumpMapper - extends Mapper { - - public static final Log LOG = LogFactory.getLog( - MySQLDumpMapper.class.getName()); - - private Configuration conf; - - // AsyncSinks used to import data from mysqldump directly into HDFS. + extends org.apache.sqoop.mapreduce.MySQLDumpMapper { /** - * Copies data directly from mysqldump into HDFS, after stripping some - * header and footer characters that are attached to each line in mysqldump. + * @deprecated Moving to use org.apache.sqoop namespace. */ - static class CopyingAsyncSink extends ErrorableAsyncSink { - private final MySQLDumpMapper.Context context; - private final PerfCounters counters; + public static class CopyingAsyncSink + extends org.apache.sqoop.mapreduce.MySQLDumpMapper.CopyingAsyncSink { - CopyingAsyncSink(final MySQLDumpMapper.Context context, + protected CopyingAsyncSink(final MySQLDumpMapper.Context context, final PerfCounters ctrs) { - this.context = context; - this.counters = ctrs; + super(context, 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. + * @deprecated Moving to use org.apache.sqoop namespace. */ - static class ReparsingAsyncSink extends ErrorableAsyncSink { - private final MySQLDumpMapper.Context context; - private final Configuration conf; - private final PerfCounters counters; + public static class ReparsingAsyncSink + extends org.apache.sqoop.mapreduce.MySQLDumpMapper.ReparsingAsyncSink { - ReparsingAsyncSink(final MySQLDumpMapper.Context c, + protected ReparsingAsyncSink(final MySQLDumpMapper.Context c, final Configuration conf, final PerfCounters ctrs) { - this.context = c; - this.conf = conf; - this.counters = ctrs; + super(c, conf, 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 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 args = new ArrayList(); - 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="" 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(); - } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/MySQLExportJob.java b/src/java/com/cloudera/sqoop/mapreduce/MySQLExportJob.java index b537912b..2eb8442a 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/MySQLExportJob.java +++ b/src/java/com/cloudera/sqoop/mapreduce/MySQLExportJob.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,99 +18,16 @@ 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.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 static final Log LOG = - LogFactory.getLog(MySQLExportJob.class.getName()); +public class MySQLExportJob + extends org.apache.sqoop.mapreduce.MySQLExportJob { 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 getMapperClass() { - if (inputIsSequenceFiles()) { - return MySQLRecordExportMapper.class; - } else { - return MySQLTextExportMapper.class; - } - } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/MySQLExportMapper.java b/src/java/com/cloudera/sqoop/mapreduce/MySQLExportMapper.java index 88cd398c..30046dd7 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/MySQLExportMapper.java +++ b/src/java/com/cloudera/sqoop/mapreduce/MySQLExportMapper.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,342 +18,16 @@ 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 - * 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. + * @deprecated Moving to use org.apache.sqoop namespace. */ public class MySQLExportMapper - extends Mapper { + extends org.apache.sqoop.mapreduce.MySQLExportMapper { - 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"; + 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 args = new ArrayList(); - - 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; - } - } } - diff --git a/src/java/com/cloudera/sqoop/mapreduce/MySQLRecordExportMapper.java b/src/java/com/cloudera/sqoop/mapreduce/MySQLRecordExportMapper.java index 664a0b81..08751d25 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/MySQLRecordExportMapper.java +++ b/src/java/com/cloudera/sqoop/mapreduce/MySQLRecordExportMapper.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,32 +18,9 @@ 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 - * SequenceFiles) to emit to the database. + * @deprecated Moving to use org.apache.sqoop namespace. */ public class MySQLRecordExportMapper - extends MySQLExportMapper { - - /** - * 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(); - } + extends org.apache.sqoop.mapreduce.MySQLRecordExportMapper { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/MySQLTextExportMapper.java b/src/java/com/cloudera/sqoop/mapreduce/MySQLTextExportMapper.java index 997447b9..95517e28 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/MySQLTextExportMapper.java +++ b/src/java/com/cloudera/sqoop/mapreduce/MySQLTextExportMapper.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,46 +18,9 @@ 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 - * in HDFS to emit to the database. + * @deprecated Moving to use org.apache.sqoop namespace. */ public class MySQLTextExportMapper - extends MySQLExportMapper { - - // 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(); - } + extends org.apache.sqoop.mapreduce.MySQLTextExportMapper { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/NullOutputCommitter.java b/src/java/com/cloudera/sqoop/mapreduce/NullOutputCommitter.java index 2a9a36fc..c3c9b39b 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/NullOutputCommitter.java +++ b/src/java/com/cloudera/sqoop/mapreduce/NullOutputCommitter.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,27 +18,10 @@ 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 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) { } +public class NullOutputCommitter + extends org.apache.sqoop.mapreduce.NullOutputCommitter { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/OracleExportOutputFormat.java b/src/java/com/cloudera/sqoop/mapreduce/OracleExportOutputFormat.java index 0fcb7e97..8ae10d79 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/OracleExportOutputFormat.java +++ b/src/java/com/cloudera/sqoop/mapreduce/OracleExportOutputFormat.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,94 +18,11 @@ 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; /** - * Oracle-specific SQL formatting overrides default ExportOutputFormat's. + * @deprecated Moving to use org.apache.sqoop namespace. */ public class OracleExportOutputFormat - extends ExportOutputFormat { - - @Override - /** {@inheritDoc} */ - public RecordWriter 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(); - } - } + extends org.apache.sqoop.mapreduce.OracleExportOutputFormat { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/OracleUpsertOutputFormat.java b/src/java/com/cloudera/sqoop/mapreduce/OracleUpsertOutputFormat.java index f6b56e79..aa20e7f2 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/OracleUpsertOutputFormat.java +++ b/src/java/com/cloudera/sqoop/mapreduce/OracleUpsertOutputFormat.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,115 +18,11 @@ 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; /** - * 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. + * @deprecated Moving to use org.apache.sqoop namespace. */ public class OracleUpsertOutputFormat - extends UpdateOutputFormat { - - private static final Log LOG = - LogFactory.getLog(OracleUpsertOutputFormat.class); - - @Override - /** {@inheritDoc} */ - public RecordWriter 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 updateKeyLookup = new LinkedHashSet(); - 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(); - } - } + extends org.apache.sqoop.mapreduce.OracleUpsertOutputFormat { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/RawKeyTextOutputFormat.java b/src/java/com/cloudera/sqoop/mapreduce/RawKeyTextOutputFormat.java index 97ae0629..2e9518ab 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/RawKeyTextOutputFormat.java +++ b/src/java/com/cloudera/sqoop/mapreduce/RawKeyTextOutputFormat.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -21,85 +19,25 @@ package com.cloudera.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. +/** + * @deprecated Moving to use org.apache.sqoop namespace. */ -public class RawKeyTextOutputFormat extends FileOutputFormat { +public class RawKeyTextOutputFormat + extends org.apache.sqoop.mapreduce.RawKeyTextOutputFormat { - protected static class RawKeyRecordWriter extends RecordWriter { - private static final String UTF8 = "UTF-8"; - - protected DataOutputStream out; + /** + * @deprecated Moving to use org.apache.sqoop namespace. + */ + public static class RawKeyRecordWriter + extends org.apache.sqoop.mapreduce.RawKeyTextOutputFormat. + RawKeyRecordWriter { 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 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 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(ostream); - } } diff --git a/src/java/com/cloudera/sqoop/mapreduce/SQLServerExportOutputFormat.java b/src/java/com/cloudera/sqoop/mapreduce/SQLServerExportOutputFormat.java index bc04dd48..f7841ca1 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/SQLServerExportOutputFormat.java +++ b/src/java/com/cloudera/sqoop/mapreduce/SQLServerExportOutputFormat.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,94 +18,11 @@ 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; /** - * SQLServer-specific SQL formatting overrides default ExportOutputFormat's. + * @deprecated Moving to use org.apache.sqoop namespace. */ public class SQLServerExportOutputFormat - extends ExportOutputFormat { - - @Override - /** {@inheritDoc} */ - public RecordWriter 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(); - } - } + extends org.apache.sqoop.mapreduce.SQLServerExportOutputFormat { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/SequenceFileExportMapper.java b/src/java/com/cloudera/sqoop/mapreduce/SequenceFileExportMapper.java index d4cef73a..3b5b8b7f 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/SequenceFileExportMapper.java +++ b/src/java/com/cloudera/sqoop/mapreduce/SequenceFileExportMapper.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,27 +18,9 @@ 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 - * that DBWritable to the OutputFormat for writeback to the database. + * @deprecated Moving to use org.apache.sqoop namespace. */ public class SequenceFileExportMapper - extends AutoProgressMapper { - - public SequenceFileExportMapper() { - } - - public void map(LongWritable key, SqoopRecord val, Context context) - throws IOException, InterruptedException { - context.write(val, NullWritable.get()); - } + extends org.apache.sqoop.mapreduce.SequenceFileExportMapper { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/SequenceFileImportMapper.java b/src/java/com/cloudera/sqoop/mapreduce/SequenceFileImportMapper.java index 1a92969b..152a9429 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/SequenceFileImportMapper.java +++ b/src/java/com/cloudera/sqoop/mapreduce/SequenceFileImportMapper.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,50 +18,10 @@ 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 - extends AutoProgressMapper { - - 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(); - } - } + extends org.apache.sqoop.mapreduce.SequenceFileImportMapper { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/TextExportMapper.java b/src/java/com/cloudera/sqoop/mapreduce/TextExportMapper.java index 82dcc88f..742feb6f 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/TextExportMapper.java +++ b/src/java/com/cloudera/sqoop/mapreduce/TextExportMapper.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,68 +18,9 @@ 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 - * record and emits that DBWritable to the OutputFormat for writeback to the - * database. + * @deprecated Moving to use org.apache.sqoop namespace. */ public class TextExportMapper - extends AutoProgressMapper { - - 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); - } - } + extends org.apache.sqoop.mapreduce.TextExportMapper { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/TextImportMapper.java b/src/java/com/cloudera/sqoop/mapreduce/TextImportMapper.java index 151ef071..04792972 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/TextImportMapper.java +++ b/src/java/com/cloudera/sqoop/mapreduce/TextImportMapper.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,57 +18,10 @@ 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 - extends AutoProgressMapper { - - 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(); - } - } + extends org.apache.sqoop.mapreduce.TextImportMapper { } diff --git a/src/java/com/cloudera/sqoop/mapreduce/UpdateOutputFormat.java b/src/java/com/cloudera/sqoop/mapreduce/UpdateOutputFormat.java index 68ce2ac4..fb47f9dc 100644 --- a/src/java/com/cloudera/sqoop/mapreduce/UpdateOutputFormat.java +++ b/src/java/com/cloudera/sqoop/mapreduce/UpdateOutputFormat.java @@ -1,6 +1,4 @@ /** - * Copyright 2011 The Apache Software Foundation - * * 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 @@ -20,196 +18,11 @@ 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.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. + * @deprecated Moving to use org.apache.sqoop namespace. */ public class UpdateOutputFormat - extends AsyncSqlOutputFormat { - - 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 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 { - - 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 updateKeys = new LinkedHashSet(); - 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 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(); - } - } + extends org.apache.sqoop.mapreduce.UpdateOutputFormat { } diff --git a/src/java/org/apache/sqoop/mapreduce/AsyncSqlOutputFormat.java b/src/java/org/apache/sqoop/mapreduce/AsyncSqlOutputFormat.java new file mode 100644 index 00000000..bb29bc20 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/AsyncSqlOutputFormat.java @@ -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 + extends OutputFormat { + + /** 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 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(); + 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)); + } + } + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/AsyncSqlRecordWriter.java b/src/java/org/apache/sqoop/mapreduce/AsyncSqlRecordWriter.java new file mode 100644 index 00000000..d0e17117 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/AsyncSqlRecordWriter.java @@ -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 + extends RecordWriter { + + 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 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(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 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); + } + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/AutoProgressMapper.java b/src/java/org/apache/sqoop/mapreduce/AutoProgressMapper.java new file mode 100644 index 00000000..2878ead8 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/AutoProgressMapper.java @@ -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 + extends Mapper { + + 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()); + } + } + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/AvroExportMapper.java b/src/java/org/apache/sqoop/mapreduce/AvroExportMapper.java new file mode 100644 index 00000000..959b60a9 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/AvroExportMapper.java @@ -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, 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 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 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 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()); + } + } + +} diff --git a/src/java/org/apache/sqoop/mapreduce/AvroImportMapper.java b/src/java/org/apache/sqoop/mapreduce/AvroImportMapper.java new file mode 100644 index 00000000..c6fdcf47 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/AvroImportMapper.java @@ -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, NullWritable> { + + private final AvroWrapper wrapper = + new AvroWrapper(); + 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 fieldMap = val.getFieldMap(); + GenericRecord record = new GenericData.Record(schema); + for (Map.Entry 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; + } + + +} diff --git a/src/java/org/apache/sqoop/mapreduce/AvroInputFormat.java b/src/java/org/apache/sqoop/mapreduce/AvroInputFormat.java new file mode 100644 index 00000000..0d5c08e9 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/AvroInputFormat.java @@ -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 + extends FileInputFormat, NullWritable> { + + @Override + protected List listStatus(JobContext job) throws IOException { + List result = new ArrayList(); + 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, NullWritable> createRecordReader( + InputSplit split, TaskAttemptContext context) throws IOException, + InterruptedException { + context.setStatus(split.toString()); + return new AvroRecordReader(); + } + +} + diff --git a/src/java/org/apache/sqoop/mapreduce/AvroJob.java b/src/java/org/apache/sqoop/mapreduce/AvroJob.java new file mode 100644 index 00000000..a57aaf1d --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/AvroJob.java @@ -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)); + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/AvroOutputFormat.java b/src/java/org/apache/sqoop/mapreduce/AvroOutputFormat.java new file mode 100644 index 00000000..96befd71 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/AvroOutputFormat.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.sqoop.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 + extends FileOutputFormat, NullWritable> { + + @Override + public RecordWriter, NullWritable> getRecordWriter( + TaskAttemptContext context) throws IOException, InterruptedException { + + Schema schema = AvroJob.getMapOutputSchema(context.getConfiguration()); + + final DataFileWriter WRITER = + new DataFileWriter(new GenericDatumWriter()); + + Path path = getDefaultWorkFile(context, + org.apache.avro.mapred.AvroOutputFormat.EXT); + WRITER.create(schema, + path.getFileSystem(context.getConfiguration()).create(path)); + + return new RecordWriter, NullWritable>() { + @Override + public void write(AvroWrapper wrapper, NullWritable ignore) + throws IOException { + WRITER.append(wrapper.datum()); + } + @Override + public void close(TaskAttemptContext context) throws IOException, + InterruptedException { + WRITER.close(); + } + }; + } + +} diff --git a/src/java/org/apache/sqoop/mapreduce/AvroRecordReader.java b/src/java/org/apache/sqoop/mapreduce/AvroRecordReader.java new file mode 100644 index 00000000..de033ee9 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/AvroRecordReader.java @@ -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 + extends RecordReader, NullWritable> { + + private FileReader reader; + private long start; + private long end; + private AvroWrapper 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 datumReader = new GenericDatumReader(); + 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(); + } + if (value == null) { + value = NullWritable.get(); + } + key.datum(reader.next(key.datum())); + return true; + } + + @Override + public AvroWrapper 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(); } +} + diff --git a/src/java/org/apache/sqoop/mapreduce/CombineShimRecordReader.java b/src/java/org/apache/sqoop/mapreduce/CombineShimRecordReader.java new file mode 100644 index 00000000..99cad39e --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/CombineShimRecordReader.java @@ -0,0 +1,131 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.sqoop.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 { + + public static final Log LOG = + LogFactory.getLog(CombineShimRecordReader.class.getName()); + + private CombineFileSplit split; + private TaskAttemptContext context; + private int index; + private RecordReader 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) + ReflectionUtils.newInstance(rrClass, conf); + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/DataDrivenImportJob.java b/src/java/org/apache/sqoop/mapreduce/DataDrivenImportJob.java new file mode 100644 index 00000000..503f898f --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/DataDrivenImportJob.java @@ -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 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 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 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); + } + } + } +} + diff --git a/src/java/org/apache/sqoop/mapreduce/DelegatingOutputFormat.java b/src/java/org/apache/sqoop/mapreduce/DelegatingOutputFormat.java new file mode 100644 index 00000000..2dd9be22 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/DelegatingOutputFormat.java @@ -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. + * + *

The output value is ignored.

+ * + *

The FieldMapProcessor implementation may do any arbitrary + * processing on the object. For example, it may write an object + * to HBase, etc.

+ * + *

If the FieldMapProcessor implementation also implements + * Closeable, it will be close()'d in the RecordReader's close() + * method.

+ * + *

If the FMP implements Configurable, it will be configured + * correctly via ReflectionUtils.

+ */ +public class DelegatingOutputFormat + extends OutputFormat { + + /** 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 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 { + + private Configuration conf; + + private FieldMapProcessor mapProcessor; + + public DelegatingRecordWriter(TaskAttemptContext context) + throws ClassNotFoundException { + + this.conf = context.getConfiguration(); + + @SuppressWarnings("unchecked") + Class procClass = + (Class) + 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); + } + } + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/ExportBatchOutputFormat.java b/src/java/org/apache/sqoop/mapreduce/ExportBatchOutputFormat.java new file mode 100644 index 00000000..20fc844b --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/ExportBatchOutputFormat.java @@ -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 + extends ExportOutputFormat { + + private static final Log LOG = + LogFactory.getLog(ExportBatchOutputFormat.class); + + @Override + /** {@inheritDoc} */ + public RecordWriter 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 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(); + } + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/ExportInputFormat.java b/src/java/org/apache/sqoop/mapreduce/ExportInputFormat.java new file mode 100644 index 00000000..773bfda7 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/ExportInputFormat.java @@ -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 { + + 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 stats = listStatus(job); + long count = 0; + for (FileStatus stat : stats) { + count += stat.getLen(); + } + + return count; + } + + @Override + public List 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 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); + } + +} diff --git a/src/java/org/apache/sqoop/mapreduce/ExportJobBase.java b/src/java/org/apache/sqoop/mapreduce/ExportJobBase.java new file mode 100644 index 00000000..901c59e5 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/ExportJobBase.java @@ -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 mapperClass, + final Class inputFormatClass, + final Class 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 getInputFormatClass() + throws ClassNotFoundException { + Class configuredIF = super.getInputFormatClass(); + if (null == configuredIF) { + return ExportInputFormat.class; + } else { + return configuredIF; + } + } + + @Override + protected Class getOutputFormatClass() + throws ClassNotFoundException { + Class 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; + } + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/ExportOutputFormat.java b/src/java/org/apache/sqoop/mapreduce/ExportOutputFormat.java new file mode 100644 index 00000000..c2e39b10 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/ExportOutputFormat.java @@ -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 + extends AsyncSqlOutputFormat { + + 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 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 { + + 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 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(); + } + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/HBaseImportJob.java b/src/java/org/apache/sqoop/mapreduce/HBaseImportJob.java new file mode 100644 index 00000000..6a784d21 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/HBaseImportJob.java @@ -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 getMapperClass() { + return HBaseImportMapper.class; + } + + @Override + protected Class 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); + } +} + diff --git a/src/java/org/apache/sqoop/mapreduce/HBaseImportMapper.java b/src/java/org/apache/sqoop/mapreduce/HBaseImportMapper.java new file mode 100644 index 00000000..63e6cd3f --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/HBaseImportMapper.java @@ -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 + { + + @Override + public void map(LongWritable key, SqoopRecord val, Context context) + throws IOException, InterruptedException { + context.write(val, NullWritable.get()); + } +} + diff --git a/src/java/org/apache/sqoop/mapreduce/ImportJobBase.java b/src/java/org/apache/sqoop/mapreduce/ImportJobBase.java new file mode 100644 index 00000000..ed6954a8 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/ImportJobBase.java @@ -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 mapperClass, + final Class inputFormatClass, + final Class 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 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; + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/JdbcExportJob.java b/src/java/org/apache/sqoop/mapreduce/JdbcExportJob.java new file mode 100644 index 00000000..7f69d92f --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/JdbcExportJob.java @@ -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 mapperClass, + final Class inputFormatClass, + final Class 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 columnTypeInts = + connManager.getColumnTypes(tableName, options.getSqlQuery()); + MapWritable columnTypes = new MapWritable(); + for (Map.Entry 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 getInputFormatClass() + throws ClassNotFoundException { + if (fileType == FileType.AVRO_DATA_FILE) { + return AvroInputFormat.class; + } + return super.getInputFormatClass(); + } + + @Override + protected Class 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); + } + } + +} + diff --git a/src/java/org/apache/sqoop/mapreduce/JdbcUpdateExportJob.java b/src/java/org/apache/sqoop/mapreduce/JdbcUpdateExportJob.java new file mode 100644 index 00000000..7be5ed99 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/JdbcUpdateExportJob.java @@ -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 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 mapperClass, + final Class inputFormatClass, + final Class outputFormatClass) { + super(ctxt, mapperClass, inputFormatClass, outputFormatClass); + } + + @Override + protected Class 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 updateKeys = new LinkedHashSet(); + Set updateKeysUppercase = new HashSet(); + 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); + } + } +} + diff --git a/src/java/org/apache/sqoop/mapreduce/JdbcUpsertExportJob.java b/src/java/org/apache/sqoop/mapreduce/JdbcUpsertExportJob.java new file mode 100644 index 00000000..f299f982 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/JdbcUpsertExportJob.java @@ -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 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 updateKeys = new LinkedHashSet(); + 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); + } + } +} + diff --git a/src/java/org/apache/sqoop/mapreduce/JobBase.java b/src/java/org/apache/sqoop/mapreduce/JobBase.java new file mode 100644 index 00000000..ba9cfa0f --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/JobBase.java @@ -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 mapperClass; + protected Class inputFormatClass; + protected Class 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 mapperClass, + final Class inputFormatClass, + final Class outputFormatClass) { + + this.options = opts; + this.mapperClass = mapperClass; + this.inputFormatClass = inputFormatClass; + this.outputFormatClass = outputFormatClass; + } + + /** + * @return the mapper class to use for the job. + */ + protected Class getMapperClass() + throws ClassNotFoundException { + return this.mapperClass; + } + + /** + * @return the inputformat class to use for the job. + */ + protected Class getInputFormatClass() + throws ClassNotFoundException { + return this.inputFormatClass; + } + + /** + * @return the outputformat class to use for the job. + */ + protected Class getOutputFormatClass() + throws ClassNotFoundException { + return this.outputFormatClass; + } + + /** Set the OutputFormat class to use for this job. */ + public void setOutputFormatClass(Class cls) { + this.outputFormatClass = cls; + } + + /** Set the InputFormat class to use for this job. */ + public void setInputFormatClass(Class cls) { + this.inputFormatClass = cls; + } + + /** Set the Mapper class to use for this job. */ + public void setMapperClass(Class 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 localUrls = new HashSet(); + + 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 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 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 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 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 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."); + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/MergeJob.java b/src/java/org/apache/sqoop/mapreduce/MergeJob.java new file mode 100644 index 00000000..139fa63a --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/MergeJob.java @@ -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 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); + } + } +} + + diff --git a/src/java/org/apache/sqoop/mapreduce/MergeMapperBase.java b/src/java/org/apache/sqoop/mapreduce/MergeMapperBase.java new file mode 100644 index 00000000..c4fd9f78 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/MergeMapperBase.java @@ -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 + extends Mapper { + + 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 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); + } + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/MergeRecord.java b/src/java/org/apache/sqoop/mapreduce/MergeRecord.java new file mode 100644 index 00000000..d73112c9 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/MergeRecord.java @@ -0,0 +1,131 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.sqoop.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 recordClass = + (Class) 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; + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/MergeRecordMapper.java b/src/java/org/apache/sqoop/mapreduce/MergeRecordMapper.java new file mode 100644 index 00000000..bfab0c6f --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/MergeRecordMapper.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.sqoop.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 { + + public void map(LongWritable key, SqoopRecord val, Context c) + throws IOException, InterruptedException { + processRecord(val, c); + } + +} diff --git a/src/java/org/apache/sqoop/mapreduce/MergeReducer.java b/src/java/org/apache/sqoop/mapreduce/MergeReducer.java new file mode 100644 index 00000000..cafff8ab --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/MergeReducer.java @@ -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 { + + @Override + public void reduce(Text key, Iterable 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()); + } + } +} + diff --git a/src/java/org/apache/sqoop/mapreduce/MergeTextMapper.java b/src/java/org/apache/sqoop/mapreduce/MergeTextMapper.java new file mode 100644 index 00000000..c66067e7 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/MergeTextMapper.java @@ -0,0 +1,60 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.sqoop.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 { + + private SqoopRecord record; + + @Override + protected void setup(Context c) throws IOException, InterruptedException { + Configuration conf = c.getConfiguration(); + + Class recordClass = + (Class) 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); + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/MySQLDumpImportJob.java b/src/java/org/apache/sqoop/mapreduce/MySQLDumpImportJob.java new file mode 100644 index 00000000..30fa8825 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/MySQLDumpImportJob.java @@ -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); + } + +} diff --git a/src/java/org/apache/sqoop/mapreduce/MySQLDumpInputFormat.java b/src/java/org/apache/sqoop/mapreduce/MySQLDumpInputFormat.java new file mode 100644 index 00000000..559edd29 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/MySQLDumpInputFormat.java @@ -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 { + + 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 createRecordReader(InputSplit split, + TaskAttemptContext context) { + return new MySQLDumpRecordReader(split); + } + +} + diff --git a/src/java/org/apache/sqoop/mapreduce/MySQLDumpMapper.java b/src/java/org/apache/sqoop/mapreduce/MySQLDumpMapper.java new file mode 100644 index 00000000..4ec65ef7 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/MySQLDumpMapper.java @@ -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 { + + 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 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 args = new ArrayList(); + 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="" 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(); + } +} + diff --git a/src/java/org/apache/sqoop/mapreduce/MySQLExportJob.java b/src/java/org/apache/sqoop/mapreduce/MySQLExportJob.java new file mode 100644 index 00000000..c60653c4 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/MySQLExportJob.java @@ -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 getMapperClass() { + if (inputIsSequenceFiles()) { + return MySQLRecordExportMapper.class; + } else { + return MySQLTextExportMapper.class; + } + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/MySQLExportMapper.java b/src/java/org/apache/sqoop/mapreduce/MySQLExportMapper.java new file mode 100644 index 00000000..859e2ca1 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/MySQLExportMapper.java @@ -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 + extends Mapper { + + 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 args = new ArrayList(); + + 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; + } + } +} + diff --git a/src/java/org/apache/sqoop/mapreduce/MySQLRecordExportMapper.java b/src/java/org/apache/sqoop/mapreduce/MySQLRecordExportMapper.java new file mode 100644 index 00000000..d2d0c614 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/MySQLRecordExportMapper.java @@ -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 { + + /** + * 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(); + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/MySQLTextExportMapper.java b/src/java/org/apache/sqoop/mapreduce/MySQLTextExportMapper.java new file mode 100644 index 00000000..41d90f97 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/MySQLTextExportMapper.java @@ -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 { + + // 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(); + } + +} diff --git a/src/java/org/apache/sqoop/mapreduce/NullOutputCommitter.java b/src/java/org/apache/sqoop/mapreduce/NullOutputCommitter.java new file mode 100644 index 00000000..f56430e8 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/NullOutputCommitter.java @@ -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) { } + +} + diff --git a/src/java/org/apache/sqoop/mapreduce/OracleExportOutputFormat.java b/src/java/org/apache/sqoop/mapreduce/OracleExportOutputFormat.java new file mode 100644 index 00000000..8e9cf1dd --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/OracleExportOutputFormat.java @@ -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 + extends ExportOutputFormat { + + @Override + /** {@inheritDoc} */ + public RecordWriter 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(); + } + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/OracleUpsertOutputFormat.java b/src/java/org/apache/sqoop/mapreduce/OracleUpsertOutputFormat.java new file mode 100644 index 00000000..f79aaa9d --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/OracleUpsertOutputFormat.java @@ -0,0 +1,131 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.sqoop.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 + extends UpdateOutputFormat { + + private static final Log LOG = + LogFactory.getLog(OracleUpsertOutputFormat.class); + + @Override + /** {@inheritDoc} */ + public RecordWriter 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 updateKeyLookup = new LinkedHashSet(); + 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(); + } + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/RawKeyTextOutputFormat.java b/src/java/org/apache/sqoop/mapreduce/RawKeyTextOutputFormat.java new file mode 100644 index 00000000..d02a3504 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/RawKeyTextOutputFormat.java @@ -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 extends FileOutputFormat { + + public static class RawKeyRecordWriter extends RecordWriter { + + 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 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 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(ostream); + } + +} + diff --git a/src/java/org/apache/sqoop/mapreduce/SQLServerExportOutputFormat.java b/src/java/org/apache/sqoop/mapreduce/SQLServerExportOutputFormat.java new file mode 100644 index 00000000..1b395191 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/SQLServerExportOutputFormat.java @@ -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 + extends ExportOutputFormat { + + @Override + /** {@inheritDoc} */ + public RecordWriter 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(); + } + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/SequenceFileExportMapper.java b/src/java/org/apache/sqoop/mapreduce/SequenceFileExportMapper.java new file mode 100644 index 00000000..434d7d38 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/SequenceFileExportMapper.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.sqoop.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 { + + public SequenceFileExportMapper() { + } + + public void map(LongWritable key, SqoopRecord val, Context context) + throws IOException, InterruptedException { + context.write(val, NullWritable.get()); + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/SequenceFileImportMapper.java b/src/java/org/apache/sqoop/mapreduce/SequenceFileImportMapper.java new file mode 100644 index 00000000..528eeec3 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/SequenceFileImportMapper.java @@ -0,0 +1,66 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.sqoop.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 { + + 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(); + } + } +} + diff --git a/src/java/org/apache/sqoop/mapreduce/TextExportMapper.java b/src/java/org/apache/sqoop/mapreduce/TextExportMapper.java new file mode 100644 index 00000000..7b7f3314 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/TextExportMapper.java @@ -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 { + + 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); + } + } +} diff --git a/src/java/org/apache/sqoop/mapreduce/TextImportMapper.java b/src/java/org/apache/sqoop/mapreduce/TextImportMapper.java new file mode 100644 index 00000000..a63faedc --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/TextImportMapper.java @@ -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 { + + 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(); + } + } +} + diff --git a/src/java/org/apache/sqoop/mapreduce/UpdateOutputFormat.java b/src/java/org/apache/sqoop/mapreduce/UpdateOutputFormat.java new file mode 100644 index 00000000..96cec176 --- /dev/null +++ b/src/java/org/apache/sqoop/mapreduce/UpdateOutputFormat.java @@ -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 + extends AsyncSqlOutputFormat { + + 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 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 { + + 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 updateKeys = new LinkedHashSet(); + 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 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(); + } + } +}