Modified: incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ImportTool.java
URL: http://svn.apache.org/viewvc/incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ImportTool.java?rev=1195852&r1=1195851&r2=1195852&view=diff
==============================================================================
--- incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ImportTool.java (original)
+++ incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ImportTool.java Tue Nov 1 07:10:13 2011
@@ -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
@@ -17,883 +15,20 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package com.cloudera.sqoop.tool;
-import java.io.IOException;
-
-import java.sql.Connection;
-import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.sql.Types;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.OptionBuilder;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import org.apache.hadoop.util.StringUtils;
-
-import com.cloudera.sqoop.Sqoop;
-import com.cloudera.sqoop.SqoopOptions;
-import com.cloudera.sqoop.SqoopOptions.FileLayout;
-import com.cloudera.sqoop.SqoopOptions.InvalidOptionsException;
-import com.cloudera.sqoop.cli.RelatedOptions;
-import com.cloudera.sqoop.cli.ToolOptions;
-import com.cloudera.sqoop.hive.HiveImport;
-import com.cloudera.sqoop.manager.ImportJobContext;
-
-import com.cloudera.sqoop.metastore.JobData;
-import com.cloudera.sqoop.metastore.JobStorage;
-import com.cloudera.sqoop.metastore.JobStorageFactory;
-import com.cloudera.sqoop.util.AppendUtils;
-import com.cloudera.sqoop.util.ImportException;
-import org.apache.hadoop.fs.Path;
-
/**
- * Tool that performs database imports to HDFS.
+ * @deprecated Moving to use org.apache.sqoop namespace.
*/
-public class ImportTool extends BaseSqoopTool {
-
- public static final Log LOG = LogFactory.getLog(ImportTool.class.getName());
-
- private CodeGenTool codeGenerator;
-
- // true if this is an all-tables import. Set by a subclass which
- // overrides the run() method of this tool (which can only do
- // a single table).
- private boolean allTables;
-
- // store check column type for incremental option
- private int checkColumnType;
+public class ImportTool
+ extends org.apache.sqoop.tool.ImportTool {
public ImportTool() {
- this("import", false);
+ super();
}
public ImportTool(String toolName, boolean allTables) {
- super(toolName);
- this.codeGenerator = new CodeGenTool();
- this.allTables = allTables;
- }
-
- @Override
- protected boolean init(SqoopOptions sqoopOpts) {
- boolean ret = super.init(sqoopOpts);
- codeGenerator.setManager(manager);
- return ret;
- }
-
- /**
- * @return a list of jar files generated as part of this import process
- */
- public List<String> getGeneratedJarFiles() {
- return this.codeGenerator.getGeneratedJarFiles();
- }
-
- /**
- * @return true if the supplied options specify an incremental import.
- */
- private boolean isIncremental(SqoopOptions options) {
- return !options.getIncrementalMode().equals(
- SqoopOptions.IncrementalMode.None);
- }
-
- /**
- * If this is an incremental import, then we should save the
- * user's state back to the metastore (if this job was run
- * from the metastore). Otherwise, log to the user what data
- * they need to supply next time.
- */
- private void saveIncrementalState(SqoopOptions options)
- throws IOException {
- if (!isIncremental(options)) {
- return;
- }
-
- Map<String, String> descriptor = options.getStorageDescriptor();
- String jobName = options.getJobName();
-
- if (null != jobName && null != descriptor) {
- // Actually save it back to the metastore.
- LOG.info("Saving incremental import state to the metastore");
- JobStorageFactory ssf = new JobStorageFactory(options.getConf());
- JobStorage storage = ssf.getJobStorage(descriptor);
- storage.open(descriptor);
- try {
- // Save the 'parent' SqoopOptions; this does not contain the mutations
- // to the SqoopOptions state that occurred over the course of this
- // execution, except for the one we specifically want to memorize:
- // the latest value of the check column.
- JobData data = new JobData(options.getParent(), this);
- storage.update(jobName, data);
- LOG.info("Updated data for job: " + jobName);
- } finally {
- storage.close();
- }
- } else {
- // If there wasn't a parent SqoopOptions, then the incremental
- // state data was stored in the current SqoopOptions.
- LOG.info("Incremental import complete! To run another incremental "
- + "import of all data following this import, supply the "
- + "following arguments:");
- SqoopOptions.IncrementalMode incrementalMode =
- options.getIncrementalMode();
- switch (incrementalMode) {
- case AppendRows:
- LOG.info(" --incremental append");
- break;
- case DateLastModified:
- LOG.info(" --incremental lastmodified");
- break;
- default:
- LOG.warn("Undefined incremental mode: " + incrementalMode);
- break;
- }
- LOG.info(" --check-column " + options.getIncrementalTestColumn());
- LOG.info(" --last-value " + options.getIncrementalLastValue());
- LOG.info("(Consider saving this with 'sqoop job --create')");
- }
- }
-
- /**
- * Return the max value in the incremental-import test column. This
- * value must be numeric.
- */
- private Object getMaxColumnId(SqoopOptions options) throws SQLException {
- StringBuilder sb = new StringBuilder();
- sb.append("SELECT MAX(");
- sb.append(options.getIncrementalTestColumn());
- sb.append(") FROM ");
- sb.append(options.getTableName());
-
- String where = options.getWhereClause();
- if (null != where) {
- sb.append(" WHERE ");
- sb.append(where);
- }
-
- Connection conn = manager.getConnection();
- Statement s = null;
- ResultSet rs = null;
- try {
- s = conn.createStatement();
- rs = s.executeQuery(sb.toString());
- if (!rs.next()) {
- // This probably means the table is empty.
- LOG.warn("Unexpected: empty results for max value query?");
- return null;
- }
-
- ResultSetMetaData rsmd = rs.getMetaData();
- checkColumnType = rsmd.getColumnType(1);
- if (checkColumnType == Types.TIMESTAMP) {
- return rs.getTimestamp(1);
- } else if (checkColumnType == Types.DATE) {
- return rs.getDate(1);
- } else if (checkColumnType == Types.TIME) {
- return rs.getTime(1);
- } else {
- return rs.getObject(1);
- }
- } finally {
- try {
- if (null != rs) {
- rs.close();
- }
- } catch (SQLException sqlE) {
- LOG.warn("SQL Exception closing resultset: " + sqlE);
- }
-
- try {
- if (null != s) {
- s.close();
- }
- } catch (SQLException sqlE) {
- LOG.warn("SQL Exception closing statement: " + sqlE);
- }
- }
- }
-
- /**
- * Determine if a column is date/time.
- * @return true if column type is TIMESTAMP, DATE, or TIME.
- */
- private boolean isDateTimeColumn(int columnType) {
- return (columnType == Types.TIMESTAMP)
- || (columnType == Types.DATE)
- || (columnType == Types.TIME);
- }
-
- /**
- * Initialize the constraints which set the incremental import range.
- * @return false if an import is not necessary, because the dataset has not
- * changed.
- */
- private boolean initIncrementalConstraints(SqoopOptions options,
- ImportJobContext context) throws ImportException, IOException {
-
- // If this is an incremental import, determine the constraints
- // to inject in the WHERE clause or $CONDITIONS for a query.
- // Also modify the 'last value' field of the SqoopOptions to
- // specify the current job start time / start row.
-
- if (!isIncremental(options)) {
- return true;
- }
-
- SqoopOptions.IncrementalMode incrementalMode = options.getIncrementalMode();
- String nextIncrementalValue = null;
-
- Object nextVal;
- switch (incrementalMode) {
- case AppendRows:
- try {
- nextVal = getMaxColumnId(options);
- if (isDateTimeColumn(checkColumnType)) {
- nextIncrementalValue = (nextVal == null) ? null
- : manager.datetimeToQueryString(nextVal.toString(),
- checkColumnType);
- } else {
- nextIncrementalValue = (nextVal == null) ? null : nextVal.toString();
- }
- } catch (SQLException sqlE) {
- throw new IOException(sqlE);
- }
- break;
- case DateLastModified:
- checkColumnType = Types.TIMESTAMP;
- nextVal = manager.getCurrentDbTimestamp();
- if (null == nextVal) {
- throw new IOException("Could not get current time from database");
- }
- nextIncrementalValue = manager.datetimeToQueryString(nextVal.toString(),
- checkColumnType);
- break;
- default:
- throw new ImportException("Undefined incremental import type: "
- + incrementalMode);
- }
-
- // Build the WHERE clause components that are used to import
- // only this incremental section.
- StringBuilder sb = new StringBuilder();
- String prevEndpoint = options.getIncrementalLastValue();
-
- if (isDateTimeColumn(checkColumnType) && null != prevEndpoint
- && !prevEndpoint.startsWith("\'") && !prevEndpoint.endsWith("\'")) {
- // Incremental imports based on date/time should be 'quoted' in
- // ANSI SQL. If the user didn't specify single-quotes, put them
- // around, here.
- prevEndpoint = manager.datetimeToQueryString(prevEndpoint,
- checkColumnType);
- }
-
- String checkColName = manager.escapeColName(
- options.getIncrementalTestColumn());
- LOG.info("Incremental import based on column " + checkColName);
- if (null != prevEndpoint) {
- if (prevEndpoint.equals(nextIncrementalValue)) {
- LOG.info("No new rows detected since last import.");
- return false;
- }
- LOG.info("Lower bound value: " + prevEndpoint);
- sb.append(checkColName);
- switch (incrementalMode) {
- case AppendRows:
- sb.append(" > ");
- break;
- case DateLastModified:
- sb.append(" >= ");
- break;
- default:
- throw new ImportException("Undefined comparison");
- }
- sb.append(prevEndpoint);
- sb.append(" AND ");
- }
-
- if (null != nextIncrementalValue) {
- sb.append(checkColName);
- switch (incrementalMode) {
- case AppendRows:
- sb.append(" <= ");
- break;
- case DateLastModified:
- sb.append(" < ");
- break;
- default:
- throw new ImportException("Undefined comparison");
- }
- sb.append(nextIncrementalValue);
- } else {
- sb.append(checkColName);
- sb.append(" IS NULL ");
- }
-
- LOG.info("Upper bound value: " + nextIncrementalValue);
-
- String prevWhereClause = options.getWhereClause();
- if (null != prevWhereClause) {
- sb.append(" AND (");
- sb.append(prevWhereClause);
- sb.append(")");
- }
-
- String newConstraints = sb.toString();
- options.setWhereClause(newConstraints);
-
- // Save this state for next time.
- SqoopOptions recordOptions = options.getParent();
- if (null == recordOptions) {
- recordOptions = options;
- }
- recordOptions.setIncrementalLastValue(
- (nextVal == null) ? null : nextVal.toString());
-
- return true;
- }
-
- /**
- * Import a table or query.
- * @return true if an import was performed, false otherwise.
- */
- protected boolean importTable(SqoopOptions options, String tableName,
- HiveImport hiveImport) throws IOException, ImportException {
- String jarFile = null;
-
- // Generate the ORM code for the tables.
- jarFile = codeGenerator.generateORM(options, tableName);
-
- // Do the actual import.
- ImportJobContext context = new ImportJobContext(tableName, jarFile,
- options, getOutputPath(options, tableName));
-
- // If we're doing an incremental import, set up the
- // filtering conditions used to get the latest records.
- if (!initIncrementalConstraints(options, context)) {
- return false;
- }
-
- if (null != tableName) {
- manager.importTable(context);
- } else {
- manager.importQuery(context);
- }
-
- if (options.isAppendMode()) {
- AppendUtils app = new AppendUtils(context);
- app.append();
- }
-
- // If the user wants this table to be in Hive, perform that post-load.
- if (options.doHiveImport()) {
- hiveImport.importTable(tableName, options.getHiveTableName(), false);
- }
-
- saveIncrementalState(options);
-
- return true;
- }
-
- /**
- * @return the output path for the imported files;
- * in append mode this will point to a temporary folder.
- * if importing to hbase, this may return null.
- */
- private Path getOutputPath(SqoopOptions options, String tableName) {
- // Get output directory
- String hdfsWarehouseDir = options.getWarehouseDir();
- String hdfsTargetDir = options.getTargetDir();
- Path outputPath = null;
- if (options.isAppendMode()) {
- // Use temporary path, later removed when appending
- outputPath = AppendUtils.getTempAppendDir(tableName);
- LOG.debug("Using temporary folder: " + outputPath.getName());
- } else {
- // Try in this order: target-dir or warehouse-dir
- if (hdfsTargetDir != null) {
- outputPath = new Path(hdfsTargetDir);
- } else if (hdfsWarehouseDir != null) {
- outputPath = new Path(hdfsWarehouseDir, tableName);
- } else if (null != tableName) {
- outputPath = new Path(tableName);
- }
- }
-
- return outputPath;
- }
-
- @Override
- /** {@inheritDoc} */
- public int run(SqoopOptions options) {
- HiveImport hiveImport = null;
-
- if (allTables) {
- // We got into this method, but we should be in a subclass.
- // (This method only handles a single table)
- // This should not be reached, but for sanity's sake, test here.
- LOG.error("ImportTool.run() can only handle a single table.");
- return 1;
- }
-
- if (!init(options)) {
- return 1;
- }
-
- codeGenerator.setManager(manager);
-
- try {
- if (options.doHiveImport()) {
- hiveImport = new HiveImport(options, manager, options.getConf(), false);
- }
-
- // Import a single table (or query) the user specified.
- importTable(options, options.getTableName(), hiveImport);
- } catch (IllegalArgumentException iea) {
- LOG.error("Imported Failed: " + iea.getMessage());
- if (System.getProperty(Sqoop.SQOOP_RETHROW_PROPERTY) != null) {
- throw iea;
- }
- return 1;
- } catch (IOException ioe) {
- LOG.error("Encountered IOException running import job: "
- + StringUtils.stringifyException(ioe));
- if (System.getProperty(Sqoop.SQOOP_RETHROW_PROPERTY) != null) {
- throw new RuntimeException(ioe);
- } else {
- return 1;
- }
- } catch (ImportException ie) {
- LOG.error("Error during import: " + ie.toString());
- if (System.getProperty(Sqoop.SQOOP_RETHROW_PROPERTY) != null) {
- throw new RuntimeException(ie);
- } else {
- return 1;
- }
- } finally {
- destroy(options);
- }
-
- return 0;
- }
-
- /**
- * Construct the set of options that control imports, either of one
- * table or a batch of tables.
- * @return the RelatedOptions that can be used to parse the import
- * arguments.
- */
- protected RelatedOptions getImportOptions() {
- // Imports
- RelatedOptions importOpts = new RelatedOptions("Import control arguments");
-
- importOpts.addOption(OptionBuilder
- .withDescription("Use direct import fast path")
- .withLongOpt(DIRECT_ARG)
- .create());
-
- if (!allTables) {
- importOpts.addOption(OptionBuilder.withArgName("table-name")
- .hasArg().withDescription("Table to read")
- .withLongOpt(TABLE_ARG)
- .create());
- importOpts.addOption(OptionBuilder.withArgName("col,col,col...")
- .hasArg().withDescription("Columns to import from table")
- .withLongOpt(COLUMNS_ARG)
- .create());
- importOpts.addOption(OptionBuilder.withArgName("column-name")
- .hasArg()
- .withDescription("Column of the table used to split work units")
- .withLongOpt(SPLIT_BY_ARG)
- .create());
- importOpts.addOption(OptionBuilder.withArgName("where clause")
- .hasArg().withDescription("WHERE clause to use during import")
- .withLongOpt(WHERE_ARG)
- .create());
- importOpts.addOption(OptionBuilder
- .withDescription("Imports data in append mode")
- .withLongOpt(APPEND_ARG)
- .create());
- importOpts.addOption(OptionBuilder.withArgName("dir")
- .hasArg().withDescription("HDFS plain table destination")
- .withLongOpt(TARGET_DIR_ARG)
- .create());
- importOpts.addOption(OptionBuilder.withArgName("statement")
- .hasArg()
- .withDescription("Import results of SQL 'statement'")
- .withLongOpt(SQL_QUERY_ARG)
- .create(SQL_QUERY_SHORT_ARG));
- importOpts.addOption(OptionBuilder.withArgName("statement")
- .hasArg()
- .withDescription("Set boundary query for retrieving max and min"
- + " value of the primary key")
- .withLongOpt(SQL_QUERY_BOUNDARY)
- .create());
- }
-
- importOpts.addOption(OptionBuilder.withArgName("dir")
- .hasArg().withDescription("HDFS parent for table destination")
- .withLongOpt(WAREHOUSE_DIR_ARG)
- .create());
- importOpts.addOption(OptionBuilder
- .withDescription("Imports data to SequenceFiles")
- .withLongOpt(FMT_SEQUENCEFILE_ARG)
- .create());
- importOpts.addOption(OptionBuilder
- .withDescription("Imports data as plain text (default)")
- .withLongOpt(FMT_TEXTFILE_ARG)
- .create());
- importOpts.addOption(OptionBuilder
- .withDescription("Imports data to Avro data files")
- .withLongOpt(FMT_AVRODATAFILE_ARG)
- .create());
- importOpts.addOption(OptionBuilder.withArgName("n")
- .hasArg().withDescription("Use 'n' map tasks to import in parallel")
- .withLongOpt(NUM_MAPPERS_ARG)
- .create(NUM_MAPPERS_SHORT_ARG));
- importOpts.addOption(OptionBuilder
- .withDescription("Enable compression")
- .withLongOpt(COMPRESS_ARG)
- .create(COMPRESS_SHORT_ARG));
- importOpts.addOption(OptionBuilder.withArgName("codec")
- .hasArg()
- .withDescription("Compression codec to use for import")
- .withLongOpt(COMPRESSION_CODEC_ARG)
- .create());
- importOpts.addOption(OptionBuilder.withArgName("n")
- .hasArg()
- .withDescription("Split the input stream every 'n' bytes "
- + "when importing in direct mode")
- .withLongOpt(DIRECT_SPLIT_SIZE_ARG)
- .create());
- importOpts.addOption(OptionBuilder.withArgName("n")
- .hasArg()
- .withDescription("Set the maximum size for an inline LOB")
- .withLongOpt(INLINE_LOB_LIMIT_ARG)
- .create());
- importOpts.addOption(OptionBuilder.withArgName("n")
- .hasArg()
- .withDescription("Set number 'n' of rows to fetch from the "
- + "database when more rows are needed")
- .withLongOpt(FETCH_SIZE_ARG)
- .create());
-
- return importOpts;
- }
-
- /**
- * Return options for incremental import.
- */
- protected RelatedOptions getIncrementalOptions() {
- RelatedOptions incrementalOpts =
- new RelatedOptions("Incremental import arguments");
-
- incrementalOpts.addOption(OptionBuilder.withArgName("import-type")
- .hasArg()
- .withDescription(
- "Define an incremental import of type 'append' or 'lastmodified'")
- .withLongOpt(INCREMENT_TYPE_ARG)
- .create());
- incrementalOpts.addOption(OptionBuilder.withArgName("column")
- .hasArg()
- .withDescription("Source column to check for incremental change")
- .withLongOpt(INCREMENT_COL_ARG)
- .create());
- incrementalOpts.addOption(OptionBuilder.withArgName("value")
- .hasArg()
- .withDescription("Last imported value in the incremental check column")
- .withLongOpt(INCREMENT_LAST_VAL_ARG)
- .create());
-
- return incrementalOpts;
- }
-
- @Override
- /** Configure the command-line arguments we expect to receive */
- public void configureOptions(ToolOptions toolOptions) {
-
- toolOptions.addUniqueOptions(getCommonOptions());
- toolOptions.addUniqueOptions(getImportOptions());
- if (!allTables) {
- toolOptions.addUniqueOptions(getIncrementalOptions());
- }
- toolOptions.addUniqueOptions(getOutputFormatOptions());
- toolOptions.addUniqueOptions(getInputFormatOptions());
- toolOptions.addUniqueOptions(getHiveOptions(true));
- toolOptions.addUniqueOptions(getHBaseOptions());
-
- // get common codegen opts.
- RelatedOptions codeGenOpts = getCodeGenOpts(allTables);
-
- // add import-specific codegen opts:
- codeGenOpts.addOption(OptionBuilder.withArgName("file")
- .hasArg()
- .withDescription("Disable code generation; use specified jar")
- .withLongOpt(JAR_FILE_NAME_ARG)
- .create());
-
- toolOptions.addUniqueOptions(codeGenOpts);
- }
-
- @Override
- /** {@inheritDoc} */
- public void printHelp(ToolOptions toolOptions) {
- super.printHelp(toolOptions);
- System.out.println("");
- if (allTables) {
- System.out.println("At minimum, you must specify --connect");
- } else {
- System.out.println(
- "At minimum, you must specify --connect and --table");
- }
-
- System.out.println(
- "Arguments to mysqldump and other subprograms may be supplied");
- System.out.println(
- "after a '--' on the command line.");
- }
-
- private void applyIncrementalOptions(CommandLine in, SqoopOptions out)
- throws InvalidOptionsException {
- if (in.hasOption(INCREMENT_TYPE_ARG)) {
- String incrementalTypeStr = in.getOptionValue(INCREMENT_TYPE_ARG);
- if ("append".equals(incrementalTypeStr)) {
- out.setIncrementalMode(SqoopOptions.IncrementalMode.AppendRows);
- // This argument implies ability to append to the same directory.
- out.setAppendMode(true);
- } else if ("lastmodified".equals(incrementalTypeStr)) {
- out.setIncrementalMode(SqoopOptions.IncrementalMode.DateLastModified);
- } else {
- throw new InvalidOptionsException("Unknown incremental import mode: "
- + incrementalTypeStr + ". Use 'append' or 'lastmodified'."
- + HELP_STR);
- }
- }
-
- if (in.hasOption(INCREMENT_COL_ARG)) {
- out.setIncrementalTestColumn(in.getOptionValue(INCREMENT_COL_ARG));
- }
-
- if (in.hasOption(INCREMENT_LAST_VAL_ARG)) {
- out.setIncrementalLastValue(in.getOptionValue(INCREMENT_LAST_VAL_ARG));
- }
- }
-
- @Override
- /** {@inheritDoc} */
- public void applyOptions(CommandLine in, SqoopOptions out)
- throws InvalidOptionsException {
-
- try {
- applyCommonOptions(in, out);
-
- if (in.hasOption(DIRECT_ARG)) {
- out.setDirectMode(true);
- }
-
- if (!allTables) {
- if (in.hasOption(TABLE_ARG)) {
- out.setTableName(in.getOptionValue(TABLE_ARG));
- }
-
- if (in.hasOption(COLUMNS_ARG)) {
- String[] cols= in.getOptionValue(COLUMNS_ARG).split(",");
- for (int i=0; i<cols.length; i++) {
- cols[i] = cols[i].trim();
- }
- out.setColumns(cols);
- }
-
- if (in.hasOption(SPLIT_BY_ARG)) {
- out.setSplitByCol(in.getOptionValue(SPLIT_BY_ARG));
- }
-
- if (in.hasOption(WHERE_ARG)) {
- out.setWhereClause(in.getOptionValue(WHERE_ARG));
- }
-
- if (in.hasOption(TARGET_DIR_ARG)) {
- out.setTargetDir(in.getOptionValue(TARGET_DIR_ARG));
- }
-
- if (in.hasOption(APPEND_ARG)) {
- out.setAppendMode(true);
- }
-
- if (in.hasOption(SQL_QUERY_ARG)) {
- out.setSqlQuery(in.getOptionValue(SQL_QUERY_ARG));
- }
-
- if(in.hasOption(SQL_QUERY_BOUNDARY)) {
- out.setBoundaryQuery(in.getOptionValue(SQL_QUERY_BOUNDARY));
- }
- }
-
- if (in.hasOption(WAREHOUSE_DIR_ARG)) {
- out.setWarehouseDir(in.getOptionValue(WAREHOUSE_DIR_ARG));
- }
-
- if (in.hasOption(FMT_SEQUENCEFILE_ARG)) {
- out.setFileLayout(SqoopOptions.FileLayout.SequenceFile);
- }
-
- if (in.hasOption(FMT_TEXTFILE_ARG)) {
- out.setFileLayout(SqoopOptions.FileLayout.TextFile);
- }
-
- if (in.hasOption(FMT_AVRODATAFILE_ARG)) {
- out.setFileLayout(SqoopOptions.FileLayout.AvroDataFile);
- }
-
- if (in.hasOption(NUM_MAPPERS_ARG)) {
- out.setNumMappers(Integer.parseInt(in.getOptionValue(NUM_MAPPERS_ARG)));
- }
-
- if (in.hasOption(COMPRESS_ARG)) {
- out.setUseCompression(true);
- }
-
- if (in.hasOption(COMPRESSION_CODEC_ARG)) {
- out.setCompressionCodec(in.getOptionValue(COMPRESSION_CODEC_ARG));
- }
-
- if (in.hasOption(DIRECT_SPLIT_SIZE_ARG)) {
- out.setDirectSplitSize(Long.parseLong(in.getOptionValue(
- DIRECT_SPLIT_SIZE_ARG)));
- }
-
- if (in.hasOption(INLINE_LOB_LIMIT_ARG)) {
- out.setInlineLobLimit(Long.parseLong(in.getOptionValue(
- INLINE_LOB_LIMIT_ARG)));
- }
-
- if (in.hasOption(FETCH_SIZE_ARG)) {
- out.setFetchSize(new Integer(in.getOptionValue(FETCH_SIZE_ARG)));
- }
-
- if (in.hasOption(JAR_FILE_NAME_ARG)) {
- out.setExistingJarName(in.getOptionValue(JAR_FILE_NAME_ARG));
- }
-
- applyIncrementalOptions(in, out);
- applyHiveOptions(in, out);
- applyOutputFormatOptions(in, out);
- applyInputFormatOptions(in, out);
- applyCodeGenOptions(in, out, allTables);
- applyHBaseOptions(in, out);
- } catch (NumberFormatException nfe) {
- throw new InvalidOptionsException("Error: expected numeric argument.\n"
- + "Try --help for usage.");
- }
+ super(toolName, allTables);
}
- /**
- * Validate import-specific arguments.
- * @param options the configured SqoopOptions to check
- */
- protected void validateImportOptions(SqoopOptions options)
- throws InvalidOptionsException {
- if (!allTables && options.getTableName() == null
- && options.getSqlQuery() == null) {
- throw new InvalidOptionsException(
- "--table or --" + SQL_QUERY_ARG + " is required for import. "
- + "(Or use sqoop import-all-tables.)"
- + HELP_STR);
- } else if (options.getExistingJarName() != null
- && options.getClassName() == null) {
- throw new InvalidOptionsException("Jar specified with --jar-file, but no "
- + "class specified with --class-name." + HELP_STR);
- } else if (options.getTargetDir() != null
- && options.getWarehouseDir() != null) {
- throw new InvalidOptionsException(
- "--target-dir with --warehouse-dir are incompatible options."
- + HELP_STR);
- } else if (options.getTableName() != null
- && options.getSqlQuery() != null) {
- throw new InvalidOptionsException(
- "Cannot specify --" + SQL_QUERY_ARG + " and --table together."
- + HELP_STR);
- } else if (options.getSqlQuery() != null
- && options.getTargetDir() == null && options.getHBaseTable() == null) {
- throw new InvalidOptionsException(
- "Must specify destination with --target-dir."
- + HELP_STR);
- } else if (options.getSqlQuery() != null && options.doHiveImport()
- && options.getHiveTableName() == null) {
- throw new InvalidOptionsException(
- "When importing a query to Hive, you must specify --"
- + HIVE_TABLE_ARG + "." + HELP_STR);
- } else if (options.getSqlQuery() != null && options.getNumMappers() > 1
- && options.getSplitByCol() == null) {
- throw new InvalidOptionsException(
- "When importing query results in parallel, you must specify --"
- + SPLIT_BY_ARG + "." + HELP_STR);
- } else if (options.isDirect()
- && options.getFileLayout() != SqoopOptions.FileLayout.TextFile
- && options.getConnectString().contains("jdbc:mysql://")) {
- throw new InvalidOptionsException(
- "MySQL direct export currently supports only text output format."
- + "Parameters --as-sequencefile and --as-avrodatafile are not "
- + "supported with --direct params in MySQL case.");
- } else if (!options.getMapColumnJava().isEmpty()
- && options.getFileLayout() == FileLayout.AvroDataFile) {
- throw new InvalidOptionsException(
- "Overriding column types is currently not supported with avro.");
- }
- }
-
- /**
- * Validate the incremental import options.
- */
- private void validateIncrementalOptions(SqoopOptions options)
- throws InvalidOptionsException {
- if (options.getIncrementalMode() != SqoopOptions.IncrementalMode.None
- && options.getIncrementalTestColumn() == null) {
- throw new InvalidOptionsException(
- "For an incremental import, the check column must be specified "
- + "with --" + INCREMENT_COL_ARG + ". " + HELP_STR);
- }
-
- if (options.getIncrementalMode() == SqoopOptions.IncrementalMode.None
- && options.getIncrementalTestColumn() != null) {
- throw new InvalidOptionsException(
- "You must specify an incremental import mode with --"
- + INCREMENT_TYPE_ARG + ". " + HELP_STR);
- }
-
- if (options.getIncrementalMode() != SqoopOptions.IncrementalMode.None
- && options.getTableName() == null) {
- throw new InvalidOptionsException("Incremental imports require a table."
- + HELP_STR);
- }
- }
-
- @Override
- /** {@inheritDoc} */
- public void validateOptions(SqoopOptions options)
- throws InvalidOptionsException {
-
- // If extraArguments is full, check for '--' followed by args for
- // mysqldump or other commands we rely on.
- options.setExtraArgs(getSubcommandArgs(extraArguments));
- int dashPos = getDashPosition(extraArguments);
-
- if (hasUnrecognizedArgs(extraArguments, 0, dashPos)) {
- throw new InvalidOptionsException(HELP_STR);
- }
-
- validateImportOptions(options);
- validateIncrementalOptions(options);
- validateCommonOptions(options);
- validateCodeGenOptions(options);
- validateOutputFormatOptions(options);
- validateHBaseOptions(options);
- validateHiveOptions(options);
- }
}
-
Modified: incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/JobTool.java
URL: http://svn.apache.org/viewvc/incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/JobTool.java?rev=1195852&r1=1195851&r2=1195852&view=diff
==============================================================================
--- incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/JobTool.java (original)
+++ incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/JobTool.java Tue Nov 1 07:10:13 2011
@@ -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
@@ -17,390 +15,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package com.cloudera.sqoop.tool;
-import java.io.IOException;
-
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.TreeMap;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.ParseException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.ToolRunner;
-
-import org.apache.log4j.Category;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-import com.cloudera.sqoop.Sqoop;
-import com.cloudera.sqoop.SqoopOptions;
-import com.cloudera.sqoop.SqoopOptions.InvalidOptionsException;
-import com.cloudera.sqoop.cli.ToolOptions;
-import com.cloudera.sqoop.metastore.hsqldb.HsqldbJobStorage;
-import com.cloudera.sqoop.metastore.JobData;
-import com.cloudera.sqoop.metastore.JobStorage;
-import com.cloudera.sqoop.metastore.JobStorageFactory;
-
/**
- * Tool that creates and executes saved jobs.
+ * @deprecated Moving to use org.apache.sqoop namespace.
*/
-public class JobTool extends BaseSqoopTool {
-
- public static final Log LOG = LogFactory.getLog(
- JobTool.class.getName());
-
- private enum JobOp {
- JobCreate,
- JobDelete,
- JobExecute,
- JobList,
- JobShow,
- };
-
- private Map<String, String> storageDescriptor;
- private String jobName;
- private JobOp operation;
- private JobStorage storage;
-
- public JobTool() {
- super("job");
- }
-
- /**
- * Given an array of strings, return all elements of this
- * array up to (but not including) the first instance of "--".
- */
- private String [] getElementsUpToDoubleDash(String [] array) {
- String [] parseableChildArgv = null;
- for (int i = 0; i < array.length; i++) {
- if ("--".equals(array[i])) {
- parseableChildArgv = Arrays.copyOfRange(array, 0, i);
- break;
- }
- }
-
- if (parseableChildArgv == null) {
- // Didn't find any nested '--'.
- parseableChildArgv = array;
- }
-
- return parseableChildArgv;
- }
-
- /**
- * Given an array of strings, return the first instance
- * of "--" and all following elements.
- * If no "--" exists, return null.
- */
- private String [] getElementsAfterDoubleDash(String [] array) {
- String [] extraChildArgv = null;
- for (int i = 0; i < array.length; i++) {
- if ("--".equals(array[i])) {
- extraChildArgv = Arrays.copyOfRange(array, i, array.length);
- break;
- }
- }
-
- return extraChildArgv;
- }
-
- private int configureChildTool(SqoopOptions childOptions,
- SqoopTool childTool, String [] childArgv) {
- // Within the child arguments there may be a '--' followed by
- // dependent args. Stash them off to the side.
-
- // Everything up to the '--'.
- String [] parseableChildArgv = getElementsUpToDoubleDash(childArgv);
-
- // The '--' and any subsequent args.
- String [] extraChildArgv = getElementsAfterDoubleDash(childArgv);
-
- // Now feed the arguments into the tool itself.
- try {
- childOptions = childTool.parseArguments(parseableChildArgv,
- null, childOptions, false);
- childTool.appendArgs(extraChildArgv);
- childTool.validateOptions(childOptions);
- } catch (ParseException pe) {
- LOG.error("Error parsing arguments to the job-specific tool.");
- LOG.error("See 'sqoop help <tool>' for usage.");
- return 1;
- } catch (SqoopOptions.InvalidOptionsException e) {
- System.err.println(e.getMessage());
- return 1;
- }
-
- return 0; // Success.
- }
-
- private int createJob(SqoopOptions options) throws IOException {
- // In our extraArguments array, we should have a '--' followed by
- // a tool name, and any tool-specific arguments.
- // Create an instance of the named tool and then configure it to
- // get a SqoopOptions out which we will serialize into a job.
- int dashPos = getDashPosition(extraArguments);
- int toolArgPos = dashPos + 1;
- if (null == extraArguments || toolArgPos < 0
- || toolArgPos >= extraArguments.length) {
- LOG.error("No tool specified; cannot create a job.");
- LOG.error("Use: sqoop job --create <job-name> "
- + "-- <tool-name> [tool-args]");
- return 1;
- }
-
- String jobToolName = extraArguments[toolArgPos];
- SqoopTool jobTool = SqoopTool.getTool(jobToolName);
- if (null == jobTool) {
- LOG.error("No such tool available: " + jobToolName);
- return 1;
- }
-
- // Create a SqoopOptions and Configuration based on the current one,
- // but deep-copied. This will be populated within the job.
- SqoopOptions jobOptions = new SqoopOptions();
- jobOptions.setConf(new Configuration(options.getConf()));
-
- // Get the arguments to feed to the child tool.
- String [] childArgs = Arrays.copyOfRange(extraArguments, toolArgPos + 1,
- extraArguments.length);
-
- int confRet = configureChildTool(jobOptions, jobTool, childArgs);
- if (0 != confRet) {
- // Error.
- return confRet;
- }
-
- // Now that the tool is fully configured, materialize the job.
- JobData jobData = new JobData(jobOptions, jobTool);
- this.storage.create(jobName, jobData);
- return 0; // Success.
- }
-
- private int listJobs(SqoopOptions opts) throws IOException {
- List<String> jobNames = storage.list();
- System.out.println("Available jobs:");
- for (String name : jobNames) {
- System.out.println(" " + name);
- }
- return 0;
- }
-
- private int deleteJob(SqoopOptions opts) throws IOException {
- this.storage.delete(jobName);
- return 0;
- }
-
- private int execJob(SqoopOptions opts) throws IOException {
- JobData data = this.storage.read(jobName);
- if (null == data) {
- LOG.error("No such job: " + jobName);
- return 1;
- }
-
- SqoopOptions childOpts = data.getSqoopOptions();
- SqoopTool childTool = data.getSqoopTool();
-
- // Don't overwrite the original SqoopOptions with the
- // arguments; make a child options.
-
- SqoopOptions clonedOpts = (SqoopOptions) childOpts.clone();
- clonedOpts.setParent(childOpts);
-
- int dashPos = getDashPosition(extraArguments);
- String [] childArgv;
- if (dashPos >= extraArguments.length) {
- childArgv = new String[0];
- } else {
- childArgv = Arrays.copyOfRange(extraArguments, dashPos + 1,
- extraArguments.length);
- }
-
- int confRet = configureChildTool(clonedOpts, childTool, childArgv);
- if (0 != confRet) {
- // Error.
- return confRet;
- }
-
- return childTool.run(clonedOpts);
- }
-
- private int showJob(SqoopOptions opts) throws IOException {
- JobData data = this.storage.read(jobName);
- if (null == data) {
- LOG.error("No such job: " + jobName);
- return 1;
- }
-
- SqoopOptions childOpts = data.getSqoopOptions();
- SqoopTool childTool = data.getSqoopTool();
-
- System.out.println("Job: " + jobName);
- System.out.println("Tool: " + childTool.getToolName());
-
- System.out.println("Options:");
- System.out.println("----------------------------");
- Properties props = childOpts.writeProperties();
- for (Map.Entry<Object, Object> entry : props.entrySet()) {
- System.out.println(entry.getKey().toString() + " = " + entry.getValue());
- }
-
- // TODO: This does not show entries in the Configuration
- // (SqoopOptions.getConf()) which were stored as different from the
- // default.
-
- return 0;
- }
-
- @Override
- /** {@inheritDoc} */
- public int run(SqoopOptions options) {
- // Get a JobStorage instance to use to materialize this job.
- JobStorageFactory ssf = new JobStorageFactory(options.getConf());
- this.storage = ssf.getJobStorage(storageDescriptor);
- if (null == this.storage) {
- LOG.error("There is no JobStorage implementation available");
- LOG.error("that can read your specified storage descriptor.");
- LOG.error("Don't know where to save this job info! You may");
- LOG.error("need to specify the connect string with --meta-connect.");
- return 1;
- }
-
- try {
- // Open the storage layer.
- this.storage.open(this.storageDescriptor);
-
- // And now determine what operation to perform with it.
- switch (operation) {
- case JobCreate:
- return createJob(options);
- case JobDelete:
- return deleteJob(options);
- case JobExecute:
- return execJob(options);
- case JobList:
- return listJobs(options);
- case JobShow:
- return showJob(options);
- default:
- LOG.error("Undefined job operation: " + operation);
- return 1;
- }
- } catch (IOException ioe) {
- LOG.error("I/O error performing job operation: "
- + StringUtils.stringifyException(ioe));
- return 1;
- } finally {
- if (null != this.storage) {
- try {
- storage.close();
- } catch (IOException ioe) {
- LOG.warn("IOException closing JobStorage: "
- + StringUtils.stringifyException(ioe));
- }
- }
- }
- }
-
- @Override
- /** Configure the command-line arguments we expect to receive */
- public void configureOptions(ToolOptions toolOptions) {
- toolOptions.addUniqueOptions(getJobOptions());
- }
-
- @Override
- /** {@inheritDoc} */
- public void applyOptions(CommandLine in, SqoopOptions out)
- throws InvalidOptionsException {
-
- if (in.hasOption(VERBOSE_ARG)) {
- // Immediately switch into DEBUG logging.
- Category sqoopLogger = Logger.getLogger(
- Sqoop.class.getName()).getParent();
- sqoopLogger.setLevel(Level.DEBUG);
- LOG.debug("Enabled debug logging.");
- }
-
- if (in.hasOption(HELP_ARG)) {
- ToolOptions toolOpts = new ToolOptions();
- configureOptions(toolOpts);
- printHelp(toolOpts);
- throw new InvalidOptionsException("");
- }
-
- this.storageDescriptor = new TreeMap<String, String>();
-
- if (in.hasOption(STORAGE_METASTORE_ARG)) {
- this.storageDescriptor.put(HsqldbJobStorage.META_CONNECT_KEY,
- in.getOptionValue(STORAGE_METASTORE_ARG));
- }
-
- // These are generated via an option group; exactly one
- // of this exhaustive list will always be selected.
- if (in.hasOption(JOB_CMD_CREATE_ARG)) {
- this.operation = JobOp.JobCreate;
- this.jobName = in.getOptionValue(JOB_CMD_CREATE_ARG);
- } else if (in.hasOption(JOB_CMD_DELETE_ARG)) {
- this.operation = JobOp.JobDelete;
- this.jobName = in.getOptionValue(JOB_CMD_DELETE_ARG);
- } else if (in.hasOption(JOB_CMD_EXEC_ARG)) {
- this.operation = JobOp.JobExecute;
- this.jobName = in.getOptionValue(JOB_CMD_EXEC_ARG);
- } else if (in.hasOption(JOB_CMD_LIST_ARG)) {
- this.operation = JobOp.JobList;
- } else if (in.hasOption(JOB_CMD_SHOW_ARG)) {
- this.operation = JobOp.JobShow;
- this.jobName = in.getOptionValue(JOB_CMD_SHOW_ARG);
- }
- }
-
- @Override
- /** {@inheritDoc} */
- public void validateOptions(SqoopOptions options)
- throws InvalidOptionsException {
-
- if (null == operation
- || (null == this.jobName && operation != JobOp.JobList)) {
- throw new InvalidOptionsException("No job operation specified"
- + HELP_STR);
- }
-
- if (operation == JobOp.JobCreate) {
- // Check that we have a '--' followed by at least a tool name.
- if (extraArguments == null || extraArguments.length == 0) {
- throw new InvalidOptionsException(
- "Expected: -- <tool-name> [tool-args] "
- + HELP_STR);
- }
- }
-
- int dashPos = getDashPosition(extraArguments);
- if (hasUnrecognizedArgs(extraArguments, 0, dashPos)) {
- throw new InvalidOptionsException(HELP_STR);
- }
- }
-
- @Override
- /** {@inheritDoc} */
- public void printHelp(ToolOptions opts) {
- System.out.println("usage: sqoop " + getToolName()
- + " [GENERIC-ARGS] [JOB-ARGS] [-- [<tool-name>] [TOOL-ARGS]]");
- System.out.println("");
-
- opts.printHelp();
-
- System.out.println("");
- System.out.println("Generic Hadoop command-line arguments:");
- System.out.println("(must preceed any tool-specific arguments)");
- ToolRunner.printGenericCommandUsage(System.out);
- }
+public class JobTool
+ extends org.apache.sqoop.tool.JobTool {
}
-
Modified: incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ListDatabasesTool.java
URL: http://svn.apache.org/viewvc/incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ListDatabasesTool.java?rev=1195852&r1=1195851&r2=1195852&view=diff
==============================================================================
--- incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ListDatabasesTool.java (original)
+++ incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ListDatabasesTool.java Tue Nov 1 07:10:13 2011
@@ -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
@@ -17,76 +15,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package com.cloudera.sqoop.tool;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import com.cloudera.sqoop.SqoopOptions;
-import com.cloudera.sqoop.SqoopOptions.InvalidOptionsException;
-import com.cloudera.sqoop.cli.ToolOptions;
-
/**
- * Tool that lists available databases on a server.
+ * @deprecated Moving to use org.apache.sqoop namespace.
*/
-public class ListDatabasesTool extends BaseSqoopTool {
-
- public static final Log LOG = LogFactory.getLog(
- ListDatabasesTool.class.getName());
-
- public ListDatabasesTool() {
- super("list-databases");
- }
-
- @Override
- /** {@inheritDoc} */
- public int run(SqoopOptions options) {
- if (!init(options)) {
- return 1;
- }
-
- try {
- String [] databases = manager.listDatabases();
- if (null == databases) {
- System.err.println("Could not retrieve database list from server");
- LOG.error("manager.listDatabases() returned null");
- return 1;
- } else {
- for (String db : databases) {
- System.out.println(db);
- }
- }
- } finally {
- destroy(options);
- }
-
- return 0;
- }
-
- @Override
- /** Configure the command-line arguments we expect to receive */
- public void configureOptions(ToolOptions toolOptions) {
- toolOptions.addUniqueOptions(getCommonOptions());
- }
-
- @Override
- /** {@inheritDoc} */
- public void applyOptions(CommandLine in, SqoopOptions out)
- throws InvalidOptionsException {
- applyCommonOptions(in, out);
- }
-
- @Override
- /** {@inheritDoc} */
- public void validateOptions(SqoopOptions options)
- throws InvalidOptionsException {
-
- if (hasUnrecognizedArgs(extraArguments)) {
- throw new InvalidOptionsException(HELP_STR);
- }
- validateCommonOptions(options);
- }
+public class ListDatabasesTool
+ extends org.apache.sqoop.tool.ListDatabasesTool {
}
-
Modified: incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ListTablesTool.java
URL: http://svn.apache.org/viewvc/incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ListTablesTool.java?rev=1195852&r1=1195851&r2=1195852&view=diff
==============================================================================
--- incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ListTablesTool.java (original)
+++ incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ListTablesTool.java Tue Nov 1 07:10:13 2011
@@ -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
@@ -17,76 +15,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package com.cloudera.sqoop.tool;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import com.cloudera.sqoop.SqoopOptions;
-import com.cloudera.sqoop.SqoopOptions.InvalidOptionsException;
-import com.cloudera.sqoop.cli.ToolOptions;
-
/**
- * Tool that lists available tables in a database.
+ * @deprecated Moving to use org.apache.sqoop namespace.
*/
-public class ListTablesTool extends BaseSqoopTool {
-
- public static final Log LOG = LogFactory.getLog(
- ListTablesTool.class.getName());
-
- public ListTablesTool() {
- super("list-tables");
- }
-
- @Override
- /** {@inheritDoc} */
- public int run(SqoopOptions options) {
- if (!init(options)) {
- return 1;
- }
-
- try {
- String [] tables = manager.listTables();
- if (null == tables) {
- System.err.println("Could not retrieve tables list from server");
- LOG.error("manager.listTables() returned null");
- return 1;
- } else {
- for (String tbl : tables) {
- System.out.println(tbl);
- }
- }
- } finally {
- destroy(options);
- }
-
- return 0;
- }
-
- @Override
- /** Configure the command-line arguments we expect to receive */
- public void configureOptions(ToolOptions toolOptions) {
- toolOptions.addUniqueOptions(getCommonOptions());
- }
-
- @Override
- /** {@inheritDoc} */
- public void applyOptions(CommandLine in, SqoopOptions out)
- throws InvalidOptionsException {
- applyCommonOptions(in, out);
- }
-
- @Override
- /** {@inheritDoc} */
- public void validateOptions(SqoopOptions options)
- throws InvalidOptionsException {
- if (hasUnrecognizedArgs(extraArguments)) {
- throw new InvalidOptionsException(HELP_STR);
- }
-
- validateCommonOptions(options);
- }
+public class ListTablesTool
+ extends org.apache.sqoop.tool.ListTablesTool {
}
-
Modified: incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/MergeTool.java
URL: http://svn.apache.org/viewvc/incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/MergeTool.java?rev=1195852&r1=1195851&r2=1195852&view=diff
==============================================================================
--- incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/MergeTool.java (original)
+++ incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/MergeTool.java Tue Nov 1 07:10:13 2011
@@ -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
@@ -17,223 +15,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package com.cloudera.sqoop.tool;
-import java.io.IOException;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.OptionBuilder;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import org.apache.hadoop.util.StringUtils;
-
-import org.apache.log4j.Category;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-import com.cloudera.sqoop.Sqoop;
-import com.cloudera.sqoop.SqoopOptions;
-import com.cloudera.sqoop.SqoopOptions.InvalidOptionsException;
-import com.cloudera.sqoop.cli.RelatedOptions;
-import com.cloudera.sqoop.cli.ToolOptions;
-import com.cloudera.sqoop.mapreduce.MergeJob;
-
/**
- * Tool that merges a more recent dataset on top of an older one.
+ * @deprecated Moving to use org.apache.sqoop namespace.
*/
-public class MergeTool extends BaseSqoopTool {
-
- public static final Log LOG = LogFactory.getLog(MergeTool.class.getName());
-
- public MergeTool() {
- this("merge");
- }
-
- public MergeTool(String toolName) {
- super(toolName);
- }
-
- @Override
- /** {@inheritDoc} */
- public int run(SqoopOptions options) {
- try {
- // Configure and execute a MapReduce job to merge these datasets.
- MergeJob mergeJob = new MergeJob(options);
- if (!mergeJob.runMergeJob()) {
- LOG.error("MapReduce job failed!");
- return 1;
- }
- } catch (IOException ioe) {
- LOG.error("Encountered IOException running import job: "
- + StringUtils.stringifyException(ioe));
- if (System.getProperty(Sqoop.SQOOP_RETHROW_PROPERTY) != null) {
- throw new RuntimeException(ioe);
- } else {
- return 1;
- }
- }
-
- return 0;
- }
-
- /**
- * Construct the set of options that control imports, either of one
- * table or a batch of tables.
- * @return the RelatedOptions that can be used to parse the import
- * arguments.
- */
- protected RelatedOptions getMergeOptions() {
- // Imports
- RelatedOptions mergeOpts = new RelatedOptions("Merge arguments");
-
- mergeOpts.addOption(OptionBuilder.withArgName("file")
- .hasArg().withDescription("Load class from specified jar file")
- .withLongOpt(JAR_FILE_NAME_ARG)
- .create());
-
- mergeOpts.addOption(OptionBuilder.withArgName("name")
- .hasArg().withDescription("Specify record class name to load")
- .withLongOpt(CLASS_NAME_ARG)
- .create());
-
- mergeOpts.addOption(OptionBuilder.withArgName("path")
- .hasArg().withDescription("Path to the more recent data set")
- .withLongOpt(NEW_DATASET_ARG)
- .create());
-
- mergeOpts.addOption(OptionBuilder.withArgName("path")
- .hasArg().withDescription("Path to the older data set")
- .withLongOpt(OLD_DATASET_ARG)
- .create());
-
- mergeOpts.addOption(OptionBuilder.withArgName("path")
- .hasArg().withDescription("Destination path for merged results")
- .withLongOpt(TARGET_DIR_ARG)
- .create());
-
- mergeOpts.addOption(OptionBuilder.withArgName("column")
- .hasArg().withDescription("Key column to use to join results")
- .withLongOpt(MERGE_KEY_ARG)
- .create());
-
- // Since the "common" options aren't used in the merge tool,
- // add these settings here.
- mergeOpts.addOption(OptionBuilder
- .withDescription("Print more information while working")
- .withLongOpt(VERBOSE_ARG)
- .create());
- mergeOpts.addOption(OptionBuilder
- .withDescription("Print usage instructions")
- .withLongOpt(HELP_ARG)
- .create());
-
- return mergeOpts;
- }
-
-
- @Override
- /** Configure the command-line arguments we expect to receive */
- public void configureOptions(ToolOptions toolOptions) {
- toolOptions.addUniqueOptions(getMergeOptions());
- }
-
-
- @Override
- /** {@inheritDoc} */
- public void applyOptions(CommandLine in, SqoopOptions out)
- throws InvalidOptionsException {
-
- if (in.hasOption(VERBOSE_ARG)) {
- // Immediately switch into DEBUG logging.
- Category sqoopLogger = Logger.getLogger(
- Sqoop.class.getName()).getParent();
- sqoopLogger.setLevel(Level.DEBUG);
- LOG.debug("Enabled debug logging.");
- }
-
- if (in.hasOption(HELP_ARG)) {
- ToolOptions toolOpts = new ToolOptions();
- configureOptions(toolOpts);
- printHelp(toolOpts);
- throw new InvalidOptionsException("");
- }
-
- if (in.hasOption(JAR_FILE_NAME_ARG)) {
- out.setExistingJarName(in.getOptionValue(JAR_FILE_NAME_ARG));
- }
-
- if (in.hasOption(CLASS_NAME_ARG)) {
- out.setClassName(in.getOptionValue(CLASS_NAME_ARG));
- }
-
- if (in.hasOption(NEW_DATASET_ARG)) {
- out.setMergeNewPath(in.getOptionValue(NEW_DATASET_ARG));
- }
-
- if (in.hasOption(OLD_DATASET_ARG)) {
- out.setMergeOldPath(in.getOptionValue(OLD_DATASET_ARG));
- }
-
- if (in.hasOption(TARGET_DIR_ARG)) {
- out.setTargetDir(in.getOptionValue(TARGET_DIR_ARG));
- }
-
- if (in.hasOption(MERGE_KEY_ARG)) {
- out.setMergeKeyCol(in.getOptionValue(MERGE_KEY_ARG));
- }
- }
-
- /**
- * Validate merge-specific arguments.
- * @param options the configured SqoopOptions to check
- */
- protected void validateMergeOptions(SqoopOptions options)
- throws InvalidOptionsException {
-
- if (options.getMergeNewPath() == null) {
- throw new InvalidOptionsException("Must set the new dataset path with --"
- + NEW_DATASET_ARG + "." + HELP_STR);
- }
-
- if (options.getMergeOldPath() == null) {
- throw new InvalidOptionsException("Must set the old dataset path with --"
- + OLD_DATASET_ARG + "." + HELP_STR);
- }
-
- if (options.getMergeKeyCol() == null) {
- throw new InvalidOptionsException("Must set the merge key column with --"
- + MERGE_KEY_ARG + "." + HELP_STR);
- }
-
- if (options.getTargetDir() == null) {
- throw new InvalidOptionsException("Must set the target directory with --"
- + TARGET_DIR_ARG + "." + HELP_STR);
- }
-
- if (options.getClassName() == null) {
- throw new InvalidOptionsException("Must set the SqoopRecord class "
- + "implementation to use with --" + CLASS_NAME_ARG + "."
- + HELP_STR);
- }
- }
-
- @Override
- /** {@inheritDoc} */
- public void validateOptions(SqoopOptions options)
- throws InvalidOptionsException {
-
- // If extraArguments is full, check for '--' followed by args for
- // mysqldump or other commands we rely on.
- options.setExtraArgs(getSubcommandArgs(extraArguments));
- int dashPos = getDashPosition(extraArguments);
-
- if (hasUnrecognizedArgs(extraArguments, 0, dashPos)) {
- throw new InvalidOptionsException(HELP_STR);
- }
-
- validateMergeOptions(options);
- }
+public class MergeTool
+ extends org.apache.sqoop.tool.MergeTool {
}
-
Modified: incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/MetastoreTool.java
URL: http://svn.apache.org/viewvc/incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/MetastoreTool.java?rev=1195852&r1=1195851&r2=1195852&view=diff
==============================================================================
--- incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/MetastoreTool.java (original)
+++ incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/MetastoreTool.java Tue Nov 1 07:10:13 2011
@@ -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
@@ -17,78 +15,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package com.cloudera.sqoop.tool;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.OptionBuilder;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import com.cloudera.sqoop.SqoopOptions;
-import com.cloudera.sqoop.SqoopOptions.InvalidOptionsException;
-import com.cloudera.sqoop.cli.RelatedOptions;
-import com.cloudera.sqoop.cli.ToolOptions;
-
-import com.cloudera.sqoop.metastore.hsqldb.HsqldbMetaStore;
-
/**
- * Tool that runs a standalone Sqoop metastore.
+ * @deprecated Moving to use org.apache.sqoop namespace.
*/
-public class MetastoreTool extends BaseSqoopTool {
-
- public static final Log LOG = LogFactory.getLog(
- MetastoreTool.class.getName());
-
- private HsqldbMetaStore metastore;
-
- // If set to true, shut an existing metastore down.
- private boolean shutdown = false;
-
- public MetastoreTool() {
- super("metastore");
- }
-
- @Override
- /** {@inheritDoc} */
- public int run(SqoopOptions options) {
- metastore = new HsqldbMetaStore(options.getConf());
- if (shutdown) {
- LOG.info("Shutting down metastore...");
- metastore.shutdown();
- } else {
- metastore.start();
- metastore.waitForServer();
- LOG.info("Server thread has quit.");
- }
- return 0;
- }
-
- @Override
- /** Configure the command-line arguments we expect to receive */
- public void configureOptions(ToolOptions toolOptions) {
- RelatedOptions opts = new RelatedOptions("metastore arguments");
- opts.addOption(OptionBuilder
- .withDescription("Cleanly shut down a running metastore")
- .withLongOpt(METASTORE_SHUTDOWN_ARG)
- .create());
-
- toolOptions.addUniqueOptions(opts);
- }
-
- @Override
- /** {@inheritDoc} */
- public void applyOptions(CommandLine in, SqoopOptions out)
- throws InvalidOptionsException {
- if (in.hasOption(METASTORE_SHUTDOWN_ARG)) {
- this.shutdown = true;
- }
- }
-
- @Override
- /** {@inheritDoc} */
- public void validateOptions(SqoopOptions options)
- throws InvalidOptionsException {
- }
+public class MetastoreTool
+ extends org.apache.sqoop.tool.MetastoreTool {
}
-
Modified: incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/SqoopTool.java
URL: http://svn.apache.org/viewvc/incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/SqoopTool.java?rev=1195852&r1=1195851&r2=1195852&view=diff
==============================================================================
--- incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/SqoopTool.java (original)
+++ incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/SqoopTool.java Tue Nov 1 07:10:13 2011
@@ -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
@@ -17,491 +15,39 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package com.cloudera.sqoop.tool;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.Reader;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
import java.util.Set;
-import java.util.TreeMap;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.ParseException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.ToolRunner;
-
-import com.cloudera.sqoop.SqoopOptions;
-import com.cloudera.sqoop.SqoopOptions.InvalidOptionsException;
-import com.cloudera.sqoop.cli.SqoopParser;
-import com.cloudera.sqoop.cli.ToolOptions;
-import com.cloudera.sqoop.config.ConfigurationHelper;
-
-import com.cloudera.sqoop.util.ClassLoaderStack;
/**
- * Base class for Sqoop subprograms (e.g., SqoopImport, SqoopExport, etc.)
- * Allows subprograms to configure the arguments they accept and
- * provides an entry-point to the subprogram.
+ * @deprecated Moving to use org.apache.sqoop namespace.
*/
-public abstract class SqoopTool {
-
- public static final Log LOG = LogFactory.getLog(SqoopTool.class.getName());
-
- /**
- * Configuration key that specifies the set of ToolPlugin instances to load
- * before determining which SqoopTool instance to load.
- */
- public static final String TOOL_PLUGINS_KEY = "sqoop.tool.plugins";
-
- private static final Map<String, Class<? extends SqoopTool>> TOOLS;
- private static final Map<String, String> DESCRIPTIONS;
-
- static {
- // All SqoopTool instances should be registered here so that
- // they can be found internally.
- TOOLS = new TreeMap<String, Class<? extends SqoopTool>>();
- DESCRIPTIONS = new TreeMap<String, String>();
-
- registerTool("codegen", CodeGenTool.class,
- "Generate code to interact with database records");
- registerTool("create-hive-table", CreateHiveTableTool.class,
- "Import a table definition into Hive");
- registerTool("eval", EvalSqlTool.class,
- "Evaluate a SQL statement and display the results");
- registerTool("export", ExportTool.class,
- "Export an HDFS directory to a database table");
- registerTool("import", ImportTool.class,
- "Import a table from a database to HDFS");
- registerTool("import-all-tables", ImportAllTablesTool.class,
- "Import tables from a database to HDFS");
- registerTool("help", HelpTool.class, "List available commands");
- registerTool("list-databases", ListDatabasesTool.class,
- "List available databases on a server");
- registerTool("list-tables", ListTablesTool.class,
- "List available tables in a database");
- registerTool("merge", MergeTool.class,
- "Merge results of incremental imports");
- registerTool("metastore", MetastoreTool.class,
- "Run a standalone Sqoop metastore");
- registerTool("job", JobTool.class,
- "Work with saved jobs");
- registerTool("version", VersionTool.class,
- "Display version information");
- }
-
- /**
- * Add a tool to the available set of SqoopTool instances.
- * @param toolName the name the user access the tool through.
- * @param cls the class providing the tool.
- * @param description a user-friendly description of the tool's function.
- */
- private static void registerTool(String toolName,
- Class<? extends SqoopTool> cls, String description) {
- Class<? extends SqoopTool> existing = TOOLS.get(toolName);
- if (null != existing) {
- // Already have a tool with this name. Refuse to start.
- throw new RuntimeException("A plugin is attempting to register a tool "
- + "with name " + toolName + ", but this tool already exists ("
- + existing.getName() + ")");
- }
-
- TOOLS.put(toolName, cls);
- DESCRIPTIONS.put(toolName, description);
- }
-
- /**
- * Add tool to available set of SqoopTool instances using the ToolDesc
- * struct as the sole argument.
- */
- private static void registerTool(ToolDesc toolDescription) {
- registerTool(toolDescription.getName(), toolDescription.getToolClass(),
- toolDescription.getDesc());
- }
-
- /**
- * Load plugins referenced in sqoop-site.xml or other config (e.g., tools.d/),
- * to allow external tool definitions.
- *
- * @return the Configuration used to load the plugins.
- */
- public static Configuration loadPlugins(Configuration conf) {
- conf = loadPluginsFromConfDir(conf);
- List<ToolPlugin> plugins = conf.getInstances(TOOL_PLUGINS_KEY,
- ToolPlugin.class);
- for (ToolPlugin plugin : plugins) {
- LOG.debug("Loading plugin: " + plugin.getClass().getName());
- List<ToolDesc> descriptions = plugin.getTools();
- for (ToolDesc desc : descriptions) {
- LOG.debug(" Adding tool: " + desc.getName()
- + " -> " + desc.getToolClass().getName());
- registerTool(desc);
- }
- }
-
- return conf;
- }
+public abstract class SqoopTool
+ extends org.apache.sqoop.tool.SqoopTool {
- /**
- * If $SQOOP_CONF_DIR/tools.d/ exists and sqoop.tool.plugins is not set,
- * then we look through the files in that directory; they should contain
- * lines of the form 'plugin.class.name[=/path/to/containing.jar]'.
- *
- * <p>Put all plugin.class.names into the Configuration, and load any
- * specified jars into the ClassLoader.
- * </p>
- *
- * @param conf the current configuration to populate with class names.
- * @return conf again, after possibly populating sqoop.tool.plugins.
- */
- private static Configuration loadPluginsFromConfDir(Configuration conf) {
- if (conf.get(TOOL_PLUGINS_KEY) != null) {
- LOG.debug(TOOL_PLUGINS_KEY + " is set; ignoring tools.d");
- return conf;
- }
+ public static final String TOOL_PLUGINS_KEY =
+ org.apache.sqoop.tool.SqoopTool.TOOL_PLUGINS_KEY;
- String confDirName = System.getenv("SQOOP_CONF_DIR");
- if (null == confDirName) {
- LOG.warn("$SQOOP_CONF_DIR has not been set in the environment. "
- + "Cannot check for additional configuration.");
- return conf;
- }
-
- File confDir = new File(confDirName);
- File toolsDir = new File(confDir, "tools.d");
-
- if (toolsDir.exists() && toolsDir.isDirectory()) {
- // We have a tools.d subdirectory. Get the file list, sort it,
- // and process them in order.
- String [] fileNames = toolsDir.list();
- Arrays.sort(fileNames);
-
- for (String fileName : fileNames) {
- File f = new File(toolsDir, fileName);
- if (f.isFile()) {
- loadPluginsFromFile(conf, f);
- }
- }
- }
-
- // Set the classloader in this configuration so that it will use
- // the jars we just loaded in.
- conf.setClassLoader(Thread.currentThread().getContextClassLoader());
- return conf;
- }
-
- /**
- * Read the specified file and extract any ToolPlugin implementation
- * names from there.
- * @param conf the configuration to populate.
- * @param f the file containing the configuration data to add.
- */
- private static void loadPluginsFromFile(Configuration conf, File f) {
- Reader r = null;
- try {
- // The file format is actually Java properties-file syntax.
- r = new InputStreamReader(new FileInputStream(f));
- Properties props = new Properties();
- props.load(r);
-
- for (Map.Entry<Object, Object> entry : props.entrySet()) {
- // Each key is a ToolPlugin class name.
- // Each value, if set, is the jar that contains it.
- String plugin = entry.getKey().toString();
- addPlugin(conf, plugin);
-
- String jarName = entry.getValue().toString();
- if (jarName.length() > 0) {
- ClassLoaderStack.addJarFile(jarName, plugin);
- LOG.debug("Added plugin " + plugin + " in jar " + jarName
- + " specified by " + f);
- } else if (LOG.isDebugEnabled()) {
- LOG.debug("Added plugin " + plugin + " specified by " + f);
- }
- }
- } catch (IOException ioe) {
- LOG.error("Error loading ToolPlugin information from file "
- + f + ": " + StringUtils.stringifyException(ioe));
- } finally {
- if (null != r) {
- try {
- r.close();
- } catch (IOException ioe) {
- LOG.warn("Error closing file " + f + ": " + ioe);
- }
- }
- }
- }
-
- /**
- * Add the specified plugin class name to the configuration string
- * listing plugin classes.
- */
- private static void addPlugin(Configuration conf, String pluginName) {
- String existingPlugins = conf.get(TOOL_PLUGINS_KEY);
- String newPlugins = null;
- if (null == existingPlugins || existingPlugins.length() == 0) {
- newPlugins = pluginName;
- } else {
- newPlugins = existingPlugins + "," + pluginName;
- }
-
- conf.set(TOOL_PLUGINS_KEY, newPlugins);
- }
-
- /**
- * @return the list of available tools.
- */
public static final Set<String> getToolNames() {
- return TOOLS.keySet();
+ return org.apache.sqoop.tool.SqoopTool.getToolNames();
}
- /**
- * @return the SqoopTool instance with the provided name, or null
- * if no such tool exists.
- */
public static final SqoopTool getTool(String toolName) {
- Class<? extends SqoopTool> cls = TOOLS.get(toolName);
- try {
- if (null != cls) {
- SqoopTool tool = cls.newInstance();
- tool.setToolName(toolName);
- return tool;
- }
- } catch (Exception e) {
- LOG.error(StringUtils.stringifyException(e));
- return null;
- }
-
- return null;
+ return (SqoopTool)org.apache.sqoop.tool.SqoopTool.getTool(toolName);
}
- /**
- * @return the user-friendly description for a tool, or null if the tool
- * cannot be found.
- */
public static final String getToolDescription(String toolName) {
- return DESCRIPTIONS.get(toolName);
+ return org.apache.sqoop.tool.SqoopTool.getToolDescription(toolName);
}
- /** The name of the current tool. */
- private String toolName;
-
- /** Arguments that remained unparsed after parseArguments. */
- protected String [] extraArguments;
-
public SqoopTool() {
- this.toolName = "<" + this.getClass().getName() + ">";
+ super();
}
public SqoopTool(String name) {
- this.toolName = name;
+ super(name);
}
- public String getToolName() {
- return this.toolName;
- }
-
- protected void setToolName(String name) {
- this.toolName = name;
- }
-
- /**
- * Main body of code to run the tool.
- * @param options the SqoopOptions configured via
- * configureOptions()/applyOptions().
- * @return an integer return code for external programs to consume. 0
- * represents success; nonzero means failure.
- */
- public abstract int run(SqoopOptions options);
-
- /**
- * Configure the command-line arguments we expect to receive.
- * @param opts a ToolOptions that should be populated with sets of
- * RelatedOptions for the tool.
- */
- public void configureOptions(ToolOptions opts) {
- // Default implementation does nothing.
- }
-
- /**
- * Print the help message for this tool.
- * @param opts the configured tool options
- */
- public void printHelp(ToolOptions opts) {
- System.out.println("usage: sqoop " + getToolName()
- + " [GENERIC-ARGS] [TOOL-ARGS]");
- System.out.println("");
-
- opts.printHelp();
-
- System.out.println("");
- System.out.println("Generic Hadoop command-line arguments:");
- System.out.println("(must preceed any tool-specific arguments)");
- ToolRunner.printGenericCommandUsage(System.out);
- }
-
- /** Generate the SqoopOptions containing actual argument values from
- * the extracted CommandLine arguments.
- * @param in the CLI CommandLine that contain the user's set Options.
- * @param out the SqoopOptions with all fields applied.
- * @throws InvalidOptionsException if there's a problem.
- */
- public void applyOptions(CommandLine in, SqoopOptions out)
- throws InvalidOptionsException {
- // Default implementation does nothing.
- }
-
- /**
- * Validates options and ensures that any required options are
- * present and that any mutually-exclusive options are not selected.
- * @throws InvalidOptionsException if there's a problem.
- */
- public void validateOptions(SqoopOptions options)
- throws InvalidOptionsException {
- // Default implementation does nothing.
- }
-
- /**
- * Configures a SqoopOptions according to the specified arguments.
- * Reads a set of arguments and uses them to configure a SqoopOptions
- * and its embedded configuration (i.e., through GenericOptionsParser.)
- * Stores any unparsed arguments in the extraArguments field.
- *
- * @param args the arguments to parse.
- * @param conf if non-null, set as the configuration for the returned
- * SqoopOptions.
- * @param in a (perhaps partially-configured) SqoopOptions. If null,
- * then a new SqoopOptions will be used. If this has a null configuration
- * and conf is null, then a new Configuration will be inserted in this.
- * @param useGenericOptions if true, will also parse generic Hadoop
- * options into the Configuration.
- * @return a SqoopOptions that is fully configured by a given tool.
- */
- public SqoopOptions parseArguments(String [] args,
- Configuration conf, SqoopOptions in, boolean useGenericOptions)
- throws ParseException, SqoopOptions.InvalidOptionsException {
- SqoopOptions out = in;
-
- if (null == out) {
- out = new SqoopOptions();
- }
-
- if (null != conf) {
- // User specified a configuration; use it and override any conf
- // that may have been in the SqoopOptions.
- out.setConf(conf);
- } else if (null == out.getConf()) {
- // User did not specify a configuration, but neither did the
- // SqoopOptions. Fabricate a new one.
- out.setConf(new Configuration());
- }
-
- // This tool is the "active" tool; bind it in the SqoopOptions.
- out.setActiveSqoopTool(this);
-
- String [] toolArgs = args; // args after generic parser is done.
- if (useGenericOptions) {
- try {
- toolArgs = ConfigurationHelper.parseGenericOptions(
- out.getConf(), args);
- } catch (IOException ioe) {
- ParseException pe = new ParseException(
- "Could not parse generic arguments");
- pe.initCause(ioe);
- throw pe;
- }
- }
-
- // Parse tool-specific arguments.
- ToolOptions toolOptions = new ToolOptions();
- configureOptions(toolOptions);
- CommandLineParser parser = new SqoopParser();
- CommandLine cmdLine = parser.parse(toolOptions.merge(), toolArgs, true);
- applyOptions(cmdLine, out);
- this.extraArguments = cmdLine.getArgs();
- return out;
- }
-
- /**
- * Append 'extra' to extraArguments.
- */
- public void appendArgs(String [] extra) {
- int existingLen =
- (this.extraArguments == null) ? 0 : this.extraArguments.length;
- int newLen = (extra == null) ? 0 : extra.length;
- String [] newExtra = new String[existingLen + newLen];
-
- if (null != this.extraArguments) {
- System.arraycopy(this.extraArguments, 0, newExtra, 0, existingLen);
- }
-
- if (null != extra) {
- System.arraycopy(extra, 0, newExtra, existingLen, newLen);
- }
-
- this.extraArguments = newExtra;
- }
-
- /**
- * Allow a tool to specify a set of dependency jar filenames. This is used
- * to allow tools to bundle arbitrary dependency jars necessary for a
- * MapReduce job executed by Sqoop. The jar containing the SqoopTool
- * instance itself will already be handled by Sqoop.
- *
- * <p>Called by JobBase.cacheJars().</p>
- *
- * <p>
- * This does not load the jars into the current VM; they are assumed to be
- * already on the classpath if they are needed on the client side (or
- * otherwise classloaded by the tool itself). This is purely to specify jars
- * necessary to be added to the distributed cache. The tool itself can
- * classload these jars by running loadDependencyJars().
- * </p>
- *
- * <p>See also: c.c.s.util.Jars.getJarPathForClass()</p>
- */
- public List<String> getDependencyJars() {
- // Default behavior: no additional dependencies.
- return Collections.emptyList();
- }
-
- /**
- * Loads dependency jars specified by getDependencyJars() into the current
- * classloader stack. May optionally be called by a [third-party] tool
- * before doing work, to ensure that all of its dependencies get classloaded
- * properly. Note that dependencies will not be available until after the
- * tool is already constructed.
- */
- protected void loadDependencyJars(SqoopOptions options) throws IOException {
- List<String> deps = getDependencyJars();
- if (null == deps) {
- return;
- }
-
- for (String depFilename : deps) {
- LOG.debug("Loading dependency: " + depFilename);
- ClassLoaderStack.addJarFile(depFilename, null);
- }
-
- options.getConf().setClassLoader(
- Thread.currentThread().getContextClassLoader());
- }
-
- @Override
- public String toString() {
- return getToolName();
- }
}
-
Modified: incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ToolDesc.java
URL: http://svn.apache.org/viewvc/incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ToolDesc.java?rev=1195852&r1=1195851&r2=1195852&view=diff
==============================================================================
--- incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ToolDesc.java (original)
+++ incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ToolDesc.java Tue Nov 1 07:10:13 2011
@@ -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
@@ -17,46 +15,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package com.cloudera.sqoop.tool;
/**
- * Describes a SqoopTool.
+ * @deprecated Moving to use org.apache.sqoop namespace.
*/
-public final class ToolDesc {
- private final String toolName;
- private final Class<? extends SqoopTool> toolClass;
- private final String description;
-
+public class ToolDesc
+ extends org.apache.sqoop.tool.ToolDesc {
- /**
- * Main c'tor; sets all fields that describe a SqoopTool.
- */
public ToolDesc(String name, Class<? extends SqoopTool> cls, String desc) {
- this.toolName = name;
- this.toolClass = cls;
- this.description = desc;
- }
-
- /**
- * @return the name used to invoke the tool (e.g., 'sqoop <foo>')
- */
- public String getName() {
- return toolName;
- }
-
- /**
- * @return a human-readable description of what the tool does.
- */
- public String getDesc() {
- return description;
- }
-
- /**
- * @return the class that implements SqoopTool.
- */
- public Class<? extends SqoopTool> getToolClass() {
- return toolClass;
+ super(name, cls, desc);
}
}
Modified: incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ToolPlugin.java
URL: http://svn.apache.org/viewvc/incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ToolPlugin.java?rev=1195852&r1=1195851&r2=1195852&view=diff
==============================================================================
--- incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ToolPlugin.java (original)
+++ incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/ToolPlugin.java Tue Nov 1 07:10:13 2011
@@ -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
@@ -17,20 +15,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package com.cloudera.sqoop.tool;
-import java.util.List;
-
/**
- * Abstract base class that defines the ToolPlugin API; additional SqoopTool
- * implementations may be registered with the system via ToolPlugin classes.
+ * @deprecated Moving to use org.apache.sqoop namespace.
*/
-public abstract class ToolPlugin {
- /**
- * Describes the tools made available by this plugin.
- * @return a list of ToolDesc objects containing the tool name, class,
- * and description.
- */
- public abstract List<ToolDesc> getTools();
+public abstract class ToolPlugin
+ extends org.apache.sqoop.tool.ToolPlugin {
}
Modified: incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/VersionTool.java
URL: http://svn.apache.org/viewvc/incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/VersionTool.java?rev=1195852&r1=1195851&r2=1195852&view=diff
==============================================================================
--- incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/VersionTool.java (original)
+++ incubator/sqoop/trunk/src/java/com/cloudera/sqoop/tool/VersionTool.java Tue Nov 1 07:10:13 2011
@@ -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
@@ -17,32 +15,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package com.cloudera.sqoop.tool;
-import com.cloudera.sqoop.SqoopOptions;
-import com.cloudera.sqoop.cli.ToolOptions;
-
/**
- * Tool that prints Sqoop's version.
+ * @deprecated Moving to use org.apache.sqoop namespace.
*/
-public class VersionTool extends BaseSqoopTool {
-
- public VersionTool() {
- super("version");
- }
-
- @Override
- /** {@inheritDoc} */
- public int run(SqoopOptions options) {
- // FIXME with maven buildnumber plugin
- System.out.print("FIXME ");
- return 0;
- }
-
- @Override
- public void printHelp(ToolOptions opts) {
- System.out.println("usage: sqoop " + getToolName());
- }
+public class VersionTool
+ extends org.apache.sqoop.tool.VersionTool {
}
-
|