diff --git a/.gitignore b/.gitignore index 51d1309c..235c27b9 100644 --- a/.gitignore +++ b/.gitignore @@ -19,4 +19,5 @@ .launches .settings /tags +/build.properties diff --git a/bin/.gitignore b/bin/.gitignore new file mode 100644 index 00000000..01f0b2f2 --- /dev/null +++ b/bin/.gitignore @@ -0,0 +1,16 @@ +# Licensed to Cloudera, Inc. under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# Cloudera, Inc. 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. +/sqoop-* + diff --git a/bin/sqoop b/bin/sqoop index 3625e88a..7bdf5b06 100755 --- a/bin/sqoop +++ b/bin/sqoop @@ -56,6 +56,6 @@ SQOOP_JAR=`ls -1 ${SQOOP_JAR_DIR}/sqoop-*.jar | head -n 1` # Invoke Hadoop HADOOP_CLASSPATH="${SQOOP_CLASSPATH}:${HADOOP_CLASSPATH}" \ - HADOOP_OPTS="-Dsqoop.shim.jar.dir=${SQOOP_SHIM_DIR}" \ + HADOOP_OPTS="-Dsqoop.shim.jar.dir=${SQOOP_SHIM_DIR} ${HADOOP_OPTS}" \ ${HADOOP_HOME}/bin/hadoop jar ${SQOOP_JAR} org.apache.hadoop.sqoop.Sqoop "$@" diff --git a/build.xml b/build.xml index 30c01e10..945b2892 100644 --- a/build.xml +++ b/build.xml @@ -21,6 +21,11 @@ xmlns:artifact="urn:maven-artifact-ant" xmlns:ivy="antlib:org.apache.ivy.ant"> + + + + @@ -30,13 +35,16 @@ - - - + + + + - + + + @@ -211,7 +219,21 @@ - + + + + + + + + + @@ -231,7 +253,6 @@ - @@ -268,19 +289,36 @@ + + + + + + + + + + + + + + + + + diff --git a/ivy.xml b/ivy.xml index 8c978dc2..0c657f71 100644 --- a/ivy.xml +++ b/ivy.xml @@ -80,6 +80,8 @@ rev="${hadoop-core.cloudera.version}" conf="clouderatest->default"/> + generatedJarFiles; + private String [] childPrgmArgs; - public Sqoop() { - this((Configuration) null); + /** + * Creates a new instance of Sqoop set to run the supplied SqoopTool + * with the default configuration. + * @param tool the SqoopTool to run in the main body of Sqoop. + */ + public Sqoop(SqoopTool tool) { + this(tool, (Configuration) null); } - public Sqoop(Configuration conf) { - this(conf, new SqoopOptions()); + /** + * Creates a new instance of Sqoop set to run the supplied SqoopTool + * with the provided configuration. + * @param tool the SqoopTool to run in the main body of Sqoop. + * @param conf the Configuration to use (e.g., from ToolRunner). + */ + public Sqoop(SqoopTool tool, Configuration conf) { + this(tool, conf, new SqoopOptions()); } - public Sqoop(Configuration conf, SqoopOptions opts) { - generatedJarFiles = new ArrayList(); + /** + * Creates a new instance of Sqoop set to run the supplied SqoopTool + * with the provided configuration and SqoopOptions. + * @param tool the SqoopTool to run in the main body of Sqoop. + * @param conf the Configuration to use (e.g., from ToolRunner). + * @param opts the SqoopOptions which control the tool's parameters. + */ + public Sqoop(SqoopTool tool, Configuration conf, SqoopOptions opts) { if (null != conf) { setConf(conf); } this.options = opts; this.options.setConf(getConf()); + + this.tool = tool; } + /** + * @return the SqoopOptions used in this Sqoop instance. + */ public SqoopOptions getOptions() { - return options; + return this.options; } /** - * @return a list of jar files generated as part of this im/export process + * @return the SqoopTool used in this Sqoop instance. */ - public List getGeneratedJarFiles() { - ArrayList out = new ArrayList(generatedJarFiles); - return out; - } - - /** - * Generate the .class and .jar files - * @return the filename of the emitted jar file. - * @throws IOException - */ - private String generateORM(String tableName) throws IOException { - String existingJar = options.getExistingJarName(); - if (existingJar != null) { - // The user has pre-specified a jar and class to use. Don't generate. - LOG.info("Using existing jar: " + existingJar); - return existingJar; - } - - LOG.info("Beginning code generation"); - CompilationManager compileMgr = new CompilationManager(options); - ClassWriter classWriter = new ClassWriter(options, manager, tableName, compileMgr); - classWriter.generate(); - compileMgr.compile(); - compileMgr.jar(); - String jarFile = compileMgr.getJarFilename(); - this.generatedJarFiles.add(jarFile); - return jarFile; - } - - private void importTable(String tableName) throws IOException, ImportException { - String jarFile = null; - - // Generate the ORM code for the tables. - jarFile = generateORM(tableName); - - if (options.getAction() == SqoopOptions.ControlAction.FullImport) { - // check if data import is to be performed - if (!options.doCreateHiveTableOnly()) { - // Proceed onward to do the import. - ImportJobContext context = new ImportJobContext(tableName, jarFile, options); - manager.importTable(context); - } - } - - // If the user wants this table to be in Hive, perform that post-load. - // If the user is in gen-only mode, this code will generate a Hive DDL - // statement and write it to a file, but will not actually perform the - // import. - if (options.doHiveImport()) { - hiveImport.importTable(tableName, options.getHiveTableName()); - } - } - - private void exportTable(String tableName) throws ExportException, IOException { - String jarFile = null; - - // Generate the ORM code for the tables. - jarFile = generateORM(tableName); - - ExportJobContext context = new ExportJobContext(tableName, jarFile, options); - manager.exportTable(context); + public SqoopTool getTool() { + return this.tool; } + @Override /** * Actual main entry-point for the program */ @@ -164,139 +132,90 @@ public int run(String [] args) { } try { - options.parse(args); - options.validate(); + options = tool.parseArguments(args, null, options, false); + tool.appendArgs(this.childPrgmArgs); + tool.validateOptions(options); + } catch (ParseException pe) { + // Couldn't parse arguments. Just print a usage message and exit. + new HelpTool().run(new SqoopOptions(getConf())); + return 1; } catch (SqoopOptions.InvalidOptionsException e) { + // Error validating arguments. Print an error message and exit. System.err.println(e.getMessage()); return 1; // Exit on exception here. } - // Make sure shim jar is classloaded early. - ShimLoader.getHadoopShim(getConf()); - - // Get the connection to the database. - try { - manager = new ConnFactory(getConf()).getManager(options); - } catch (Exception e) { - LOG.error("Got error creating database manager: " + e.toString()); - if (System.getProperty(SQOOP_RETHROW_PROPERTY) != null) { - throw new RuntimeException(e); - } else { - return 1; - } - } - - try { - if (options.doHiveImport()) { - hiveImport = new HiveImport(options, manager, getConf()); - } - - SqoopOptions.ControlAction action = options.getAction(); - if (action == SqoopOptions.ControlAction.ListTables) { - 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); - } - } - } else if (action == SqoopOptions.ControlAction.ListDatabases) { - 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); - } - } - } else if (action == SqoopOptions.ControlAction.DebugExec) { - // just run a SQL statement for debugging purposes. - manager.execAndPrint(options.getDebugSqlCmd()); - return 0; - } else if (action == SqoopOptions.ControlAction.Export) { - // Export a table. - try { - exportTable(options.getTableName()); - } catch (IOException ioe) { - LOG.error("Encountered IOException running export job: " + ioe.toString()); - if (System.getProperty(SQOOP_RETHROW_PROPERTY) != null) { - throw new RuntimeException(ioe); - } else { - return 1; - } - } catch (ExportException ee) { - LOG.error("Error during export: " + ee.toString()); - if (System.getProperty(SQOOP_RETHROW_PROPERTY) != null) { - throw new RuntimeException(ee); - } else { - return 1; - } - } - } else { - // This is either FullImport or GenerateOnly. - - try { - if (options.isAllTables()) { - 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 tableName : tables) { - importTable(tableName); - } - } - } else { - // just import a single table the user specified. - importTable(options.getTableName()); - } - } catch (IOException ioe) { - LOG.error("Encountered IOException running import job: " + ioe.toString()); - if (System.getProperty(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_RETHROW_PROPERTY) != null) { - throw new RuntimeException(ie); - } else { - return 1; - } - } - } - } finally { - try { - manager.close(); - } catch (SQLException sqlE) { - LOG.warn("Error while closing connection: " + sqlE); - } - } - - return 0; + return tool.run(options); } - public static void main(String [] args) { - int ret; + /** + * SqoopTools sometimes pass arguments to a child program (e.g., mysqldump). + * Users can specify additional args to these programs by preceeding the + * additional arguments with a standalone '--'; but + * ToolRunner/GenericOptionsParser will cull out this argument. We remove + * the child-program arguments in advance, and store them to be readded + * later. + * @param argv the argv in to the SqoopTool + * @return the argv with a "--" and any subsequent arguments removed. + */ + private String [] stashChildPrgmArgs(String [] argv) { + for (int i = 0; i < argv.length; i++) { + if ("--".equals(argv[i])) { + this.childPrgmArgs = Arrays.copyOfRange(argv, i, argv.length); + return Arrays.copyOfRange(argv, 0, i); + } + } + + // Didn't find child-program arguments. + return argv; + } + + /** + * Given a Sqoop object and a set of arguments to deliver to + * its embedded SqoopTool, run the tool, wrapping the call to + * ToolRunner. + * This entry-point is preferred to ToolRunner.run() because + * it has a chance to stash child program arguments before + * GenericOptionsParser would remove them. + */ + public static int runSqoop(Sqoop sqoop, String [] args) { try { - Sqoop importer = new Sqoop(); - ret = ToolRunner.run(importer, args); + String [] toolArgs = sqoop.stashChildPrgmArgs(args); + return ToolRunner.run(sqoop, toolArgs); } catch (Exception e) { LOG.error("Got exception running Sqoop: " + e.toString()); e.printStackTrace(); - ret = 1; if (System.getProperty(SQOOP_RETHROW_PROPERTY) != null) { throw new RuntimeException(e); } + return 1; } + } + + /** + * Entry-point that parses the correct SqoopTool to use from the args, + * but does not call System.exit() as main() will. + */ + public static int runTool(String [] args) { + String toolName = args[0]; + SqoopTool tool = SqoopTool.getTool(toolName); + if (null == tool) { + System.err.println("No such sqoop tool: " + toolName + ". See 'sqoop help'."); + return 1; + } + + Sqoop sqoop = new Sqoop(tool); + return runSqoop(sqoop, Arrays.copyOfRange(args, 1, args.length)); + } + + public static void main(String [] args) { + if (args.length == 0) { + System.err.println("Try 'sqoop help' for usage."); + System.exit(1); + } + + int ret = runTool(args); System.exit(ret); } } diff --git a/src/java/org/apache/hadoop/sqoop/SqoopOptions.java b/src/java/org/apache/hadoop/sqoop/SqoopOptions.java index cba05470..8bf25c00 100644 --- a/src/java/org/apache/hadoop/sqoop/SqoopOptions.java +++ b/src/java/org/apache/hadoop/sqoop/SqoopOptions.java @@ -64,16 +64,6 @@ public String toString() { } } - // control-flow selector based on command-line switches. - public enum ControlAction { - ListDatabases, // list available databases and exit. - ListTables, // list available tables and exit. - GenerateOnly, // generate ORM code but do not import. - FullImport, // generate code (as needed) and import. - DebugExec, // just execute a single sql command and print its results. - Export // export a table from HDFS to a database. - } - // selects in-HDFS destination file format public enum FileLayout { TextFile, @@ -81,18 +71,19 @@ public enum FileLayout { } - // TODO(aaron): Adding something here? Add a getter, a cmdline switch, and a properties file - // entry in loadFromProperties(). Add a default value in initDefaults() if you need one. - // Make sure you add the stub to the testdata/sqoop.properties.template file. + // TODO(aaron): Adding something here? Add a setter and a getter. + // Add a default value in initDefaults() if you need one. + // If you want to load from a properties file, add an entry in the + // loadFromProperties() method. + // Then add command-line arguments in the appropriate tools. The + // names of all command-line args are stored as constants in BaseSqoopTool. private String connectString; private String tableName; private String [] columns; - private boolean allTables; private String username; private String password; private String codeOutputDir; private String jarOutputDir; - private ControlAction action; private String hadoopHome; private String splitByCol; private String whereClause; @@ -104,7 +95,6 @@ public enum FileLayout { private String tmpDir; // where temp data goes; usually /tmp private String hiveHome; private boolean hiveImport; - private boolean createHiveTableOnly; private boolean overwriteHiveTable; private String hiveTableName; private String packageName; // package to prepend to auto-named classes. @@ -212,7 +202,6 @@ private void loadFromProperties() { this.direct = getBooleanProperty(props, "direct.import", this.direct); this.hiveImport = getBooleanProperty(props, "hive.import", this.hiveImport); - this.createHiveTableOnly = getBooleanProperty(props, "hive.create.table.only", this.createHiveTableOnly); this.overwriteHiveTable = getBooleanProperty(props, "hive.overwrite.table", this.overwriteHiveTable); this.useCompression = getBooleanProperty(props, "compression", this.useCompression); this.directSplitSize = getLongProperty(props, "direct.split.size", @@ -241,7 +230,6 @@ public String getTempDir() { private void initDefaults(Configuration baseConfiguration) { // first, set the true defaults if nothing else happens. // default action is to run the full pipeline. - this.action = ControlAction.FullImport; this.hadoopHome = System.getenv("HADOOP_HOME"); // Set this with $HIVE_HOME, but -Dhive.home can override. @@ -291,102 +279,6 @@ private void initDefaults(Configuration baseConfiguration) { loadFromProperties(); } - /** - * Allow the user to enter his password on the console without printing characters. - * @return the password as a string - */ - private String securePasswordEntry() { - return new String(System.console().readPassword("Enter password: ")); - } - - /** - * Print usage strings for the program's arguments. - */ - public static void printUsage() { - System.out.println("Usage: hadoop sqoop.jar org.apache.hadoop.sqoop.Sqoop (options)"); - System.out.println(""); - System.out.println("Database connection options:"); - System.out.println("--connect (jdbc-uri) Specify JDBC connect string"); - System.out.println("--driver (class-name) Manually specify JDBC driver class to use"); - System.out.println("--username (username) Set authentication username"); - System.out.println("--password (password) Set authentication password"); - System.out.println("-P Read password from console"); - System.out.println("--direct Use direct import fast path (mysql only)"); - System.out.println(""); - System.out.println("Import control options:"); - System.out.println("--table (tablename) Table to read"); - System.out.println("--columns (col,col,col...) Columns to export from table"); - System.out.println("--split-by (column-name) Column of the table used to split work units"); - System.out.println("--where (where clause) Where clause to use during export"); - System.out.println("--hadoop-home (dir) Override $HADOOP_HOME"); - System.out.println("--hive-home (dir) Override $HIVE_HOME"); - System.out.println("--warehouse-dir (dir) HDFS path for table destination"); - System.out.println("--as-sequencefile Imports data to SequenceFiles"); - System.out.println("--as-textfile Imports data as plain text (default)"); - System.out.println("--all-tables Import all tables in database"); - System.out.println(" (Ignores --table, --columns and --split-by)"); - System.out.println("--hive-import If set, then import the table into Hive."); - System.out.println(" (Uses Hive's default delimiters if none are set.)"); - System.out.println("--hive-table (tablename) Sets the table name to use when importing"); - System.out.println(" to hive."); - System.out.println("-m, --num-mappers (n) Use 'n' map tasks to import in parallel"); - System.out.println("-z, --compress Enable compression"); - System.out.println("--direct-split-size (n) Split the input stream every 'n' bytes"); - System.out.println(" when importing in direct mode."); - System.out.println("--inline-lob-limit (n) Set the maximum size for an inline LOB"); - System.out.println(""); - System.out.println("Export options:"); - System.out.println("--export-dir (dir) Export from an HDFS path into a table"); - System.out.println(" (set with --table)"); - System.out.println(""); - System.out.println("Output line formatting options:"); - System.out.println("--fields-terminated-by (char) Sets the field separator character"); - System.out.println("--lines-terminated-by (char) Sets the end-of-line character"); - System.out.println("--optionally-enclosed-by (char) Sets a field enclosing character"); - System.out.println("--enclosed-by (char) Sets a required field enclosing char"); - System.out.println("--escaped-by (char) Sets the escape character"); - System.out.println("--mysql-delimiters Uses MySQL's default delimiter set"); - System.out.println(" fields: , lines: \\n escaped-by: \\ optionally-enclosed-by: '"); - System.out.println(""); - System.out.println("Input parsing options:"); - System.out.println("--input-fields-terminated-by (char) Sets the input field separator"); - System.out.println("--input-lines-terminated-by (char) Sets the input end-of-line char"); - System.out.println("--input-optionally-enclosed-by (char) Sets a field enclosing character"); - System.out.println("--input-enclosed-by (char) Sets a required field encloser"); - System.out.println("--input-escaped-by (char) Sets the input escape character"); - System.out.println(""); - System.out.println("Code generation options:"); - System.out.println("--outdir (dir) Output directory for generated code"); - System.out.println("--bindir (dir) Output directory for compiled objects"); - System.out.println("--generate-only Stop after code generation; do not import"); - System.out.println("--package-name (name) Put auto-generated classes in this package"); - System.out.println("--class-name (name) When generating one class, use this name."); - System.out.println(" This overrides --package-name."); - System.out.println(""); - System.out.println("Library loading options:"); - System.out.println("--jar-file (file) Disable code generation; use specified jar"); - System.out.println("--class-name (name) The class within the jar that represents"); - System.out.println(" the table to import/export"); - System.out.println(""); - System.out.println("Additional commands:"); - System.out.println("--list-tables List tables in database and exit"); - System.out.println("--list-databases List all databases available and exit"); - System.out.println("--debug-sql (statement) Execute 'statement' in SQL and exit"); - System.out.println("--verbose Print more information while working"); - System.out.println(""); - System.out.println("Database-specific options:"); - System.out.println("Arguments may be passed to the database manager after a lone '-':"); - System.out.println(" MySQL direct mode: arguments passed directly to mysqldump"); - System.out.println(""); - System.out.println("Generic Hadoop command-line options:"); - ToolRunner.printGenericCommandUsage(System.out); - System.out.println(""); - System.out.println("At minimum, you must specify --connect " - + "and either --table or --all-tables."); - System.out.println("Alternatively, you can specify --generate-only or one of the additional"); - System.out.println("commands."); - } - /** * Given a string containing a single character or an escape sequence representing * a char, return that char itself. @@ -397,12 +289,16 @@ public static void printUsage() { * * Strings like "\0ooo" return the character specified by the octal sequence 'ooo' * Strings like "\0xhhh" or "\0Xhhh" return the character specified by the hex sequence 'hhh' + * + * If the input string contains leading or trailing spaces, these are ignored. */ - static char toChar(String charish) throws InvalidOptionsException { - if (null == charish) { + public static char toChar(String charish) throws InvalidOptionsException { + if (null == charish || charish.length() == 0) { throw new InvalidOptionsException("Character argument expected." + "\nTry --help for usage instructions."); - } else if (charish.startsWith("\\0x") || charish.startsWith("\\0X")) { + } + + if (charish.startsWith("\\0x") || charish.startsWith("\\0X")) { if (charish.length() == 3) { throw new InvalidOptionsException("Base-16 value expected for character argument." + "\nTry --help for usage instructions."); @@ -452,9 +348,6 @@ static char toChar(String charish) throws InvalidOptionsException { + "\nTry --help for usage instructions."); } } - } else if (charish.length() == 0) { - throw new InvalidOptionsException("Character argument expected." - + "\nTry --help for usage instructions."); } else { // it's a normal character. if (charish.length() > 1) { @@ -466,252 +359,6 @@ static char toChar(String charish) throws InvalidOptionsException { } } - /** - * Read args from the command-line into member fields. - * @throws Exception if there's a problem parsing arguments. - */ - public void parse(String [] args) throws InvalidOptionsException { - - if (LOG.isDebugEnabled()) { - LOG.debug("Parsing sqoop arguments:"); - for (String arg : args) { - LOG.debug(" " + arg); - } - } - - int i = 0; - try { - for (i = 0; i < args.length; i++) { - if (args[i].equals("--connect")) { - this.connectString = args[++i]; - } else if (args[i].equals("--driver")) { - this.driverClassName = args[++i]; - } else if (args[i].equals("--table")) { - this.tableName = args[++i]; - } else if (args[i].equals("--columns")) { - String columnString = args[++i]; - this.columns = columnString.split(","); - } else if (args[i].equals("--split-by")) { - this.splitByCol = args[++i]; - } else if (args[i].equals("--where")) { - this.whereClause = args[++i]; - } else if (args[i].equals("--list-tables")) { - this.action = ControlAction.ListTables; - } else if (args[i].equals("--all-tables")) { - this.allTables = true; - } else if (args[i].equals("--export-dir")) { - this.exportDir = args[++i]; - this.action = ControlAction.Export; - } else if (args[i].equals("--local")) { - // TODO(aaron): Remove this after suitable deprecation time period. - LOG.warn("--local is deprecated; use --direct instead."); - this.direct = true; - } else if (args[i].equals("--direct")) { - this.direct = true; - } else if (args[i].equals("--username")) { - this.username = args[++i]; - if (null == this.password) { - // Set password to empty if the username is set first, - // to ensure that they're either both null or neither. - this.password = ""; - } - } else if (args[i].equals("--password")) { - LOG.warn("Setting your password on the command-line is insecure. " - + "Consider using -P instead."); - this.password = args[++i]; - } else if (args[i].equals("-P")) { - this.password = securePasswordEntry(); - } else if (args[i].equals("--hadoop-home")) { - this.hadoopHome = args[++i]; - } else if (args[i].equals("--hive-home")) { - this.hiveHome = args[++i]; - } else if (args[i].equals("--hive-import")) { - this.hiveImport = true; - } else if (args[i].equals("--hive-create-only")) { - this.createHiveTableOnly = true; - } else if (args[i].equals("--hive-overwrite")) { - this.overwriteHiveTable = true; - } else if (args[i].equals("--hive-table")) { - this.hiveTableName = args[++i]; - } else if (args[i].equals("--num-mappers") || args[i].equals("-m")) { - String numMappersStr = args[++i]; - this.numMappers = Integer.valueOf(numMappersStr); - } else if (args[i].equals("--fields-terminated-by")) { - this.outputFieldDelim = SqoopOptions.toChar(args[++i]); - this.areDelimsManuallySet = true; - } else if (args[i].equals("--lines-terminated-by")) { - this.outputRecordDelim = SqoopOptions.toChar(args[++i]); - this.areDelimsManuallySet = true; - } else if (args[i].equals("--optionally-enclosed-by")) { - this.outputEnclosedBy = SqoopOptions.toChar(args[++i]); - this.outputMustBeEnclosed = false; - this.areDelimsManuallySet = true; - } else if (args[i].equals("--enclosed-by")) { - this.outputEnclosedBy = SqoopOptions.toChar(args[++i]); - this.outputMustBeEnclosed = true; - this.areDelimsManuallySet = true; - } else if (args[i].equals("--escaped-by")) { - this.outputEscapedBy = SqoopOptions.toChar(args[++i]); - this.areDelimsManuallySet = true; - } else if (args[i].equals("--mysql-delimiters")) { - this.outputFieldDelim = ','; - this.outputRecordDelim = '\n'; - this.outputEnclosedBy = '\''; - this.outputEscapedBy = '\\'; - this.outputMustBeEnclosed = false; - this.areDelimsManuallySet = true; - } else if (args[i].equals("--input-fields-terminated-by")) { - this.inputFieldDelim = SqoopOptions.toChar(args[++i]); - } else if (args[i].equals("--input-lines-terminated-by")) { - this.inputRecordDelim = SqoopOptions.toChar(args[++i]); - } else if (args[i].equals("--input-optionally-enclosed-by")) { - this.inputEnclosedBy = SqoopOptions.toChar(args[++i]); - this.inputMustBeEnclosed = false; - } else if (args[i].equals("--input-enclosed-by")) { - this.inputEnclosedBy = SqoopOptions.toChar(args[++i]); - this.inputMustBeEnclosed = true; - } else if (args[i].equals("--input-escaped-by")) { - this.inputEscapedBy = SqoopOptions.toChar(args[++i]); - } else if (args[i].equals("--outdir")) { - this.codeOutputDir = args[++i]; - } else if (args[i].equals("--as-sequencefile")) { - this.layout = FileLayout.SequenceFile; - } else if (args[i].equals("--as-textfile")) { - this.layout = FileLayout.TextFile; - } else if (args[i].equals("--bindir")) { - this.jarOutputDir = args[++i]; - } else if (args[i].equals("--warehouse-dir")) { - this.warehouseDir = args[++i]; - } else if (args[i].equals("--package-name")) { - this.packageName = args[++i]; - } else if (args[i].equals("--class-name")) { - this.className = args[++i]; - } else if (args[i].equals("-z") || args[i].equals("--compress")) { - this.useCompression = true; - } else if (args[i].equals("--direct-split-size")) { - this.directSplitSize = Long.parseLong(args[++i]); - } else if (args[i].equals("--inline-lob-limit")) { - this.maxInlineLobSize = Long.parseLong(args[++i]); - } else if (args[i].equals("--jar-file")) { - this.existingJarFile = args[++i]; - } else if (args[i].equals("--list-databases")) { - this.action = ControlAction.ListDatabases; - } else if (args[i].equals("--generate-only")) { - this.action = ControlAction.GenerateOnly; - } else if (args[i].equals("--debug-sql")) { - this.action = ControlAction.DebugExec; - // read the entire remainder of the commandline into the debug sql statement. - if (null == this.debugSqlCmd) { - this.debugSqlCmd = ""; - } - for (i++; i < args.length; i++) { - this.debugSqlCmd = this.debugSqlCmd + args[i] + " "; - } - } else if (args[i].equals("--verbose")) { - // Immediately switch into DEBUG logging. - Category sqoopLogger = - Logger.getLogger(SqoopOptions.class.getName()).getParent(); - sqoopLogger.setLevel(Level.DEBUG); - - } else if (args[i].equals("--help")) { - printUsage(); - throw new InvalidOptionsException(""); - } else if (args[i].equals("-")) { - // Everything after a '--' goes into extraArgs. - ArrayList extra = new ArrayList(); - for (i++; i < args.length; i++) { - extra.add(args[i]); - } - this.extraArgs = extra.toArray(new String[0]); - } else { - throw new InvalidOptionsException("Invalid argument: " + args[i] + ".\n" - + "Try --help for usage."); - } - } - } catch (ArrayIndexOutOfBoundsException oob) { - throw new InvalidOptionsException("Error: " + args[--i] + " expected argument.\n" - + "Try --help for usage."); - } catch (NumberFormatException nfe) { - throw new InvalidOptionsException("Error: " + args[--i] + " expected numeric argument.\n" - + "Try --help for usage."); - } - } - - private static final String HELP_STR = "\nTry --help for usage instructions."; - - /** - * Validates options and ensures that any required options are - * present and that any mutually-exclusive options are not selected. - * @throws Exception if there's a problem. - */ - public void validate() throws InvalidOptionsException { - if (this.allTables && this.columns != null) { - // If we're reading all tables in a database, can't filter column names. - throw new InvalidOptionsException("--columns and --all-tables are incompatible options." - + HELP_STR); - } else if (this.allTables && this.splitByCol != null) { - // If we're reading all tables in a database, can't set pkey - throw new InvalidOptionsException("--split-by and --all-tables are incompatible options." - + HELP_STR); - } else if (this.allTables && this.className != null) { - // If we're reading all tables, can't set individual class name - throw new InvalidOptionsException("--class-name and --all-tables are incompatible options." - + HELP_STR); - } else if (this.allTables && this.hiveTableName != null) { - // If we're reading all tables, can't set hive target table name - throw new InvalidOptionsException( - "--hive-table and --all-tables are incompatible options." - + HELP_STR); - } else if (this.hiveTableName != null && !this.hiveImport) { - throw new InvalidOptionsException( - "--hive-table is invalid without --hive-import" - + HELP_STR); - } else if (this.connectString == null) { - throw new InvalidOptionsException("Error: Required argument --connect is missing." - + HELP_STR); - } else if (this.className != null && this.packageName != null) { - throw new InvalidOptionsException( - "--class-name overrides --package-name. You cannot use both." + HELP_STR); - } else if (this.action == ControlAction.FullImport && !this.allTables - && this.tableName == null) { - throw new InvalidOptionsException( - "One of --table or --all-tables is required for import." + HELP_STR); - } else if (this.action == ControlAction.Export && this.allTables) { - throw new InvalidOptionsException("You cannot export with --all-tables." + HELP_STR); - } else if (this.action == ControlAction.Export && this.tableName == null) { - throw new InvalidOptionsException("Export requires a --table argument." + HELP_STR); - } else if (this.existingJarFile != null && this.className == null) { - throw new InvalidOptionsException("Jar specified with --jar-file, but no " - + "class specified with --class-name." + HELP_STR); - } else if (this.existingJarFile != null && this.action == ControlAction.GenerateOnly) { - throw new InvalidOptionsException("Cannot generate code using existing jar." + HELP_STR); - } - - if (this.hiveImport) { - if (!areDelimsManuallySet) { - // user hasn't manually specified delimiters, and wants to import straight to Hive. - // Use Hive-style delimiters. - LOG.info("Using Hive-specific delimiters for output. You can override"); - LOG.info("delimiters with --fields-terminated-by, etc."); - this.outputFieldDelim = (char)0x1; // ^A - this.outputRecordDelim = '\n'; - this.outputEnclosedBy = '\000'; // no enclosing in Hive. - this.outputEscapedBy = '\000'; // no escaping in Hive - this.outputMustBeEnclosed = false; - } - - if (this.getOutputEscapedBy() != '\000') { - LOG.warn("Hive does not support escape characters in fields;"); - LOG.warn("parse errors in Hive may result from using --escaped-by."); - } - - if (this.getOutputEnclosedBy() != '\000') { - LOG.warn("Hive does not support quoted strings; parse errors"); - LOG.warn("in Hive may result from using --enclosed-by."); - } - } - } - /** get the temporary directory; guaranteed to end in File.separator * (e.g., '/') */ @@ -719,6 +366,10 @@ public String getTmpDir() { return tmpDir; } + public void setTmpDir(String tmp) { + this.tmpDir = tmpDir; + } + public String getConnectString() { return connectString; } @@ -739,10 +390,18 @@ public String getExportDir() { return exportDir; } + public void setExportDir(String exportDir) { + this.exportDir = exportDir; + } + public String getExistingJarName() { return existingJarFile; } + public void setExistingJarName(String jarFile) { + this.existingJarFile = jarFile; + } + public String[] getColumns() { if (null == columns) { return null; @@ -751,34 +410,70 @@ public String[] getColumns() { } } + public void setColumns(String [] cols) { + if (null == cols) { + this.columns = null; + } else { + this.columns = Arrays.copyOf(cols, cols.length); + } + } + public String getSplitByCol() { return splitByCol; } + + public void setSplitByCol(String splitBy) { + this.splitByCol = splitBy; + } public String getWhereClause() { return whereClause; } - public ControlAction getAction() { - return action; - } - - public boolean isAllTables() { - return allTables; + public void setWhereClause(String where) { + this.whereClause = where; } public String getUsername() { return username; } + public void setUsername(String user) { + this.username = user; + } + public String getPassword() { return password; } + /** + * Allow the user to enter his password on the console without printing characters. + * @return the password as a string + */ + private String securePasswordEntry() { + return new String(System.console().readPassword("Enter password: ")); + } + + /** + * Set the password in this SqoopOptions from the console without printing + * characters. + */ + public void setPasswordFromConsole() { + this.password = securePasswordEntry(); + } + + public void setPassword(String pass) { + this.password = pass; + } + public boolean isDirect() { return direct; } + public void setDirectMode(boolean isDirect) { + this.direct = isDirect; + } + /** * @return the number of map tasks to use for import */ @@ -786,6 +481,10 @@ public int getNumMappers() { return this.numMappers; } + public void setNumMappers(int numMappers) { + this.numMappers = numMappers; + } + /** * @return the user-specified absolute class name for the table */ @@ -793,6 +492,10 @@ public String getClassName() { return className; } + public void setClassName(String className) { + this.className = className; + } + /** * @return the user-specified package to prepend to table names via --package-name. */ @@ -800,20 +503,25 @@ public String getPackageName() { return packageName; } + public void setPackageName(String packageName) { + this.packageName = packageName; + } + public String getHiveHome() { return hiveHome; } + + public void setHiveHome(String hiveHome) { + this.hiveHome = hiveHome; + } /** @return true if we should import the table into Hive */ public boolean doHiveImport() { return hiveImport; } - /** - * @return the user-specified option to create tables in hive with no loading - */ - public boolean doCreateHiveTableOnly() { - return createHiveTableOnly; + public void setHiveImport(boolean hiveImport) { + this.hiveImport = hiveImport; } /** @@ -823,6 +531,10 @@ public boolean doOverwriteHiveTable() { return overwriteHiveTable; } + public void setOverwriteHiveTable(boolean overwrite) { + this.overwriteHiveTable = overwrite; + } + /** * @return location where .java files go; guaranteed to end with '/' */ @@ -834,6 +546,10 @@ public String getCodeOutputDir() { } } + public void setCodeOutputDir(String outputDir) { + this.codeOutputDir = outputDir; + } + /** * @return location where .jar and .class files go; guaranteed to end with '/' */ @@ -845,6 +561,10 @@ public String getJarOutputDir() { } } + public void setJarOutputDir(String outDir) { + this.jarOutputDir = outDir; + } + /** * Return the value of $HADOOP_HOME * @return $HADOOP_HOME, or null if it's not set. @@ -853,6 +573,10 @@ public String getHadoopHome() { return hadoopHome; } + public void setHadoopHome(String hadoopHome) { + this.hadoopHome = hadoopHome; + } + /** * @return a sql command to execute and exit with. */ @@ -860,6 +584,10 @@ public String getDebugSqlCmd() { return debugSqlCmd; } + public void setDebugSqlCmd(String sqlStatement) { + this.debugSqlCmd = sqlStatement; + } + /** * @return The JDBC driver class name specified with --driver */ @@ -867,6 +595,10 @@ public String getDriverClassName() { return driverClassName; } + public void setDriverClassName(String driverClass) { + this.driverClassName = driverClass; + } + /** * @return the base destination path for table uploads. */ @@ -874,6 +606,10 @@ public String getWarehouseDir() { return warehouseDir; } + public void setWarehouseDir(String warehouse) { + this.warehouseDir = warehouse; + } + /** * @return the destination file format */ @@ -881,12 +617,8 @@ public FileLayout getFileLayout() { return this.layout; } - public void setUsername(String name) { - this.username = name; - } - - public void setPassword(String pass) { - this.password = pass; + public void setFileLayout(FileLayout layout) { + this.layout = layout; } /** @@ -901,6 +633,10 @@ public char getInputFieldDelim() { } } + public void setInputFieldsTerminatedBy(char c) { + this.inputFieldDelim = c; + } + /** * @return the record delimiter to use when parsing lines. Defaults to the record delim * to use when printing lines. @@ -913,6 +649,10 @@ public char getInputRecordDelim() { } } + public void setInputLinesTerminatedBy(char c) { + this.inputRecordDelim = c; + } + /** * @return the character that may enclose fields when parsing lines. Defaults to the * enclosing-char to use when printing lines. @@ -925,6 +665,10 @@ public char getInputEnclosedBy() { } } + public void setInputEnclosedBy(char c) { + this.inputEnclosedBy = c; + } + /** * @return the escape character to use when parsing lines. Defaults to the escape * character used when printing lines. @@ -937,6 +681,10 @@ public char getInputEscapedBy() { } } + public void setInputEscapedBy(char c) { + this.inputEscapedBy = c; + } + /** * @return true if fields must be enclosed by the --enclosed-by character when parsing. * Defaults to false. Set true when --input-enclosed-by is used. @@ -949,6 +697,10 @@ public boolean isInputEncloseRequired() { } } + public void setInputEncloseRequired(boolean required) { + this.inputMustBeEnclosed = required; + } + /** * @return the character to print between fields when importing them to text. */ @@ -956,6 +708,10 @@ public char getOutputFieldDelim() { return this.outputFieldDelim; } + public void setFieldsTerminatedBy(char c) { + this.outputFieldDelim = c; + } + /** * @return the character to print between records when importing them to text. @@ -964,6 +720,10 @@ public char getOutputRecordDelim() { return this.outputRecordDelim; } + public void setLinesTerminatedBy(char c) { + this.outputRecordDelim = c; + } + /** * @return a character which may enclose the contents of fields when imported to text. */ @@ -971,6 +731,10 @@ public char getOutputEnclosedBy() { return this.outputEnclosedBy; } + public void setEnclosedBy(char c) { + this.outputEnclosedBy = c; + } + /** * @return a character which signifies an escape sequence when importing to text. */ @@ -978,6 +742,10 @@ public char getOutputEscapedBy() { return this.outputEscapedBy; } + public void setEscapedBy(char c) { + this.outputEscapedBy = c; + } + /** * @return true if fields imported to text must be enclosed by the EnclosedBy char. * default is false; set to true if --enclosed-by is used instead of --optionally-enclosed-by. @@ -986,6 +754,10 @@ public boolean isOutputEncloseRequired() { return this.outputMustBeEnclosed; } + public void setOutputEncloseRequired(boolean required) { + this.outputMustBeEnclosed = required; + } + /** * @return true if the user wants imported results to be compressed. */ @@ -993,10 +765,14 @@ public boolean shouldUseCompression() { return this.useCompression; } + public void setUseCompression(boolean useCompression) { + this.useCompression = useCompression; + } + /** * @return the name of the destination table when importing to Hive */ - public String getHiveTableName( ) { + public String getHiveTableName() { if (null != this.hiveTableName) { return this.hiveTableName; } else { @@ -1004,6 +780,10 @@ public String getHiveTableName( ) { } } + public void setHiveTableName(String tableName) { + this.hiveTableName = tableName; + } + /** * @return the file size to split by when using --direct mode. */ @@ -1011,6 +791,10 @@ public long getDirectSplitSize() { return this.directSplitSize; } + public void setDirectSplitSize(long splitSize) { + this.directSplitSize = splitSize; + } + /** * @return the max size of a LOB before we spill to a separate file. */ @@ -1018,6 +802,24 @@ public long getInlineLobLimit() { return this.maxInlineLobSize; } + public void setInlineLobLimit(long limit) { + this.maxInlineLobSize = limit; + } + + /** + * @return true if the delimiters have been explicitly set by the user. + */ + public boolean explicitDelims() { + return areDelimsManuallySet; + } + + /** + * Flag the delimiter settings as explicit user settings, or implicit. + */ + public void setExplicitDelims(boolean explicit) { + this.areDelimsManuallySet = explicit; + } + public Configuration getConf() { return conf; } @@ -1040,4 +842,16 @@ public void setConf(Configuration config) { } return out; } + + public void setExtraArgs(String [] args) { + if (null == args) { + this.extraArgs = null; + return; + } + + this.extraArgs = new String[args.length]; + for (int i = 0; i < args.length; i++) { + this.extraArgs[i] = args[i]; + } + } } diff --git a/src/java/org/apache/hadoop/sqoop/cli/RelatedOptions.java b/src/java/org/apache/hadoop/sqoop/cli/RelatedOptions.java new file mode 100644 index 00000000..4a4ef10e --- /dev/null +++ b/src/java/org/apache/hadoop/sqoop/cli/RelatedOptions.java @@ -0,0 +1,49 @@ +/** + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. 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.hadoop.sqoop.cli; + +import org.apache.commons.cli.Options; + +/** + * Class that extends a set of options to provide a title for a + * related set of options. + */ +public class RelatedOptions extends Options { + + private String relatedTitle; + + public RelatedOptions() { + this(""); + } + + public RelatedOptions(final String title) { + super(); + this.relatedTitle = title; + } + + public String getTitle() { + return relatedTitle; + } + + @Override + public String toString() { + return relatedTitle + "\n" + super.toString(); + } +} + diff --git a/src/java/org/apache/hadoop/sqoop/cli/SqoopParser.java b/src/java/org/apache/hadoop/sqoop/cli/SqoopParser.java new file mode 100644 index 00000000..1810476f --- /dev/null +++ b/src/java/org/apache/hadoop/sqoop/cli/SqoopParser.java @@ -0,0 +1,123 @@ +/** + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. 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.hadoop.sqoop.cli; + +import java.util.Arrays; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.ListIterator; +import java.util.Properties; + +import org.apache.commons.cli.GnuParser; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.MissingArgumentException; +import org.apache.commons.cli.ParseException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.util.StringUtils; + +/** + * Options parser that follows Sqoop's specific options-parsing needs. + *
    + *
  • Option values may contain the '"' character as the first + * or last character in the value.
  • + *
  • The argument '--' must be preserved in the returned (unparsed) + * argument list.
  • + *
+ */ +public class SqoopParser extends GnuParser { + + public static final Log LOG = LogFactory.getLog(SqoopParser.class.getName()); + + // We need a handle to Option.addValueForProcessing(String). + // Since Hadoop will load this in a different classloader than + // this Sqoop class, we cannot see their package-specific methods. + // So we just call it by reflection. As long as we're at it, this + // allows us to also put SqoopParser in its own package. + static java.lang.reflect.Method addValForProcessing; + + static { + try { + addValForProcessing = Option.class.getDeclaredMethod( + "addValueForProcessing", String.class); + addValForProcessing.setAccessible(true); + } catch (NoSuchMethodException nsme) { + LOG.error("Could not load required method of Parser: " + + StringUtils.stringifyException(nsme)); + addValForProcessing = null; + } + } + + @Override + /** + * Processes arguments to options but only strips matched quotes. + */ + public void processArgs(Option opt, ListIterator iter) + throws ParseException { + // Loop until an option is found. + while (iter.hasNext()) { + String str = (String) iter.next(); + + if (getOptions().hasOption(str) && str.startsWith("-")) { + // found an Option, not an argument. + iter.previous(); + break; + } + + // Otherwise, this is a value. + try { + // Note that we only strip matched quotes here. + addValForProcessing.invoke(opt, stripMatchedQuotes(str)); + } catch (IllegalAccessException iae) { + throw new RuntimeException(iae); + } catch (java.lang.reflect.InvocationTargetException ite) { + // Any runtime exception thrown within addValForProcessing() + // will be wrapped in an InvocationTargetException. + iter.previous(); + break; + } catch (RuntimeException re) { + iter.previous(); + break; + } + } + + if (opt.getValues() == null && !opt.hasOptionalArg()) { + throw new MissingArgumentException(opt); + } + } + + /** + * Util.stripLeadingAndTrailingQuotes() will strip a '"' + * character from either or both sides of a string. We only + * strip the matched pair. + */ + private String stripMatchedQuotes(String in) { + if (null == in || "\"".equals(in)) { + return in; // single quote stays as-is. + } else if(in.startsWith("\"") && in.endsWith("\"")) { + // Strip this pair of matched quotes. + return in.substring(1, in.length() - 1); + } else { + // return as-is. + return in; + } + } +} + diff --git a/src/java/org/apache/hadoop/sqoop/cli/ToolOptions.java b/src/java/org/apache/hadoop/sqoop/cli/ToolOptions.java new file mode 100644 index 00000000..ee558a65 --- /dev/null +++ b/src/java/org/apache/hadoop/sqoop/cli/ToolOptions.java @@ -0,0 +1,151 @@ +/** + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. 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.hadoop.sqoop.cli; + +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; + +/** + * Class that holds several sets of related options, providing a container + * for all the options associated with a single tool. + * The order in which sets of related options are added to this tool is + * preserved in printing and iteration. + */ +public class ToolOptions implements Iterable { + + private List optGroups; + + public ToolOptions() { + this.optGroups = new ArrayList(); + } + + /** + * Add a block of related options to the options for this tool. + * @param opts the set of RelatedOptions to add. + */ + public void addOptions(RelatedOptions opts) { + optGroups.add(opts); + } + + /** + * Add a block of related options to the options for this tool, + * if a block has not already been added with the same title. + * @param opts the set of RelatedOptions to add. + */ + public void addUniqueOptions(RelatedOptions opts) { + if (!containsGroup(opts.getTitle())) { + optGroups.add(opts); + } + } + + /** + * Reports whether this collection of RelatedOptions contains + * a RelatedOptions with the specified title. + * @param title the group title to search for + * @return true if a RelatedOptions with this group title is + * in the collection. + */ + public boolean containsGroup(String title) { + for (RelatedOptions related : this) { + if (related.getTitle().equals(title)) { + return true; + } + } + + return false; + } + + /** + * Provide an iterator over all sets of RelatedOptions. + * @return an iterator returning each RelatedOptions element. + */ + public Iterator iterator() { + return optGroups.iterator(); + } + + + /** + * Flatten the different sets of related options into a single collection + * of options. + * @return all options in the ToolOptions as a single set + */ + public Options merge() { + Options mergedOpts = new Options(); + int totalOpts = 0; + for (RelatedOptions relatedOpts : this) { + for (Object optObj : relatedOpts.getOptions()) { + Option opt = (Option) optObj; + mergedOpts.addOption(opt); + totalOpts++; + } + } + + return mergedOpts; + } + + /** + * Print the help to the console using a default help formatter. + */ + public void printHelp() { + printHelp(new HelpFormatter()); + } + + /** + * Print the help to the console using the specified help formatter + * @param formatter the HelpFormatter to use. + */ + public void printHelp(HelpFormatter formatter) { + printHelp(formatter, new PrintWriter(System.out, true)); + } + + /** + * Print the help to the specified PrintWriter, using the specified + * help formatter. + * @param formatter the HelpFormatter to use. + * @param pw the PrintWriter to emit to. + */ + public void printHelp(HelpFormatter formatter, PrintWriter pw) { + boolean first = true; + for (RelatedOptions optGroup : optGroups) { + if (!first) { + pw.println(""); + } + pw.println(optGroup.getTitle() + ":"); + formatter.printOptions(pw, formatter.getWidth(), optGroup, 0, 4); + first = false; + } + } + + @Override + public String toString() { + StringWriter sw = new StringWriter(); + printHelp(new HelpFormatter(), new PrintWriter(sw)); + sw.flush(); + return sw.getBuffer().toString(); + } + +} + diff --git a/src/java/org/apache/hadoop/sqoop/hive/HiveImport.java b/src/java/org/apache/hadoop/sqoop/hive/HiveImport.java index 0fa6742e..2d52ce93 100644 --- a/src/java/org/apache/hadoop/sqoop/hive/HiveImport.java +++ b/src/java/org/apache/hadoop/sqoop/hive/HiveImport.java @@ -49,11 +49,14 @@ public class HiveImport { private SqoopOptions options; private ConnManager connManager; private Configuration configuration; + private boolean generateOnly; - public HiveImport(final SqoopOptions opts, final ConnManager connMgr, final Configuration conf) { + public HiveImport(final SqoopOptions opts, final ConnManager connMgr, + final Configuration conf, final boolean generateOnly) { this.options = opts; this.connManager = connMgr; this.configuration = conf; + this.generateOnly = generateOnly; } @@ -110,7 +113,7 @@ private void removeTempLogs(String tableName) throws IOException { * do any side-effecting actions in Hive. */ private boolean isGenerateOnly() { - return options.getAction() == SqoopOptions.ControlAction.GenerateOnly; + return generateOnly; } /** @@ -134,9 +137,11 @@ private File getScriptFile(String outputTableName) throws IOException { * * @param inputTableName the name of the table as loaded into HDFS * @param outputTableName the name of the table to create in Hive. + * @param createOnly if true, run the CREATE TABLE statement but not + * LOAD DATA. */ - public void importTable(String inputTableName, String outputTableName) - throws IOException { + public void importTable(String inputTableName, String outputTableName, + boolean createOnly) throws IOException { if (!isGenerateOnly()) { removeTempLogs(inputTableName); @@ -177,7 +182,7 @@ public void importTable(String inputTableName, String outputTableName) FileOutputStream fos = new FileOutputStream(scriptFile); w = new BufferedWriter(new OutputStreamWriter(fos)); w.write(createTableStr, 0, createTableStr.length()); - if (!options.doCreateHiveTableOnly()) { + if (!createOnly) { w.write(loadDataStmtStr, 0, loadDataStmtStr.length()); } } catch (IOException ioe) { diff --git a/src/java/org/apache/hadoop/sqoop/manager/MySQLManager.java b/src/java/org/apache/hadoop/sqoop/manager/MySQLManager.java index 95471c65..95b2ec8c 100644 --- a/src/java/org/apache/hadoop/sqoop/manager/MySQLManager.java +++ b/src/java/org/apache/hadoop/sqoop/manager/MySQLManager.java @@ -19,6 +19,7 @@ package org.apache.hadoop.sqoop.manager; import java.io.IOException; +import java.io.PrintWriter; import java.net.URI; import java.net.URISyntaxException; import java.sql.PreparedStatement; @@ -31,6 +32,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.sqoop.SqoopOptions; import org.apache.hadoop.sqoop.util.ImportException; @@ -191,7 +193,8 @@ private void checkDateTimeBehavior(ImportJobContext context) { * @param stmt The SQL statement to execute * @return A ResultSet encapsulating the results or null on error */ - protected ResultSet execute(String stmt, Object... args) throws SQLException { + protected ResultSet execute(String stmt, Object... args) + throws SQLException { // Free any previous resources. release(); @@ -210,6 +213,33 @@ protected ResultSet execute(String stmt, Object... args) throws SQLException { return statement.executeQuery(); } + @Override + public void execAndPrint(String s) { + // Override default execAndPrint() with a special version that forces + // use of fully-buffered ResultSets (MySQLManager uses streaming ResultSets + // in the default execute() method; but the execAndPrint() method needs to + // issue overlapped queries for metadata.) + + ResultSet results = null; + try { + // Use default execute() statement which does not issue the + // MySQL-specific setFetchSize() command. + results = super.execute(s); + } catch (SQLException sqlE) { + LOG.error("Error executing statement: " + + StringUtils.stringifyException(sqlE)); + release(); + return; + } + + PrintWriter pw = new PrintWriter(System.out, true); + try { + formatAndPrintResultSet(results, pw); + } finally { + pw.close(); + } + } + public void release() { if (null != this.lastStatement) { try { diff --git a/src/java/org/apache/hadoop/sqoop/manager/SqlManager.java b/src/java/org/apache/hadoop/sqoop/manager/SqlManager.java index 2ff7ee96..a605da37 100644 --- a/src/java/org/apache/hadoop/sqoop/manager/SqlManager.java +++ b/src/java/org/apache/hadoop/sqoop/manager/SqlManager.java @@ -29,6 +29,7 @@ import org.apache.hadoop.sqoop.util.ResultSetPrinter; import java.io.IOException; +import java.io.PrintWriter; import java.sql.Connection; import java.sql.DatabaseMetaData; import java.sql.DriverManager; @@ -46,6 +47,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.util.StringUtils; /** * ConnManager implementation for generic SQL-compliant database. @@ -407,36 +409,36 @@ public void close() throws SQLException { } /** - * Poor man's SQL query interface; used for debugging. - * @param s + * Prints the contents of a ResultSet to the specified PrintWriter. + * The ResultSet is closed at the end of this method. + * @param results the ResultSet to print. + * @param pw the location to print the data to. */ - public void execAndPrint(String s) { - System.out.println("Executing statement: " + s); - ResultSet results; - try { - results = execute(s); - } catch (SQLException sqlE) { - LOG.error("Error executing statement: " + sqlE.toString()); - release(); - return; - } - + protected void formatAndPrintResultSet(ResultSet results, PrintWriter pw) { try { try { int cols = results.getMetaData().getColumnCount(); - System.out.println("Got " + cols + " columns back"); + pw.println("Got " + cols + " columns back"); if (cols > 0) { - System.out.println("Schema: " + results.getMetaData().getSchemaName(1)); - System.out.println("Table: " + results.getMetaData().getTableName(1)); + ResultSetMetaData rsmd = results.getMetaData(); + String schema = rsmd.getSchemaName(1); + String table = rsmd.getTableName(1); + if (null != schema) { + pw.println("Schema: " + schema); + } + + if (null != table) { + pw.println("Table: " + table); + } } } catch (SQLException sqlE) { LOG.error("SQLException reading result metadata: " + sqlE.toString()); } try { - new ResultSetPrinter().printResultSet(System.out, results); + new ResultSetPrinter().printResultSet(pw, results); } catch (IOException ioe) { - LOG.error("IOException writing results to stdout: " + ioe.toString()); + LOG.error("IOException writing results: " + ioe.toString()); return; } } finally { @@ -451,6 +453,29 @@ public void execAndPrint(String s) { } } + /** + * Poor man's SQL query interface; used for debugging. + * @param s the SQL statement to execute. + */ + public void execAndPrint(String s) { + ResultSet results = null; + try { + results = execute(s); + } catch (SQLException sqlE) { + LOG.error("Error executing statement: " + + StringUtils.stringifyException(sqlE)); + release(); + return; + } + + PrintWriter pw = new PrintWriter(System.out, true); + try { + formatAndPrintResultSet(results, pw); + } finally { + pw.close(); + } + } + /** * Create a connection to the database; usually used only from within * getConnection(), which enforces a singleton guarantee around the diff --git a/src/java/org/apache/hadoop/sqoop/orm/ClassWriter.java b/src/java/org/apache/hadoop/sqoop/orm/ClassWriter.java index 07c130cb..b51d8f4e 100644 --- a/src/java/org/apache/hadoop/sqoop/orm/ClassWriter.java +++ b/src/java/org/apache/hadoop/sqoop/orm/ClassWriter.java @@ -906,6 +906,7 @@ public StringBuilder generateClassForColumns(Map columnTypes, sb.append("import java.sql.Timestamp;\n"); sb.append("import java.util.Iterator;\n"); sb.append("import java.util.List;\n"); + sb.append("\n"); String className = tableNameInfo.getShortClassForTable(tableName); sb.append("public class " + className diff --git a/src/java/org/apache/hadoop/sqoop/shims/HadoopShim.java b/src/java/org/apache/hadoop/sqoop/shims/HadoopShim.java index 0400d2d6..655fcad6 100644 --- a/src/java/org/apache/hadoop/sqoop/shims/HadoopShim.java +++ b/src/java/org/apache/hadoop/sqoop/shims/HadoopShim.java @@ -37,6 +37,16 @@ public abstract class HadoopShim { protected HadoopShim() { } + /** + * Parse arguments in 'args' via the GenericOptionsParser and + * embed the results in the supplied configuration. + * @param conf the configuration to populate with generic options. + * @param args the arguments to process. + * @return the unused args to be passed to the application itself. + */ + public abstract String [] parseGenericOptions(Configuration conf, + String [] args) throws IOException; + /** * @return the number of mapper output records from a job using its counters. */ diff --git a/src/java/org/apache/hadoop/sqoop/tool/BaseSqoopTool.java b/src/java/org/apache/hadoop/sqoop/tool/BaseSqoopTool.java new file mode 100644 index 00000000..319c5edc --- /dev/null +++ b/src/java/org/apache/hadoop/sqoop/tool/BaseSqoopTool.java @@ -0,0 +1,633 @@ +/** + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. 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.hadoop.sqoop.tool; + +import java.sql.SQLException; +import java.util.Arrays; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Options; +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 org.apache.hadoop.sqoop.ConnFactory; +import org.apache.hadoop.sqoop.Sqoop; +import org.apache.hadoop.sqoop.SqoopOptions; +import org.apache.hadoop.sqoop.SqoopOptions.InvalidOptionsException; +import org.apache.hadoop.sqoop.cli.RelatedOptions; +import org.apache.hadoop.sqoop.cli.ToolOptions; +import org.apache.hadoop.sqoop.manager.ConnManager; +import org.apache.hadoop.sqoop.shims.ShimLoader; + +/** + * Layer on top of SqoopTool that provides some basic common code + * that most SqoopTool implementations will use. + * + * Subclasses should call init() at the top of their run() method, + * and call destroy() at the end in a finally block. + */ +public abstract class BaseSqoopTool extends SqoopTool { + + public static final Log LOG = LogFactory.getLog(BaseSqoopTool.class.getName()); + + public static final String HELP_STR = "\nTry --help for usage instructions."; + + // Here are all the arguments that are used by the standard sqoop tools. + // Their names are recorded here so that tools can share them and their + // use consistently. The argument parser applies the leading '--' to each + // string. + public static final String CONNECT_STRING_ARG = "connect"; + public static final String DRIVER_ARG = "driver"; + public static final String USERNAME_ARG = "username"; + public static final String PASSWORD_ARG = "password"; + public static final String PASSWORD_PROMPT_ARG = "P"; + public static final String DIRECT_ARG = "direct"; + public static final String TABLE_ARG = "table"; + public static final String COLUMNS_ARG = "columns"; + public static final String SPLIT_BY_ARG = "split-by"; + public static final String WHERE_ARG = "where"; + public static final String HADOOP_HOME_ARG = "hadoop-home"; + public static final String HIVE_HOME_ARG = "hive-home"; + public static final String WAREHOUSE_DIR_ARG = "warehouse-dir"; + public static final String FMT_SEQUENCEFILE_ARG = "as-sequencefile"; + public static final String FMT_TEXTFILE_ARG = "as-textfile"; + public static final String HIVE_IMPORT_ARG = "hive-import"; + public static final String HIVE_TABLE_ARG = "hive-table"; + public static final String HIVE_OVERWRITE_ARG = "hive-overwrite"; + public static final String NUM_MAPPERS_ARG = "num-mappers"; + public static final String NUM_MAPPERS_SHORT_ARG = "m"; + public static final String COMPRESS_ARG = "compress"; + public static final String COMPRESS_SHORT_ARG = "z"; + public static final String DIRECT_SPLIT_SIZE_ARG = "direct-split-size"; + public static final String INLINE_LOB_LIMIT_ARG = "inline-lob-limit"; + public static final String EXPORT_PATH_ARG = "export-dir"; + public static final String FIELDS_TERMINATED_BY_ARG = "fields-terminated-by"; + public static final String LINES_TERMINATED_BY_ARG = "lines-terminated-by"; + public static final String OPTIONALLY_ENCLOSED_BY_ARG = + "optionally-enclosed-by"; + public static final String ENCLOSED_BY_ARG = "enclosed-by"; + public static final String ESCAPED_BY_ARG = "escaped-by"; + public static final String MYSQL_DELIMITERS_ARG = "mysql-delimiters"; + public static final String INPUT_FIELDS_TERMINATED_BY_ARG = + "input-fields-terminated-by"; + public static final String INPUT_LINES_TERMINATED_BY_ARG = + "input-lines-terminated-by"; + public static final String INPUT_OPTIONALLY_ENCLOSED_BY_ARG = + "input-optionally-enclosed-by"; + public static final String INPUT_ENCLOSED_BY_ARG = "input-enclosed-by"; + public static final String INPUT_ESCAPED_BY_ARG = "input-escaped-by"; + public static final String CODE_OUT_DIR_ARG = "outdir"; + public static final String BIN_OUT_DIR_ARG = "bindir"; + public static final String PACKAGE_NAME_ARG = "package-name"; + public static final String CLASS_NAME_ARG = "class-name"; + public static final String JAR_FILE_NAME_ARG = "jar-file"; + public static final String DEBUG_SQL_ARG = "expr"; + public static final String DEBUG_SQL_SHORT_ARG = "e"; + public static final String VERBOSE_ARG = "verbose"; + public static final String HELP_ARG = "help"; + + + public BaseSqoopTool() { + } + + public BaseSqoopTool(String toolName) { + super(toolName); + } + + protected ConnManager manager; + + public ConnManager getManager() { + return manager; + } + + protected void setManager(ConnManager mgr) { + this.manager = mgr; + } + + /** + * Should be called at the beginning of the run() method to initialize + * the connection manager, etc. If this succeeds (returns true), it should + * be paired with a call to destroy(). + * @return true on success, false on failure. + */ + protected boolean init(SqoopOptions sqoopOpts) { + + // Make sure shim jar is classloaded early. + ShimLoader.getHadoopShim(sqoopOpts.getConf()); + + // Get the connection to the database. + try { + this.manager = new ConnFactory(sqoopOpts.getConf()).getManager(sqoopOpts); + return true; + } catch (Exception e) { + LOG.error("Got error creating database manager: " + + StringUtils.stringifyException(e)); + if (System.getProperty(Sqoop.SQOOP_RETHROW_PROPERTY) != null) { + throw new RuntimeException(e); + } + } + + return false; + } + + /** + * Should be called in a 'finally' block at the end of the run() method. + */ + protected void destroy(SqoopOptions sqoopOpts) { + if (null != manager) { + try { + manager.close(); + } catch (SQLException sqlE) { + LOG.warn("Error while closing connection: " + sqlE); + } + } + } + + /** + * Examines a subset of the arrray presented, and determines if it + * contains any non-empty arguments. If so, logs the arguments + * and returns true. + * + * @param argv an array of strings to check. + * @param offset the first element of the array to check + * @param len the number of elements to check + * @return true if there are any non-null, non-empty argument strings + * present. + */ + protected boolean hasUnrecognizedArgs(String [] argv, int offset, int len) { + if (argv == null) { + return false; + } + + boolean unrecognized = false; + boolean printedBanner = false; + for (int i = offset; i < Math.min(argv.length, offset + len); i++) { + if (argv[i] != null && argv[i].length() > 0) { + if (!printedBanner) { + LOG.error("Error parsing arguments for " + getToolName() + ":"); + printedBanner = true; + } + LOG.error("Unrecognized argument: " + argv[i]); + unrecognized = true; + } + } + + return unrecognized; + } + + protected boolean hasUnrecognizedArgs(String [] argv) { + if (null == argv) { + return false; + } + return hasUnrecognizedArgs(argv, 0, argv.length); + } + + + /** + * If argv contains an entry "--", return an array containing all elements + * after the "--" separator. Otherwise, return null. + * @param argv a set of arguments to scan for the subcommand arguments. + */ + protected String [] getSubcommandArgs(String [] argv) { + if (null == argv) { + return null; + } + + for (int i = 0; i < argv.length; i++) { + if (argv[i].equals("--")) { + return Arrays.copyOfRange(argv, i + 1, argv.length); + } + } + + return null; + } + + /** + * @return RelatedOptions used by most/all Sqoop tools. + */ + protected RelatedOptions getCommonOptions() { + // Connection args (common) + RelatedOptions commonOpts = new RelatedOptions("Common arguments"); + commonOpts.addOption(OptionBuilder.withArgName("jdbc-uri") + .hasArg().withDescription("Specify JDBC connect string") + .withLongOpt(CONNECT_STRING_ARG) + .create()); + commonOpts.addOption(OptionBuilder.withArgName("class-name") + .hasArg().withDescription("Manually specify JDBC driver class to use") + .withLongOpt(DRIVER_ARG) + .create()); + commonOpts.addOption(OptionBuilder.withArgName("username") + .hasArg().withDescription("Set authentication username") + .withLongOpt(USERNAME_ARG) + .create()); + commonOpts.addOption(OptionBuilder.withArgName("password") + .hasArg().withDescription("Set authentication password") + .withLongOpt(PASSWORD_ARG) + .create()); + commonOpts.addOption(OptionBuilder.withDescription("Read password from console") + .create(PASSWORD_PROMPT_ARG)); + + commonOpts.addOption(OptionBuilder.withArgName("dir") + .hasArg().withDescription("Override $HADOOP_HOME") + .withLongOpt(HADOOP_HOME_ARG) + .create()); + + // misc (common) + commonOpts.addOption(OptionBuilder + .withDescription("Print more information while working") + .withLongOpt(VERBOSE_ARG) + .create()); + commonOpts.addOption(OptionBuilder + .withDescription("Print usage instructions") + .withLongOpt(HELP_ARG) + .create()); + + return commonOpts; + } + + /** + * @param explicitHiveImport true if the user has an explicit --hive-import + * available, or false if this is implied by the tool. + * @return options governing interaction with Hive + */ + protected RelatedOptions getHiveOptions(boolean explicitHiveImport) { + RelatedOptions hiveOpts = new RelatedOptions("Hive arguments"); + if (explicitHiveImport) { + hiveOpts.addOption(OptionBuilder + .withDescription("Import tables into Hive " + + "(Uses Hive's default delimiters if none are set.)") + .withLongOpt(HIVE_IMPORT_ARG) + .create()); + } + + hiveOpts.addOption(OptionBuilder.withArgName("dir") + .hasArg().withDescription("Override $HIVE_HOME") + .withLongOpt(HIVE_HOME_ARG) + .create()); + hiveOpts.addOption(OptionBuilder + .withDescription("Overwrite existing data in the Hive table") + .withLongOpt(HIVE_OVERWRITE_ARG) + .create()); + hiveOpts.addOption(OptionBuilder.withArgName("table-name") + .hasArg() + .withDescription("Sets the table name to use when importing to hive") + .withLongOpt(HIVE_TABLE_ARG) + .create()); + + return hiveOpts; + } + + /** + * @return options governing output format delimiters + */ + protected RelatedOptions getOutputFormatOptions() { + RelatedOptions formatOpts = new RelatedOptions( + "Output line formatting arguments"); + formatOpts.addOption(OptionBuilder.withArgName("char") + .hasArg() + .withDescription("Sets the field separator character") + .withLongOpt(FIELDS_TERMINATED_BY_ARG) + .create()); + formatOpts.addOption(OptionBuilder.withArgName("char") + .hasArg() + .withDescription("Sets the end-of-line character") + .withLongOpt(LINES_TERMINATED_BY_ARG) + .create()); + formatOpts.addOption(OptionBuilder.withArgName("char") + .hasArg() + .withDescription("Sets a field enclosing character") + .withLongOpt(OPTIONALLY_ENCLOSED_BY_ARG) + .create()); + formatOpts.addOption(OptionBuilder.withArgName("char") + .hasArg() + .withDescription("Sets a required field enclosing character") + .withLongOpt(ENCLOSED_BY_ARG) + .create()); + formatOpts.addOption(OptionBuilder.withArgName("char") + .hasArg() + .withDescription("Sets the escape character") + .withLongOpt(ESCAPED_BY_ARG) + .create()); + formatOpts.addOption(OptionBuilder + .withDescription("Uses MySQL's default delimiter set: " + + "fields: , lines: \\n escaped-by: \\ optionally-enclosed-by: '") + .withLongOpt(MYSQL_DELIMITERS_ARG) + .create()); + + return formatOpts; + } + + /** + * @return options governing input format delimiters + */ + protected RelatedOptions getInputFormatOptions() { + RelatedOptions inputFormatOpts = new RelatedOptions("Input parsing arguments"); + inputFormatOpts.addOption(OptionBuilder.withArgName("char") + .hasArg() + .withDescription("Sets the input field separator") + .withLongOpt(INPUT_FIELDS_TERMINATED_BY_ARG) + .create()); + inputFormatOpts.addOption(OptionBuilder.withArgName("char") + .hasArg() + .withDescription("Sets the input end-of-line char") + .withLongOpt(INPUT_LINES_TERMINATED_BY_ARG) + .create()); + inputFormatOpts.addOption(OptionBuilder.withArgName("char") + .hasArg() + .withDescription("Sets a field enclosing character") + .withLongOpt(INPUT_OPTIONALLY_ENCLOSED_BY_ARG) + .create()); + inputFormatOpts.addOption(OptionBuilder.withArgName("char") + .hasArg() + .withDescription("Sets a required field encloser") + .withLongOpt(INPUT_ENCLOSED_BY_ARG) + .create()); + inputFormatOpts.addOption(OptionBuilder.withArgName("char") + .hasArg() + .withDescription("Sets the input escape character") + .withLongOpt(INPUT_ESCAPED_BY_ARG) + .create()); + + return inputFormatOpts; + } + + /** + * @param multiTable true if these options will be used for bulk code-gen. + * @return options related to code generation. + */ + protected RelatedOptions getCodeGenOpts(boolean multiTable) { + RelatedOptions codeGenOpts = new RelatedOptions("Code generation arguments"); + codeGenOpts.addOption(OptionBuilder.withArgName("dir") + .hasArg() + .withDescription("Output directory for generated code") + .withLongOpt(CODE_OUT_DIR_ARG) + .create()); + codeGenOpts.addOption(OptionBuilder.withArgName("dir") + .hasArg() + .withDescription("Output directory for compiled objects") + .withLongOpt(BIN_OUT_DIR_ARG) + .create()); + codeGenOpts.addOption(OptionBuilder.withArgName("name") + .hasArg() + .withDescription("Put auto-generated classes in this package") + .withLongOpt(PACKAGE_NAME_ARG) + .create()); + if (!multiTable) { + codeGenOpts.addOption(OptionBuilder.withArgName("name") + .hasArg() + .withDescription("Sets the generated class name." + + "This overrides --" + PACKAGE_NAME_ARG + ". When combined " + + "with --" + JAR_FILE_NAME_ARG + ", sets the input class.") + .withLongOpt(CLASS_NAME_ARG) + .create()); + } + return codeGenOpts; + } + + + /** + * Apply common command-line to the state. + */ + protected void applyCommonOptions(CommandLine in, SqoopOptions out) + throws InvalidOptionsException { + + // common options. + 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(CONNECT_STRING_ARG)) { + out.setConnectString(in.getOptionValue(CONNECT_STRING_ARG)); + } + + if (in.hasOption(DRIVER_ARG)) { + out.setDriverClassName(in.getOptionValue(DRIVER_ARG)); + } + + if (in.hasOption(USERNAME_ARG)) { + out.setUsername(in.getOptionValue(USERNAME_ARG)); + if (null == out.getPassword()) { + // Set password to empty if the username is set first, + // to ensure that they're either both null or neither is. + out.setPassword(""); + } + } + + if (in.hasOption(PASSWORD_ARG)) { + LOG.warn("Setting your password on the command-line is insecure. " + + "Consider using -" + PASSWORD_PROMPT_ARG + " instead."); + out.setPassword(in.getOptionValue(PASSWORD_ARG)); + } + + if (in.hasOption(PASSWORD_PROMPT_ARG)) { + out.setPasswordFromConsole(); + } + + if (in.hasOption(HADOOP_HOME_ARG)) { + out.setHadoopHome(in.getOptionValue(HADOOP_HOME_ARG)); + } + + } + + protected void applyHiveOptions(CommandLine in, SqoopOptions out) + throws InvalidOptionsException { + + if (in.hasOption(HIVE_HOME_ARG)) { + out.setHiveHome(in.getOptionValue(HIVE_HOME_ARG)); + } + + if (in.hasOption(HIVE_IMPORT_ARG)) { + out.setHiveImport(true); + } + + if (in.hasOption(HIVE_OVERWRITE_ARG)) { + out.setOverwriteHiveTable(true); + } + + if (in.hasOption(HIVE_TABLE_ARG)) { + out.setHiveTableName(in.getOptionValue(HIVE_TABLE_ARG)); + } + } + + protected void applyOutputFormatOptions(CommandLine in, SqoopOptions out) + throws InvalidOptionsException { + if (in.hasOption(FIELDS_TERMINATED_BY_ARG)) { + out.setFieldsTerminatedBy(SqoopOptions.toChar( + in.getOptionValue(FIELDS_TERMINATED_BY_ARG))); + out.setExplicitDelims(true); + } + + if (in.hasOption(LINES_TERMINATED_BY_ARG)) { + out.setLinesTerminatedBy(SqoopOptions.toChar( + in.getOptionValue(LINES_TERMINATED_BY_ARG))); + out.setExplicitDelims(true); + } + + if (in.hasOption(OPTIONALLY_ENCLOSED_BY_ARG)) { + out.setEnclosedBy(SqoopOptions.toChar( + in.getOptionValue(OPTIONALLY_ENCLOSED_BY_ARG))); + out.setOutputEncloseRequired(false); + out.setExplicitDelims(true); + } + + if (in.hasOption(ENCLOSED_BY_ARG)) { + out.setEnclosedBy(SqoopOptions.toChar( + in.getOptionValue(ENCLOSED_BY_ARG))); + out.setOutputEncloseRequired(true); + out.setExplicitDelims(true); + } + + if (in.hasOption(ESCAPED_BY_ARG)) { + out.setEscapedBy(SqoopOptions.toChar( + in.getOptionValue(ESCAPED_BY_ARG))); + out.setExplicitDelims(true); + } + + if (in.hasOption(MYSQL_DELIMITERS_ARG)) { + out.setOutputEncloseRequired(false); + out.setFieldsTerminatedBy(','); + out.setLinesTerminatedBy('\n'); + out.setEscapedBy('\\'); + out.setEnclosedBy('\''); + out.setExplicitDelims(true); + } + } + + protected void applyInputFormatOptions(CommandLine in, SqoopOptions out) + throws InvalidOptionsException { + if (in.hasOption(INPUT_FIELDS_TERMINATED_BY_ARG)) { + out.setInputFieldsTerminatedBy(SqoopOptions.toChar( + in.getOptionValue(INPUT_FIELDS_TERMINATED_BY_ARG))); + } + + if (in.hasOption(INPUT_LINES_TERMINATED_BY_ARG)) { + out.setInputLinesTerminatedBy(SqoopOptions.toChar( + in.getOptionValue(INPUT_LINES_TERMINATED_BY_ARG))); + } + + if (in.hasOption(INPUT_OPTIONALLY_ENCLOSED_BY_ARG)) { + out.setInputEnclosedBy(SqoopOptions.toChar( + in.getOptionValue(INPUT_OPTIONALLY_ENCLOSED_BY_ARG))); + out.setInputEncloseRequired(false); + } + + if (in.hasOption(INPUT_ENCLOSED_BY_ARG)) { + out.setInputEnclosedBy(SqoopOptions.toChar( + in.getOptionValue(INPUT_ENCLOSED_BY_ARG))); + out.setInputEncloseRequired(true); + } + + if (in.hasOption(INPUT_ESCAPED_BY_ARG)) { + out.setInputEscapedBy(SqoopOptions.toChar( + in.getOptionValue(INPUT_ESCAPED_BY_ARG))); + } + } + + protected void applyCodeGenOptions(CommandLine in, SqoopOptions out, + boolean multiTable) throws InvalidOptionsException { + if (in.hasOption(CODE_OUT_DIR_ARG)) { + out.setCodeOutputDir(in.getOptionValue(CODE_OUT_DIR_ARG)); + } + + if (in.hasOption(BIN_OUT_DIR_ARG)) { + out.setJarOutputDir(in.getOptionValue(BIN_OUT_DIR_ARG)); + } + + if (in.hasOption(PACKAGE_NAME_ARG)) { + out.setPackageName(in.getOptionValue(PACKAGE_NAME_ARG)); + } + + if (!multiTable && in.hasOption(CLASS_NAME_ARG)) { + out.setClassName(in.getOptionValue(CLASS_NAME_ARG)); + } + } + + protected void validateCommonOptions(SqoopOptions options) + throws InvalidOptionsException { + if (options.getConnectString() == null) { + throw new InvalidOptionsException( + "Error: Required argument --connect is missing." + + HELP_STR); + } + } + + protected void validateCodeGenOptions(SqoopOptions options) + throws InvalidOptionsException { + if (options.getClassName() != null && options.getPackageName() != null) { + throw new InvalidOptionsException( + "--class-name overrides --package-name. You cannot use both." + + HELP_STR); + } + } + + protected void validateOutputFormatOptions(SqoopOptions options) + throws InvalidOptionsException { + if (options.doHiveImport()) { + if (!options.explicitDelims()) { + // user hasn't manually specified delimiters, and wants to import + // straight to Hive. Use Hive-style delimiters. + LOG.info("Using Hive-specific delimiters for output. You can override"); + LOG.info("delimiters with --fields-terminated-by, etc."); + options.setFieldsTerminatedBy((char) 0x1); // ^A + options.setLinesTerminatedBy('\n'); + options.setEnclosedBy('\000'); // no enclosing in Hive. + options.setEscapedBy('\000'); // no escaping in Hive. + options.setOutputEncloseRequired(false); + } + + if (options.getOutputEscapedBy() != '\000') { + LOG.warn("Hive does not support escape characters in fields;"); + LOG.warn("parse errors in Hive may result from using --escaped-by."); + } + + if (options.getOutputEnclosedBy() != '\000') { + LOG.warn("Hive does not support quoted strings; parse errors"); + LOG.warn("in Hive may result from using --enclosed-by."); + } + } + } + + protected void validateHiveOptions(SqoopOptions options) { + // Empty; this method is present to maintain API consistency, and + // is reserved for future constraints on Hive options. + } + + + +} + diff --git a/src/java/org/apache/hadoop/sqoop/tool/CodeGenTool.java b/src/java/org/apache/hadoop/sqoop/tool/CodeGenTool.java new file mode 100644 index 00000000..8201d2d4 --- /dev/null +++ b/src/java/org/apache/hadoop/sqoop/tool/CodeGenTool.java @@ -0,0 +1,193 @@ +/** + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. 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.hadoop.sqoop.tool; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +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.hadoop.sqoop.Sqoop; +import org.apache.hadoop.sqoop.SqoopOptions; +import org.apache.hadoop.sqoop.SqoopOptions.InvalidOptionsException; +import org.apache.hadoop.sqoop.cli.RelatedOptions; +import org.apache.hadoop.sqoop.cli.ToolOptions; +import org.apache.hadoop.sqoop.hive.HiveImport; +import org.apache.hadoop.sqoop.manager.ImportJobContext; +import org.apache.hadoop.sqoop.orm.ClassWriter; +import org.apache.hadoop.sqoop.orm.CompilationManager; +import org.apache.hadoop.sqoop.util.ImportException; + +/** + * Tool that generates code from a database schema. + */ +public class CodeGenTool extends BaseSqoopTool { + + public static final Log LOG = LogFactory.getLog(CodeGenTool.class.getName()); + + private List generatedJarFiles; + + public CodeGenTool() { + super("codegen"); + generatedJarFiles = new ArrayList(); + } + + /** + * @return a list of jar files generated as part of this import process + */ + public List getGeneratedJarFiles() { + ArrayList out = new ArrayList(generatedJarFiles); + return out; + } + + /** + * Generate the .class and .jar files + * @return the filename of the emitted jar file. + * @throws IOException + */ + public String generateORM(SqoopOptions options, String tableName) + throws IOException { + String existingJar = options.getExistingJarName(); + if (existingJar != null) { + // This code generator is being invoked as part of an import or export + // process, and the user has pre-specified a jar and class to use. + // Don't generate. + LOG.info("Using existing jar: " + existingJar); + return existingJar; + } + + LOG.info("Beginning code generation"); + CompilationManager compileMgr = new CompilationManager(options); + ClassWriter classWriter = new ClassWriter(options, manager, tableName, + compileMgr); + classWriter.generate(); + compileMgr.compile(); + compileMgr.jar(); + String jarFile = compileMgr.getJarFilename(); + this.generatedJarFiles.add(jarFile); + return jarFile; + } + + + @Override + /** {@inheritDoc} */ + public int run(SqoopOptions options) { + if (!init(options)) { + return 1; + } + + try { + generateORM(options, options.getTableName()); + + // If the user has also specified Hive import code generation, + // use a HiveImport to generate the DDL statements and write + // them to files (but don't actually perform the import -- thus + // the generateOnly=true in the constructor). + if (options.doHiveImport()) { + HiveImport hiveImport = new HiveImport(options, manager, + options.getConf(), true); + hiveImport.importTable(options.getTableName(), + options.getHiveTableName(), true); + } + + } catch (IOException ioe) { + LOG.error("Encountered IOException running codegen job: " + + StringUtils.stringifyException(ioe)); + if (System.getProperty(Sqoop.SQOOP_RETHROW_PROPERTY) != null) { + throw new RuntimeException(ioe); + } else { + return 1; + } + } finally { + destroy(options); + } + + return 0; + } + + @Override + /** Configure the command-line arguments we expect to receive */ + public void configureOptions(ToolOptions toolOptions) { + + toolOptions.addUniqueOptions(getCommonOptions()); + + RelatedOptions codeGenOpts = getCodeGenOpts(false); + codeGenOpts.addOption(OptionBuilder.withArgName("table-name") + .hasArg() + .withDescription("Table to generate code for") + .withLongOpt(TABLE_ARG) + .create()); + toolOptions.addUniqueOptions(codeGenOpts); + + toolOptions.addUniqueOptions(getOutputFormatOptions()); + toolOptions.addUniqueOptions(getInputFormatOptions()); + toolOptions.addUniqueOptions(getHiveOptions(true)); + } + + @Override + /** {@inheritDoc} */ + public void printHelp(ToolOptions toolOptions) { + super.printHelp(toolOptions); + System.out.println(""); + System.out.println( + "At minimum, you must specify --connect and --table"); + } + + @Override + /** {@inheritDoc} */ + public void applyOptions(CommandLine in, SqoopOptions out) + throws InvalidOptionsException { + + if (in.hasOption(TABLE_ARG)) { + out.setTableName(in.getOptionValue(TABLE_ARG)); + } + + applyCommonOptions(in, out); + applyOutputFormatOptions(in, out); + applyInputFormatOptions(in, out); + applyCodeGenOptions(in, out, false); + applyHiveOptions(in, out); + } + + @Override + /** {@inheritDoc} */ + public void validateOptions(SqoopOptions options) + throws InvalidOptionsException { + + if (hasUnrecognizedArgs(extraArguments)) { + throw new InvalidOptionsException(HELP_STR); + } + + validateCommonOptions(options); + validateCodeGenOptions(options); + validateOutputFormatOptions(options); + validateHiveOptions(options); + + if (options.getTableName() == null) { + throw new InvalidOptionsException( + "--table is required for code generation." + HELP_STR); + } + } +} + diff --git a/src/java/org/apache/hadoop/sqoop/tool/CreateHiveTableTool.java b/src/java/org/apache/hadoop/sqoop/tool/CreateHiveTableTool.java new file mode 100644 index 00000000..24af6359 --- /dev/null +++ b/src/java/org/apache/hadoop/sqoop/tool/CreateHiveTableTool.java @@ -0,0 +1,142 @@ +/** + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. 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.hadoop.sqoop.tool; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +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.hadoop.sqoop.Sqoop; +import org.apache.hadoop.sqoop.SqoopOptions; +import org.apache.hadoop.sqoop.SqoopOptions.InvalidOptionsException; +import org.apache.hadoop.sqoop.cli.RelatedOptions; +import org.apache.hadoop.sqoop.cli.ToolOptions; +import org.apache.hadoop.sqoop.hive.HiveImport; +import org.apache.hadoop.sqoop.manager.ImportJobContext; +import org.apache.hadoop.sqoop.orm.ClassWriter; +import org.apache.hadoop.sqoop.orm.CompilationManager; +import org.apache.hadoop.sqoop.util.ImportException; + +/** + * Tool that creates a Hive table definition. + */ +public class CreateHiveTableTool extends BaseSqoopTool { + + public static final Log LOG = LogFactory.getLog( + CreateHiveTableTool.class.getName()); + + public CreateHiveTableTool() { + super("create-hive-table"); + } + + @Override + /** {@inheritDoc} */ + public int run(SqoopOptions options) { + if (!init(options)) { + return 1; + } + + try { + HiveImport hiveImport = new HiveImport(options, manager, + options.getConf(), false); + hiveImport.importTable(options.getTableName(), + options.getHiveTableName(), true); + } catch (IOException ioe) { + LOG.error("Encountered IOException running create table job: " + + StringUtils.stringifyException(ioe)); + if (System.getProperty(Sqoop.SQOOP_RETHROW_PROPERTY) != null) { + throw new RuntimeException(ioe); + } else { + return 1; + } + } finally { + destroy(options); + } + + return 0; + } + + @Override + /** Configure the command-line arguments we expect to receive */ + public void configureOptions(ToolOptions toolOptions) { + + toolOptions.addUniqueOptions(getCommonOptions()); + + RelatedOptions hiveOpts = getHiveOptions(false); + hiveOpts.addOption(OptionBuilder.withArgName("table-name") + .hasArg() + .withDescription("The db table to read the definition from") + .withLongOpt(TABLE_ARG) + .create()); + toolOptions.addUniqueOptions(hiveOpts); + + toolOptions.addUniqueOptions(getOutputFormatOptions()); + } + + @Override + /** {@inheritDoc} */ + public void printHelp(ToolOptions toolOptions) { + super.printHelp(toolOptions); + System.out.println(""); + System.out.println( + "At minimum, you must specify --connect and --table"); + } + + @Override + /** {@inheritDoc} */ + public void applyOptions(CommandLine in, SqoopOptions out) + throws InvalidOptionsException { + + if (in.hasOption(TABLE_ARG)) { + out.setTableName(in.getOptionValue(TABLE_ARG)); + } + + out.setHiveImport(true); + + applyCommonOptions(in, out); + applyHiveOptions(in, out); + applyOutputFormatOptions(in, out); + } + + @Override + /** {@inheritDoc} */ + public void validateOptions(SqoopOptions options) + throws InvalidOptionsException { + + if (hasUnrecognizedArgs(extraArguments)) { + throw new InvalidOptionsException(HELP_STR); + } + + validateCommonOptions(options); + validateOutputFormatOptions(options); + validateHiveOptions(options); + + if (options.getTableName() == null) { + throw new InvalidOptionsException( + "--table is required for table definition importing." + HELP_STR); + } + } +} + diff --git a/src/java/org/apache/hadoop/sqoop/tool/EvalSqlTool.java b/src/java/org/apache/hadoop/sqoop/tool/EvalSqlTool.java new file mode 100644 index 00000000..56d0a7a1 --- /dev/null +++ b/src/java/org/apache/hadoop/sqoop/tool/EvalSqlTool.java @@ -0,0 +1,104 @@ +/** + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. 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.hadoop.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 org.apache.hadoop.sqoop.SqoopOptions; +import org.apache.hadoop.sqoop.SqoopOptions.InvalidOptionsException; +import org.apache.hadoop.sqoop.cli.RelatedOptions; +import org.apache.hadoop.sqoop.cli.ToolOptions; + +/** + * Tool that evaluates a SQL statement and displays the results. + */ +public class EvalSqlTool extends BaseSqoopTool { + + public static final Log LOG = LogFactory.getLog(EvalSqlTool.class.getName()); + + public EvalSqlTool() { + super("eval"); + } + + @Override + /** {@inheritDoc} */ + public int run(SqoopOptions options) { + if (!init(options)) { + return 1; + } + + try { + // just run a SQL statement for debugging purposes. + manager.execAndPrint(options.getDebugSqlCmd()); + } finally { + destroy(options); + } + + return 0; + } + + @Override + /** Configure the command-line arguments we expect to receive */ + public void configureOptions(ToolOptions toolOptions) { + toolOptions.addUniqueOptions(getCommonOptions()); + + RelatedOptions evalOpts = new RelatedOptions("SQL evaluation arguments"); + evalOpts.addOption(OptionBuilder.withArgName("statement") + .hasArg() + .withDescription("Execute 'statement' in SQL and exit") + .withLongOpt(DEBUG_SQL_ARG) + .create(DEBUG_SQL_SHORT_ARG)); + + toolOptions.addUniqueOptions(evalOpts); + } + + @Override + /** {@inheritDoc} */ + public void applyOptions(CommandLine in, SqoopOptions out) + throws InvalidOptionsException { + + applyCommonOptions(in, out); + if (in.hasOption(DEBUG_SQL_ARG)) { + out.setDebugSqlCmd(in.getOptionValue(DEBUG_SQL_ARG)); + } + } + + @Override + /** {@inheritDoc} */ + public void validateOptions(SqoopOptions options) + throws InvalidOptionsException { + + if (hasUnrecognizedArgs(extraArguments)) { + throw new InvalidOptionsException(HELP_STR); + } + + String sqlCmd = options.getDebugSqlCmd(); + if (null == sqlCmd || sqlCmd.length() == 0) { + throw new InvalidOptionsException( + "This command requires the " + DEBUG_SQL_ARG + " argument." + + HELP_STR); + } + + validateCommonOptions(options); + } +} + diff --git a/src/java/org/apache/hadoop/sqoop/tool/ExportTool.java b/src/java/org/apache/hadoop/sqoop/tool/ExportTool.java new file mode 100644 index 00000000..4ed3e897 --- /dev/null +++ b/src/java/org/apache/hadoop/sqoop/tool/ExportTool.java @@ -0,0 +1,265 @@ +/** + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. 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.hadoop.sqoop.tool; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +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.hadoop.sqoop.Sqoop; +import org.apache.hadoop.sqoop.SqoopOptions; +import org.apache.hadoop.sqoop.SqoopOptions.InvalidOptionsException; +import org.apache.hadoop.sqoop.cli.RelatedOptions; +import org.apache.hadoop.sqoop.cli.ToolOptions; +import org.apache.hadoop.sqoop.hive.HiveImport; +import org.apache.hadoop.sqoop.manager.ConnManager; +import org.apache.hadoop.sqoop.manager.ExportJobContext; +import org.apache.hadoop.sqoop.manager.ImportJobContext; +import org.apache.hadoop.sqoop.orm.ClassWriter; +import org.apache.hadoop.sqoop.orm.CompilationManager; +import org.apache.hadoop.sqoop.shims.ShimLoader; +import org.apache.hadoop.sqoop.tool.SqoopTool; +import org.apache.hadoop.sqoop.util.ExportException; +import org.apache.hadoop.sqoop.util.ImportException; + +/** + * Tool that performs HDFS exports to databases. + */ +public class ExportTool extends BaseSqoopTool { + + public static final Log LOG = LogFactory.getLog(ExportTool.class.getName()); + + private CodeGenTool codeGenerator; + + public ExportTool() { + super("export"); + this.codeGenerator = new CodeGenTool(); + } + + /** + * @return a list of jar files generated as part of this im/export process + */ + public List getGeneratedJarFiles() { + return codeGenerator.getGeneratedJarFiles(); + } + + private void exportTable(SqoopOptions options, String tableName) + throws ExportException, IOException { + String jarFile = null; + + // Generate the ORM code for the tables. + jarFile = codeGenerator.generateORM(options, tableName); + + ExportJobContext context = new ExportJobContext(tableName, jarFile, options); + manager.exportTable(context); + } + + @Override + /** {@inheritDoc} */ + public int run(SqoopOptions options) { + + if (!init(options)) { + return 1; + } + + codeGenerator.setManager(manager); + + try { + exportTable(options, options.getTableName()); + } catch (IOException ioe) { + LOG.error("Encountered IOException running export job: " + ioe.toString()); + if (System.getProperty(Sqoop.SQOOP_RETHROW_PROPERTY) != null) { + throw new RuntimeException(ioe); + } else { + return 1; + } + } catch (ExportException ee) { + LOG.error("Error during export: " + ee.toString()); + if (System.getProperty(Sqoop.SQOOP_RETHROW_PROPERTY) != null) { + throw new RuntimeException(ee); + } else { + return 1; + } + } finally { + destroy(options); + } + + return 0; + } + + /** + * Construct the set of options that control exports + * @return the RelatedOptions that can be used to parse the export + * arguments. + */ + protected RelatedOptions getExportOptions() { + RelatedOptions exportOpts = new RelatedOptions("Export control arguments"); + + exportOpts.addOption(OptionBuilder + .withDescription("Use direct export fast path") + .withLongOpt(DIRECT_ARG) + .create()); + exportOpts.addOption(OptionBuilder.withArgName("table-name") + .hasArg().withDescription("Table to populate") + .withLongOpt(TABLE_ARG) + .create()); + exportOpts.addOption(OptionBuilder.withArgName("n") + .hasArg().withDescription("Use 'n' map tasks to export in parallel") + .withLongOpt(NUM_MAPPERS_ARG) + .create(NUM_MAPPERS_SHORT_ARG)); + exportOpts.addOption(OptionBuilder.withArgName("dir") + .hasArg() + .withDescription("HDFS source path for the export") + .withLongOpt(EXPORT_PATH_ARG) + .create()); + + return exportOpts; + } + + @Override + /** Configure the command-line arguments we expect to receive */ + public void configureOptions(ToolOptions toolOptions) { + + toolOptions.addUniqueOptions(getCommonOptions()); + toolOptions.addUniqueOptions(getExportOptions()); + + // Input parsing delimiters + toolOptions.addUniqueOptions(getInputFormatOptions()); + + // Used when sending data to a direct-mode export. + toolOptions.addUniqueOptions(getOutputFormatOptions()); + + // get common codegen opts. + RelatedOptions codeGenOpts = getCodeGenOpts(false); + + // add export-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(""); + System.out.println( + "At minimum, you must specify --connect, --export-dir, and --table"); + } + + @Override + /** {@inheritDoc} */ + public void applyOptions(CommandLine in, SqoopOptions out) + throws InvalidOptionsException { + + try { + applyCommonOptions(in, out); + + if (in.hasOption(DIRECT_ARG)) { + out.setDirectMode(true); + } + + if (in.hasOption(TABLE_ARG)) { + out.setTableName(in.getOptionValue(TABLE_ARG)); + } + + if (in.hasOption(NUM_MAPPERS_ARG)) { + out.setNumMappers(Integer.parseInt(in.getOptionValue(NUM_MAPPERS_ARG))); + } + + if (in.hasOption(EXPORT_PATH_ARG)) { + out.setExportDir(in.getOptionValue(EXPORT_PATH_ARG)); + } + + if (in.hasOption(JAR_FILE_NAME_ARG)) { + out.setExistingJarName(in.getOptionValue(JAR_FILE_NAME_ARG)); + } + + applyInputFormatOptions(in, out); + applyOutputFormatOptions(in, out); + applyOutputFormatOptions(in, out); + applyCodeGenOptions(in, out, false); + } catch (NumberFormatException nfe) { + throw new InvalidOptionsException("Error: expected numeric argument.\n" + + "Try --help for usage."); + } + } + + /** + * Validate export-specific arguments. + * @param options the configured SqoopOptions to check + */ + protected void validateExportOptions(SqoopOptions options) + throws InvalidOptionsException { + if (options.getTableName() == null) { + throw new InvalidOptionsException("Export requires a --table argument." + + HELP_STR); + } else if (options.getExportDir() == null) { + throw new InvalidOptionsException( + "Export requires an --export-dir argument." + + 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); + } + } + + @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 = extraArguments.length; + for (int i = 0; i < extraArguments.length; i++) { + if (extraArguments[i].equals("--")) { + dashPos = i; + break; + } + } + + if (hasUnrecognizedArgs(extraArguments, 0, dashPos)) { + throw new InvalidOptionsException(HELP_STR); + } + + validateExportOptions(options); + validateOutputFormatOptions(options); + validateCommonOptions(options); + validateCodeGenOptions(options); + } +} + diff --git a/src/java/org/apache/hadoop/sqoop/tool/HelpTool.java b/src/java/org/apache/hadoop/sqoop/tool/HelpTool.java new file mode 100644 index 00000000..c2cfe64d --- /dev/null +++ b/src/java/org/apache/hadoop/sqoop/tool/HelpTool.java @@ -0,0 +1,111 @@ +/** + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. 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.hadoop.sqoop.tool; + +import java.util.Set; + +import org.apache.hadoop.sqoop.SqoopOptions; +import org.apache.hadoop.sqoop.cli.RelatedOptions; +import org.apache.hadoop.sqoop.cli.ToolOptions; + +/** + * Tool that explains the usage of Sqoop. + */ +public class HelpTool extends BaseSqoopTool { + + public HelpTool() { + super("help"); + } + + /** + * @param str the string to right-side pad + * @param num the minimum number of characters to return + * @return 'str' with enough right padding to make it num characters long. + */ + private static String padRight(String str, int num) { + StringBuilder sb = new StringBuilder(); + sb.append(str); + for (int count = str.length(); count < num; count++) { + sb.append(" "); + } + + return sb.toString(); + } + + /** + * Print out a list of all SqoopTool implementations and their + * descriptions. + */ + private void printAvailableTools() { + System.out.println("usage: sqoop COMMAND [ARGS]"); + System.out.println(""); + System.out.println("Available commands:"); + + Set toolNames = getToolNames(); + + int maxWidth = 0; + for (String tool : toolNames) { + maxWidth = Math.max(maxWidth, tool.length()); + } + + for (String tool : toolNames) { + System.out.println(" " + padRight(tool, maxWidth+2) + + getToolDescription(tool)); + } + + System.out.println(""); + System.out.println( + "See 'sqoop help COMMAND' for information on a specific command."); + } + + + @Override + /** {@inheritDoc} */ + public int run(SqoopOptions options) { + + if (this.extraArguments.length > 0) { + if (hasUnrecognizedArgs(extraArguments, 1, extraArguments.length)) { + return 1; + } + + SqoopTool subTool = SqoopTool.getTool(extraArguments[0]); + if (null == subTool) { + System.out.println("No such tool: " + extraArguments[0]); + System.out.println( + "Try 'sqoop help' for a list of available commands."); + return 1; + } else { + ToolOptions toolOpts = new ToolOptions(); + subTool.configureOptions(toolOpts); + subTool.printHelp(toolOpts); + return 0; + } + } else { + printAvailableTools(); + } + + return 0; + } + + @Override + public void printHelp(ToolOptions opts) { + System.out.println("usage: sqoop " + getToolName() + " [COMMAND]"); + } +} + diff --git a/src/java/org/apache/hadoop/sqoop/tool/ImportAllTablesTool.java b/src/java/org/apache/hadoop/sqoop/tool/ImportAllTablesTool.java new file mode 100644 index 00000000..f1dbb883 --- /dev/null +++ b/src/java/org/apache/hadoop/sqoop/tool/ImportAllTablesTool.java @@ -0,0 +1,108 @@ +/** + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. 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.hadoop.sqoop.tool; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +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.hadoop.sqoop.Sqoop; +import org.apache.hadoop.sqoop.SqoopOptions; +import org.apache.hadoop.sqoop.SqoopOptions.InvalidOptionsException; +import org.apache.hadoop.sqoop.hive.HiveImport; +import org.apache.hadoop.sqoop.manager.ConnManager; +import org.apache.hadoop.sqoop.manager.ExportJobContext; +import org.apache.hadoop.sqoop.manager.ImportJobContext; +import org.apache.hadoop.sqoop.orm.ClassWriter; +import org.apache.hadoop.sqoop.orm.CompilationManager; +import org.apache.hadoop.sqoop.shims.ShimLoader; +import org.apache.hadoop.sqoop.tool.SqoopTool; +import org.apache.hadoop.sqoop.util.ExportException; +import org.apache.hadoop.sqoop.util.ImportException; + +/** + * Tool that performs database imports of all tables in a database to HDFS. + */ +public class ImportAllTablesTool extends ImportTool { + + public static final Log LOG = LogFactory.getLog( + ImportAllTablesTool.class.getName()); + + public ImportAllTablesTool() { + super("import-all-tables", true); + } + + @Override + /** {@inheritDoc} */ + public int run(SqoopOptions options) { + HiveImport hiveImport = null; + + if (!init(options)) { + return 1; + } + + try { + if (options.doHiveImport()) { + hiveImport = new HiveImport(options, manager, options.getConf(), false); + } + + 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 tableName : tables) { + importTable(options, tableName, hiveImport); + } + } + } catch (IOException ioe) { + LOG.error("Encountered IOException running import job: " + ioe.toString()); + 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; + } + +} + diff --git a/src/java/org/apache/hadoop/sqoop/tool/ImportTool.java b/src/java/org/apache/hadoop/sqoop/tool/ImportTool.java new file mode 100644 index 00000000..0e0a9c78 --- /dev/null +++ b/src/java/org/apache/hadoop/sqoop/tool/ImportTool.java @@ -0,0 +1,371 @@ +/** + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. 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.hadoop.sqoop.tool; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +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.sqoop.Sqoop; +import org.apache.hadoop.sqoop.SqoopOptions; +import org.apache.hadoop.sqoop.SqoopOptions.InvalidOptionsException; +import org.apache.hadoop.sqoop.cli.RelatedOptions; +import org.apache.hadoop.sqoop.cli.ToolOptions; +import org.apache.hadoop.sqoop.hive.HiveImport; +import org.apache.hadoop.sqoop.manager.ImportJobContext; +import org.apache.hadoop.sqoop.orm.ClassWriter; +import org.apache.hadoop.sqoop.orm.CompilationManager; +import org.apache.hadoop.sqoop.util.ImportException; + +/** + * Tool that performs database imports to HDFS. + */ +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; + + public ImportTool() { + this("import", false); + } + + 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 getGeneratedJarFiles() { + return this.codeGenerator.getGeneratedJarFiles(); + } + + protected void 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); + manager.importTable(context); + + // If the user wants this table to be in Hive, perform that post-load. + if (options.doHiveImport()) { + hiveImport.importTable(tableName, options.getHiveTableName(), false); + } + } + + @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 the user specified. + importTable(options, options.getTableName(), hiveImport); + } catch (IOException ioe) { + LOG.error("Encountered IOException running import job: " + ioe.toString()); + 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.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.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("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()); + + return importOpts; + } + + @Override + /** Configure the command-line arguments we expect to receive */ + public void configureOptions(ToolOptions toolOptions) { + + toolOptions.addUniqueOptions(getCommonOptions()); + toolOptions.addUniqueOptions(getImportOptions()); + toolOptions.addUniqueOptions(getOutputFormatOptions()); + toolOptions.addUniqueOptions(getInputFormatOptions()); + toolOptions.addUniqueOptions(getHiveOptions(true)); + + // 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."); + } + + @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)) { + out.setColumns(in.getOptionValue(COLUMNS_ARG).split(",")); + } + + 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(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(NUM_MAPPERS_ARG)) { + out.setNumMappers(Integer.parseInt(in.getOptionValue(NUM_MAPPERS_ARG))); + } + + if (in.hasOption(COMPRESS_ARG)) { + out.setUseCompression(true); + } + + 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(JAR_FILE_NAME_ARG)) { + out.setExistingJarName(in.getOptionValue(JAR_FILE_NAME_ARG)); + } + + applyHiveOptions(in, out); + applyOutputFormatOptions(in, out); + applyInputFormatOptions(in, out); + applyCodeGenOptions(in, out, allTables); + } catch (NumberFormatException nfe) { + throw new InvalidOptionsException("Error: expected numeric argument.\n" + + "Try --help for usage."); + } + } + + /** + * Validate import-specific arguments. + * @param options the configured SqoopOptions to check + */ + protected void validateImportOptions(SqoopOptions options) + throws InvalidOptionsException { + if (!allTables && options.getTableName() == null) { + throw new InvalidOptionsException( + "--table 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); + } + } + + @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 = extraArguments.length; + for (int i = 0; i < extraArguments.length; i++) { + if (extraArguments[i].equals("--")) { + dashPos = i; + break; + } + } + + if (hasUnrecognizedArgs(extraArguments, 0, dashPos)) { + throw new InvalidOptionsException(HELP_STR); + } + + validateImportOptions(options); + validateCommonOptions(options); + validateCodeGenOptions(options); + validateOutputFormatOptions(options); + } +} + diff --git a/src/java/org/apache/hadoop/sqoop/tool/ListDatabasesTool.java b/src/java/org/apache/hadoop/sqoop/tool/ListDatabasesTool.java new file mode 100644 index 00000000..b806494c --- /dev/null +++ b/src/java/org/apache/hadoop/sqoop/tool/ListDatabasesTool.java @@ -0,0 +1,91 @@ +/** + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. 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.hadoop.sqoop.tool; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import org.apache.hadoop.sqoop.SqoopOptions; +import org.apache.hadoop.sqoop.SqoopOptions.InvalidOptionsException; +import org.apache.hadoop.sqoop.cli.RelatedOptions; +import org.apache.hadoop.sqoop.cli.ToolOptions; + +/** + * Tool that lists available databases on a server + */ +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); + } +} + diff --git a/src/java/org/apache/hadoop/sqoop/tool/ListTablesTool.java b/src/java/org/apache/hadoop/sqoop/tool/ListTablesTool.java new file mode 100644 index 00000000..e7a055d8 --- /dev/null +++ b/src/java/org/apache/hadoop/sqoop/tool/ListTablesTool.java @@ -0,0 +1,91 @@ +/** + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. 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.hadoop.sqoop.tool; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import org.apache.hadoop.sqoop.SqoopOptions; +import org.apache.hadoop.sqoop.SqoopOptions.InvalidOptionsException; +import org.apache.hadoop.sqoop.cli.RelatedOptions; +import org.apache.hadoop.sqoop.cli.ToolOptions; + +/** + * Tool that lists available tables in a database. + */ +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); + } +} + diff --git a/src/java/org/apache/hadoop/sqoop/tool/SqoopTool.java b/src/java/org/apache/hadoop/sqoop/tool/SqoopTool.java new file mode 100644 index 00000000..b554e3f4 --- /dev/null +++ b/src/java/org/apache/hadoop/sqoop/tool/SqoopTool.java @@ -0,0 +1,287 @@ +/** + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. 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.hadoop.sqoop.tool; + +import java.io.IOException; +import java.util.Map; +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.GenericOptionsParser; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.util.ToolRunner; + +import org.apache.hadoop.sqoop.SqoopOptions; +import org.apache.hadoop.sqoop.SqoopOptions.InvalidOptionsException; +import org.apache.hadoop.sqoop.cli.RelatedOptions; +import org.apache.hadoop.sqoop.cli.SqoopParser; +import org.apache.hadoop.sqoop.cli.ToolOptions; +import org.apache.hadoop.sqoop.shims.ShimLoader; + +/** + * 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. + */ +public abstract class SqoopTool { + + public static final Log LOG = LogFactory.getLog(SqoopTool.class.getName()); + + private static final Map> TOOLS; + private static final Map DESCRIPTIONS; + + static { + // All SqoopTool instances should be registered here so that + // they can be found internally. + TOOLS = new TreeMap>(); + DESCRIPTIONS = new TreeMap(); + + 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"); + } + + /** + * 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 final void registerTool(String toolName, + Class cls, String description) { + TOOLS.put(toolName, cls); + DESCRIPTIONS.put(toolName, description); + } + + /** + * @return the list of available tools. + */ + public static final Set getToolNames() { + return TOOLS.keySet(); + } + + /** + * @return the SqoopTool instance with the provided name, or null + * if no such tool exists. + */ + public static final SqoopTool getTool(String toolName) { + Class 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 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); + } + + /** 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() + ">"; + } + + public SqoopTool(String name) { + this.toolName = 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()); + } + + String [] toolArgs = args; // args after generic parser is done. + if (useGenericOptions) { + try { + toolArgs = ShimLoader.getHadoopShim().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; + } + + @Override + public String toString() { + return getToolName(); + } +} + diff --git a/src/java/org/apache/hadoop/sqoop/util/ClassLoaderStack.java b/src/java/org/apache/hadoop/sqoop/util/ClassLoaderStack.java index 5b698f35..7744ef22 100644 --- a/src/java/org/apache/hadoop/sqoop/util/ClassLoaderStack.java +++ b/src/java/org/apache/hadoop/sqoop/util/ClassLoaderStack.java @@ -44,7 +44,7 @@ private ClassLoaderStack() { * Sets the classloader for the current thread */ public static void setCurrentClassLoader(ClassLoader cl) { - LOG.info("Restoring classloader: " + cl.toString()); + LOG.debug("Restoring classloader: " + cl.toString()); Thread.currentThread().setContextClassLoader(cl); } @@ -72,13 +72,13 @@ public static ClassLoader addJarFile(String jarFile, String testClassName) LOG.debug("Testing class in jar: " + testClassName); Class.forName(testClassName, true, cl); } - LOG.info("Loaded jar into current JVM: " + urlPath); + LOG.debug("Loaded jar into current JVM: " + urlPath); } catch (ClassNotFoundException cnfe) { throw new IOException("Could not load jar " + jarFile + " into JVM. (Could not find class " + testClassName + ".)", cnfe); } - LOG.info("Added classloader for jar " + jarFile + ": " + cl); + LOG.debug("Added classloader for jar " + jarFile + ": " + cl); Thread.currentThread().setContextClassLoader(cl); return prevClassLoader; } diff --git a/src/java/org/apache/hadoop/sqoop/util/ResultSetPrinter.java b/src/java/org/apache/hadoop/sqoop/util/ResultSetPrinter.java index c5c11ac4..ba2ac70e 100644 --- a/src/java/org/apache/hadoop/sqoop/util/ResultSetPrinter.java +++ b/src/java/org/apache/hadoop/sqoop/util/ResultSetPrinter.java @@ -19,20 +19,17 @@ package org.apache.hadoop.sqoop.util; import java.io.IOException; -import java.io.OutputStream; -import java.nio.ByteBuffer; -import java.nio.CharBuffer; +import java.io.PrintWriter; import java.sql.ResultSet; import java.sql.ResultSetMetaData; import java.sql.SQLException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.util.StringUtils; /** * Utility methods to format and print ResultSet objects - * - * */ public class ResultSetPrinter { @@ -41,21 +38,6 @@ public class ResultSetPrinter { // max output width to allocate to any column of the printed results. private static final int MAX_COL_WIDTH = 20; - // length of the byte buffer, in bytes, to allocate. - private static final int BUFFER_LEN = 4096; - - // maximum number of characters to deserialize from the stringbuilder - // into the byte buffer at a time. Factor of 2 off b/c of Unicode. - private static final int MAX_CHARS = 2048; - - private ByteBuffer bytebuf; - private char [] charArray; - - public ResultSetPrinter() { - bytebuf = ByteBuffer.allocate(BUFFER_LEN); - charArray = new char[MAX_CHARS]; - } - /** * Print 'str' to the string builder, padded to 'width' chars */ @@ -74,35 +56,14 @@ private static void printPadded(StringBuilder sb, String str, int width) { } } - - /** - * Takes the contents of the StringBuilder and prints it on the OutputStream - */ - private void sendToStream(StringBuilder sb, OutputStream os) throws IOException { - - int pos = 0; // current pos in the string builder - int len = sb.length(); // total length (in characters) to send to os. - CharBuffer charbuf = bytebuf.asCharBuffer(); - - while (pos < len) { - int copyLen = Math.min(sb.length(), MAX_CHARS); - sb.getChars(pos, copyLen, charArray, 0); - - charbuf.put(charArray, 0, copyLen); - os.write(bytebuf.array()); - - pos += copyLen; - } - - } - private static final String COL_SEPARATOR = " | "; /** * Format the contents of the ResultSet into something that could be printed * neatly; the results are appended to the supplied StringBuilder. */ - public final void printResultSet(OutputStream os, ResultSet results) throws IOException { + public final void printResultSet(PrintWriter pw, ResultSet results) + throws IOException { try { StringBuilder sbNames = new StringBuilder(); int cols = results.getMetaData().getColumnCount(); @@ -128,9 +89,9 @@ public final void printResultSet(OutputStream os, ResultSet results) throws IOEx } sbPad.append('\n'); - sendToStream(sbPad, os); - sendToStream(sbNames, os); - sendToStream(sbPad, os); + pw.print(sbPad.toString()); + pw.print(sbNames.toString()); + pw.print(sbPad.toString()); while (results.next()) { StringBuilder sb = new StringBuilder(); @@ -139,12 +100,13 @@ public final void printResultSet(OutputStream os, ResultSet results) throws IOEx sb.append(COL_SEPARATOR); } sb.append('\n'); - sendToStream(sb, os); + pw.print(sb.toString()); } - sendToStream(sbPad, os); + pw.print(sbPad.toString()); } catch (SQLException sqlException) { - LOG.error("Error reading from database: " + sqlException.toString()); + LOG.error("Error reading from database: " + + StringUtils.stringifyException(sqlException)); } } diff --git a/src/scripts/create-tool-scripts.sh b/src/scripts/create-tool-scripts.sh new file mode 100755 index 00000000..3a175004 --- /dev/null +++ b/src/scripts/create-tool-scripts.sh @@ -0,0 +1,63 @@ +#!/bin/sh +# +# Licensed to Cloudera, Inc. under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# Cloudera, Inc. 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. + +# Arguments to this script: +# The output directory for the result scripts +# The path to the script template file. +# This script should be run in the base directory of the project. +# It will run 'bin/sqoop help' and determine the names of all subprograms, +# and then generate wrapper scripts for each of these. + +bin=`dirname $0` +bin=`cd ${bin} && pwd` + +outdir=$1 +template=$2 + +if [ -z "$outdir" ]; then + echo "Output directory is required as the first argument." + exit 1 +fi + +if [ -z "$template" ]; then + echo "The script template filename is required as the second argument." + exit 1 +fi + +if [ ! -d "$outdir" ]; then + echo "Output directory does not exist!" + exit 1 +fi + +if [ ! -f "$template" ]; then + echo "Could not find template file: " $template + exit 1 +fi + +# Query 'sqoop help' for the list of available commands. +subprograms=`bin/sqoop help | awk ' BEGIN {use=0} /^$/ {use=0} \ + { if (use) { print $1 } else { } } /^Available/ {use=1} '` + +# For each of these, copy the template into place and sed the +# subcommand's name in. +for p in $subprograms; do + target="${outdir}/sqoop-${p}" + cp "${template}" "${target}" + chmod +x "${target}" + sed -i -e "s/SUBCOMMANDMARKER/$p/" "${target}" +done + diff --git a/src/scripts/tool-script.sh.template b/src/scripts/tool-script.sh.template new file mode 100644 index 00000000..4de96f3d --- /dev/null +++ b/src/scripts/tool-script.sh.template @@ -0,0 +1,22 @@ +#!/bin/sh +# +# Licensed to Cloudera, Inc. under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# Cloudera, Inc. 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. + +prgm=`readlink -f $0` +bin=`dirname ${prgm}` +bin=`cd ${bin} && pwd` + +exec ${bin}/sqoop SUBCOMMANDMARKER "$@" diff --git a/src/shims/common/org/apache/hadoop/sqoop/shims/CommonHadoopShim.java b/src/shims/common/org/apache/hadoop/sqoop/shims/CommonHadoopShim.java index 0a6e1616..9969066c 100644 --- a/src/shims/common/org/apache/hadoop/sqoop/shims/CommonHadoopShim.java +++ b/src/shims/common/org/apache/hadoop/sqoop/shims/CommonHadoopShim.java @@ -18,7 +18,11 @@ package org.apache.hadoop.sqoop.shims; +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.lib.db.DBConfiguration; +import org.apache.hadoop.util.GenericOptionsParser; /** * Contains code which belongs in all Hadoop shims which is syntactically @@ -27,6 +31,17 @@ */ public abstract class CommonHadoopShim extends HadoopShim { + @Override + public String [] parseGenericOptions(Configuration conf, String [] args) + throws IOException { + // This needs to be shimmed because in Apache Hadoop this can throw + // an IOException, but it does not do so in CDH. We just mandate in + // this method that an IOException is possible. + GenericOptionsParser genericParser = new GenericOptionsParser( + conf, args); + return genericParser.getRemainingArgs(); + } + @Override public String getDbInputClassProperty() { return DBConfiguration.INPUT_CLASS_PROPERTY; diff --git a/src/test/org/apache/hadoop/sqoop/TestAllTables.java b/src/test/org/apache/hadoop/sqoop/TestAllTables.java index 591da515..75b7002e 100644 --- a/src/test/org/apache/hadoop/sqoop/TestAllTables.java +++ b/src/test/org/apache/hadoop/sqoop/TestAllTables.java @@ -34,6 +34,7 @@ import org.apache.hadoop.sqoop.testutil.CommonArgs; import org.apache.hadoop.sqoop.testutil.HsqldbTestServer; import org.apache.hadoop.sqoop.testutil.ImportJobTestCase; +import org.apache.hadoop.sqoop.tool.ImportAllTablesTool; /** * Test the --all-tables functionality that can import multiple tables. @@ -51,7 +52,6 @@ public class TestAllTables extends ImportJobTestCase { CommonArgs.addHadoopFlags(args); } - args.add("--all-tables"); args.add("--warehouse-dir"); args.add(getWarehouseDir()); args.add("--connect"); @@ -108,7 +108,7 @@ public void setUp() { public void testMultiTableImport() throws IOException { String [] argv = getArgv(true); - runImport(argv); + runImport(new ImportAllTablesTool(), argv); Path warehousePath = new Path(this.getWarehouseDir()); int i = 0; diff --git a/src/test/org/apache/hadoop/sqoop/TestExport.java b/src/test/org/apache/hadoop/sqoop/TestExport.java index 69a9caf5..76e0edf7 100644 --- a/src/test/org/apache/hadoop/sqoop/TestExport.java +++ b/src/test/org/apache/hadoop/sqoop/TestExport.java @@ -46,6 +46,7 @@ import org.apache.hadoop.sqoop.lib.RecordParser; import org.apache.hadoop.sqoop.lib.SqoopRecord; import org.apache.hadoop.sqoop.testutil.ExportJobTestCase; +import org.apache.hadoop.sqoop.tool.CodeGenTool; import org.apache.hadoop.sqoop.util.ClassLoaderStack; import org.junit.Before; @@ -70,6 +71,31 @@ public void setUp() { } } + /** + * @return an argv for the CodeGenTool to use when creating tables to export. + */ + protected String [] getCodeGenArgv(String... extraArgs) { + List codeGenArgv = new ArrayList(); + + if (null != extraArgs) { + for (String arg : extraArgs) { + codeGenArgv.add(arg); + } + } + + codeGenArgv.add("--table"); + codeGenArgv.add(getTableName()); + codeGenArgv.add("--connect"); + codeGenArgv.add(getConnectString()); + codeGenArgv.add("--fields-terminated-by"); + codeGenArgv.add("\\t"); + codeGenArgv.add("--lines-terminated-by"); + codeGenArgv.add("\\n"); + + return codeGenArgv.toArray(new String[0]); + } + + private String getRecordLine(int recordNum, ColumnGenerator... extraCols) { String idStr = Integer.toString(recordNum); StringBuilder sb = new StringBuilder(); @@ -429,9 +455,8 @@ public void testMultiMapTextExport() throws IOException, SQLException { verifyExport(RECORDS_PER_MAP * NUM_FILES); } - /** Export some rows from a SequenceFile, make sure they import correctly */ - public void testSequenceFileExport() throws IOException, SQLException { + public void testSequenceFileExport() throws Exception { final int TOTAL_RECORDS = 10; @@ -439,7 +464,14 @@ public void testSequenceFileExport() throws IOException, SQLException { LOG.info("Creating initial schema for SeqFile test"); createTable(); LOG.info("Generating code..."); - List generatedJars = runExport(getArgv(true, "--generate-only")); + CodeGenTool codeGen = new CodeGenTool(); + String [] codeGenArgs = getCodeGenArgv(); + SqoopOptions options = codeGen.parseArguments( + codeGenArgs, null, null, true); + codeGen.validateOptions(options); + int ret = codeGen.run(options); + assertEquals(0, ret); + List generatedJars = codeGen.getGeneratedJarFiles(); // Now, wipe the created table so we can export on top of it again. LOG.info("Resetting schema and data..."); diff --git a/src/test/org/apache/hadoop/sqoop/TestMultiMaps.java b/src/test/org/apache/hadoop/sqoop/TestMultiMaps.java index 0b8b06f5..1cd53b22 100644 --- a/src/test/org/apache/hadoop/sqoop/TestMultiMaps.java +++ b/src/test/org/apache/hadoop/sqoop/TestMultiMaps.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.commons.cli.ParseException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -37,6 +38,7 @@ import org.apache.hadoop.sqoop.testutil.HsqldbTestServer; import org.apache.hadoop.sqoop.testutil.ImportJobTestCase; import org.apache.hadoop.sqoop.testutil.SeqFileReader; +import org.apache.hadoop.sqoop.tool.ImportTool; import org.apache.hadoop.sqoop.util.ClassLoaderStack; /** @@ -120,8 +122,10 @@ public void runMultiMapTest(String splitByCol, int expectedSum) String [] argv = getArgv(true, columns, splitByCol); runImport(argv); try { - SqoopOptions opts = new SqoopOptions(); - opts.parse(getArgv(false, columns, splitByCol)); + ImportTool importTool = new ImportTool(); + SqoopOptions opts = importTool.parseArguments( + getArgv(false, columns, splitByCol), + null, null, true); CompilationManager compileMgr = new CompilationManager(opts); String jarFileName = compileMgr.getJarFilename(); @@ -162,6 +166,8 @@ public void runMultiMapTest(String splitByCol, int expectedSum) assertEquals("Total sum of first db column mismatch", expectedSum, curSum); } catch (InvalidOptionsException ioe) { fail(ioe.toString()); + } catch (ParseException pe) { + fail(pe.toString()); } finally { IOUtils.closeStream(reader); diff --git a/src/test/org/apache/hadoop/sqoop/TestSplitBy.java b/src/test/org/apache/hadoop/sqoop/TestSplitBy.java index 241beb27..2e45f9fa 100644 --- a/src/test/org/apache/hadoop/sqoop/TestSplitBy.java +++ b/src/test/org/apache/hadoop/sqoop/TestSplitBy.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.ArrayList; +import org.apache.commons.cli.ParseException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.SequenceFile; @@ -32,6 +33,7 @@ import org.apache.hadoop.sqoop.testutil.HsqldbTestServer; import org.apache.hadoop.sqoop.testutil.ImportJobTestCase; import org.apache.hadoop.sqoop.testutil.SeqFileReader; +import org.apache.hadoop.sqoop.tool.ImportTool; import org.apache.hadoop.sqoop.util.ClassLoaderStack; /** @@ -99,11 +101,13 @@ public void runSplitByTest(String splitByCol, int expectedSum) String [] argv = getArgv(true, columns, splitByCol); runImport(argv); try { - SqoopOptions opts = new SqoopOptions(); - opts.parse(getArgv(false, columns, splitByCol)); + SqoopOptions opts = new ImportTool().parseArguments( + getArgv(false, columns, splitByCol), + null, null, true); CompilationManager compileMgr = new CompilationManager(opts); String jarFileName = compileMgr.getJarFilename(); + LOG.debug("Got jar from import job: " + jarFileName); prevClassLoader = ClassLoaderStack.addJarFile(jarFileName, getTableName()); @@ -130,6 +134,8 @@ public void runSplitByTest(String splitByCol, int expectedSum) assertEquals("Total sum of first db column mismatch", expectedSum, curSum); } catch (InvalidOptionsException ioe) { fail(ioe.toString()); + } catch (ParseException pe) { + fail(pe.toString()); } finally { IOUtils.closeStream(reader); diff --git a/src/test/org/apache/hadoop/sqoop/TestSqoopOptions.java b/src/test/org/apache/hadoop/sqoop/TestSqoopOptions.java index 546dce19..3ebb43cc 100644 --- a/src/test/org/apache/hadoop/sqoop/TestSqoopOptions.java +++ b/src/test/org/apache/hadoop/sqoop/TestSqoopOptions.java @@ -20,6 +20,8 @@ import junit.framework.TestCase; +import org.apache.hadoop.sqoop.tool.ImportTool; + /** * Test aspects of the SqoopOptions class @@ -27,11 +29,11 @@ public class TestSqoopOptions extends TestCase { // tests for the toChar() parser - public void testNormalChar() throws SqoopOptions.InvalidOptionsException { + public void testNormalChar() throws Exception { assertEquals('a', SqoopOptions.toChar("a")); } - public void testEmptyString() throws SqoopOptions.InvalidOptionsException { + public void testEmptyString() throws Exception { try { SqoopOptions.toChar(""); fail("Expected exception"); @@ -40,7 +42,7 @@ public void testEmptyString() throws SqoopOptions.InvalidOptionsException { } } - public void testNullString() throws SqoopOptions.InvalidOptionsException { + public void testNullString() throws Exception { try { SqoopOptions.toChar(null); fail("Expected exception"); @@ -49,40 +51,46 @@ public void testNullString() throws SqoopOptions.InvalidOptionsException { } } - public void testTooLong() throws SqoopOptions.InvalidOptionsException { + public void testTooLong() throws Exception { // Should just use the first character and log a warning. assertEquals('x', SqoopOptions.toChar("xyz")); } - public void testHexChar1() throws SqoopOptions.InvalidOptionsException { + public void testHexChar1() throws Exception { assertEquals(0xF, SqoopOptions.toChar("\\0xf")); } - public void testHexChar2() throws SqoopOptions.InvalidOptionsException { + public void testHexChar2() throws Exception { assertEquals(0xF, SqoopOptions.toChar("\\0xF")); } - public void testHexChar3() throws SqoopOptions.InvalidOptionsException { + public void testHexChar3() throws Exception { assertEquals(0xF0, SqoopOptions.toChar("\\0xf0")); } - public void testHexChar4() throws SqoopOptions.InvalidOptionsException { + public void testHexChar4() throws Exception { assertEquals(0xF0, SqoopOptions.toChar("\\0Xf0")); } - public void testEscapeChar1() throws SqoopOptions.InvalidOptionsException { + public void testEscapeChar1() throws Exception { assertEquals('\n', SqoopOptions.toChar("\\n")); } - public void testEscapeChar2() throws SqoopOptions.InvalidOptionsException { + public void testEscapeChar2() throws Exception { assertEquals('\\', SqoopOptions.toChar("\\\\")); } - public void testEscapeChar3() throws SqoopOptions.InvalidOptionsException { + public void testEscapeChar3() throws Exception { assertEquals('\\', SqoopOptions.toChar("\\")); } - public void testUnknownEscape1() throws SqoopOptions.InvalidOptionsException { + public void testWhitespaceToChar() throws Exception { + assertEquals(' ', SqoopOptions.toChar(" ")); + assertEquals(' ', SqoopOptions.toChar(" ")); + assertEquals('\t', SqoopOptions.toChar("\t")); + } + + public void testUnknownEscape1() throws Exception { try { SqoopOptions.toChar("\\Q"); fail("Expected exception"); @@ -91,7 +99,7 @@ public void testUnknownEscape1() throws SqoopOptions.InvalidOptionsException { } } - public void testUnknownEscape2() throws SqoopOptions.InvalidOptionsException { + public void testUnknownEscape2() throws Exception { try { SqoopOptions.toChar("\\nn"); fail("Expected exception"); @@ -100,31 +108,31 @@ public void testUnknownEscape2() throws SqoopOptions.InvalidOptionsException { } } - public void testEscapeNul1() throws SqoopOptions.InvalidOptionsException { + public void testEscapeNul1() throws Exception { assertEquals('\000', SqoopOptions.toChar("\\0")); } - public void testEscapeNul2() throws SqoopOptions.InvalidOptionsException { + public void testEscapeNul2() throws Exception { assertEquals('\000', SqoopOptions.toChar("\\00")); } - public void testEscapeNul3() throws SqoopOptions.InvalidOptionsException { + public void testEscapeNul3() throws Exception { assertEquals('\000', SqoopOptions.toChar("\\0000")); } - public void testEscapeNul4() throws SqoopOptions.InvalidOptionsException { + public void testEscapeNul4() throws Exception { assertEquals('\000', SqoopOptions.toChar("\\0x0")); } - public void testOctalChar1() throws SqoopOptions.InvalidOptionsException { + public void testOctalChar1() throws Exception { assertEquals(04, SqoopOptions.toChar("\\04")); } - public void testOctalChar2() throws SqoopOptions.InvalidOptionsException { + public void testOctalChar2() throws Exception { assertEquals(045, SqoopOptions.toChar("\\045")); } - public void testErrOctalChar() throws SqoopOptions.InvalidOptionsException { + public void testErrOctalChar() throws Exception { try { SqoopOptions.toChar("\\095"); fail("Expected exception"); @@ -133,7 +141,7 @@ public void testErrOctalChar() throws SqoopOptions.InvalidOptionsException { } } - public void testErrHexChar() throws SqoopOptions.InvalidOptionsException { + public void testErrHexChar() throws Exception { try { SqoopOptions.toChar("\\0x9K5"); fail("Expected exception"); @@ -142,21 +150,25 @@ public void testErrHexChar() throws SqoopOptions.InvalidOptionsException { } } + private SqoopOptions parse(String [] argv) throws Exception { + ImportTool importTool = new ImportTool(); + return importTool.parseArguments(argv, null, null, false); + } + // test that setting output delimiters also sets input delimiters - public void testDelimitersInherit() throws SqoopOptions.InvalidOptionsException { + public void testDelimitersInherit() throws Exception { String [] args = { "--fields-terminated-by", "|" }; - SqoopOptions opts = new SqoopOptions(); - opts.parse(args); + SqoopOptions opts = parse(args); assertEquals('|', opts.getInputFieldDelim()); assertEquals('|', opts.getOutputFieldDelim()); } // test that setting output delimiters and setting input delims separately works - public void testDelimOverride1() throws SqoopOptions.InvalidOptionsException { + public void testDelimOverride1() throws Exception { String [] args = { "--fields-terminated-by", "|", @@ -164,14 +176,13 @@ public void testDelimOverride1() throws SqoopOptions.InvalidOptionsException { "*" }; - SqoopOptions opts = new SqoopOptions(); - opts.parse(args); + SqoopOptions opts = parse(args); assertEquals('*', opts.getInputFieldDelim()); assertEquals('|', opts.getOutputFieldDelim()); } // test that the order in which delims are specified doesn't matter - public void testDelimOverride2() throws SqoopOptions.InvalidOptionsException { + public void testDelimOverride2() throws Exception { String [] args = { "--input-fields-terminated-by", "*", @@ -179,50 +190,46 @@ public void testDelimOverride2() throws SqoopOptions.InvalidOptionsException { "|" }; - SqoopOptions opts = new SqoopOptions(); - opts.parse(args); + SqoopOptions opts = parse(args); assertEquals('*', opts.getInputFieldDelim()); assertEquals('|', opts.getOutputFieldDelim()); } - public void testBadNumMappers1() { + public void testBadNumMappers1() throws Exception { String [] args = { "--num-mappers", "x" }; try { - SqoopOptions opts = new SqoopOptions(); - opts.parse(args); + SqoopOptions opts = parse(args); fail("Expected InvalidOptionsException"); } catch (SqoopOptions.InvalidOptionsException ioe) { // expected. } } - public void testBadNumMappers2() { + public void testBadNumMappers2() throws Exception { String [] args = { "-m", "x" }; try { - SqoopOptions opts = new SqoopOptions(); - opts.parse(args); + SqoopOptions opts = parse(args); fail("Expected InvalidOptionsException"); } catch (SqoopOptions.InvalidOptionsException ioe) { // expected. } } - public void testGoodNumMappers() throws SqoopOptions.InvalidOptionsException { + public void testGoodNumMappers() throws Exception { String [] args = { "-m", "4" }; - SqoopOptions opts = new SqoopOptions(); - opts.parse(args); + SqoopOptions opts = parse(args); assertEquals(4, opts.getNumMappers()); } } diff --git a/src/test/org/apache/hadoop/sqoop/TestWhere.java b/src/test/org/apache/hadoop/sqoop/TestWhere.java index 079facf4..7f080ed6 100644 --- a/src/test/org/apache/hadoop/sqoop/TestWhere.java +++ b/src/test/org/apache/hadoop/sqoop/TestWhere.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.ArrayList; +import org.apache.commons.cli.ParseException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.SequenceFile; @@ -32,6 +33,7 @@ import org.apache.hadoop.sqoop.testutil.HsqldbTestServer; import org.apache.hadoop.sqoop.testutil.ImportJobTestCase; import org.apache.hadoop.sqoop.testutil.SeqFileReader; +import org.apache.hadoop.sqoop.tool.ImportTool; import org.apache.hadoop.sqoop.util.ClassLoaderStack; /** @@ -104,8 +106,9 @@ public void runWhereTest(String whereClause, String firstValStr, int numExpected String [] argv = getArgv(true, columns, whereClause); runImport(argv); try { - SqoopOptions opts = new SqoopOptions(); - opts.parse(getArgv(false, columns, whereClause)); + SqoopOptions opts = new ImportTool().parseArguments( + getArgv(false, columns, whereClause), + null, null, true); CompilationManager compileMgr = new CompilationManager(opts); String jarFileName = compileMgr.getJarFilename(); @@ -146,6 +149,8 @@ public void runWhereTest(String whereClause, String firstValStr, int numExpected assertEquals("Incorrect number of results for query", numExpectedResults, totalResults); } catch (InvalidOptionsException ioe) { fail(ioe.toString()); + } catch (ParseException pe) { + fail(pe.toString()); } finally { IOUtils.closeStream(reader); diff --git a/src/test/org/apache/hadoop/sqoop/hive/TestHiveImport.java b/src/test/org/apache/hadoop/sqoop/hive/TestHiveImport.java index 5a850b35..58c348d9 100644 --- a/src/test/org/apache/hadoop/sqoop/hive/TestHiveImport.java +++ b/src/test/org/apache/hadoop/sqoop/hive/TestHiveImport.java @@ -32,6 +32,10 @@ import org.apache.hadoop.sqoop.testutil.CommonArgs; import org.apache.hadoop.sqoop.testutil.HsqldbTestServer; import org.apache.hadoop.sqoop.testutil.ImportJobTestCase; +import org.apache.hadoop.sqoop.tool.CodeGenTool; +import org.apache.hadoop.sqoop.tool.CreateHiveTableTool; +import org.apache.hadoop.sqoop.tool.ImportTool; +import org.apache.hadoop.sqoop.tool.SqoopTool; /** * Test HiveImport capability after an import to HDFS. @@ -40,6 +44,21 @@ public class TestHiveImport extends ImportJobTestCase { public static final Log LOG = LogFactory.getLog(TestHiveImport.class.getName()); + /** + * Sets the expected number of columns in the table being manipulated + * by the test. Under the hood, this sets the expected column names + * to DATA_COLi for 0 <= i < numCols. + * @param numCols the number of columns to be created. + */ + private void setNumCols(int numCols) { + String [] cols = new String[numCols]; + for (int i = 0; i < numCols; i++) { + cols[i] = "DATA_COL" + i; + } + + setColNames(cols); + } + /** * Create the argv to pass to Sqoop * @return the argv as an array of strings. @@ -51,6 +70,12 @@ public class TestHiveImport extends ImportJobTestCase { CommonArgs.addHadoopFlags(args); } + if (null != moreArgs) { + for (String arg: moreArgs) { + args.add(arg); + } + } + args.add("--table"); args.add(getTableName()); args.add("--warehouse-dir"); @@ -62,12 +87,48 @@ public class TestHiveImport extends ImportJobTestCase { if (null != colNames) { args.add("--split-by"); args.add(colNames[0]); + } else { + fail("Could not determine column names."); } + args.add("--num-mappers"); args.add("1"); - if (null != moreArgs) { - for (String arg: moreArgs) { + for (String a : args) { + LOG.debug("ARG : "+ a); + } + + return args.toArray(new String[0]); + } + + /** + * @return the argv to supply to a code-gen only job for Hive imports. + */ + protected String [] getCodeGenArgs() { + ArrayList args = new ArrayList(); + + args.add("--table"); + args.add(getTableName()); + args.add("--connect"); + args.add(HsqldbTestServer.getUrl()); + args.add("--hive-import"); + + return args.toArray(new String[0]); + } + + /** + * @return the argv to supply to a ddl-executing-only job for Hive imports. + */ + protected String [] getCreateHiveTableArgs(String [] extraArgs) { + ArrayList args = new ArrayList(); + + args.add("--table"); + args.add(getTableName()); + args.add("--connect"); + args.add(HsqldbTestServer.getUrl()); + + if (null != extraArgs) { + for (String arg : extraArgs) { args.add(arg); } } @@ -75,44 +136,46 @@ public class TestHiveImport extends ImportJobTestCase { return args.toArray(new String[0]); } - private SqoopOptions getSqoopOptions(String [] extraArgs) { - SqoopOptions opts = new SqoopOptions(); + private SqoopOptions getSqoopOptions(String [] args, SqoopTool tool) { + SqoopOptions opts = null; try { - opts.parse(getArgv(false, extraArgs)); - } catch (SqoopOptions.InvalidOptionsException ioe) { - fail("Invalid options: " + ioe.toString()); + opts = tool.parseArguments(args, null, null, true); + } catch (Exception e) { + fail("Invalid options: " + e.toString()); } return opts; } private void runImportTest(String tableName, String [] types, String [] values, - String verificationScript, String [] extraArgs) throws IOException { + String verificationScript, String [] args, SqoopTool tool) throws IOException { // create a table and populate it with a row... - setCurTableName(tableName); createTableWithColTypes(types, values); // set up our mock hive shell to compare our generated script // against the correct expected one. - SqoopOptions options = getSqoopOptions(extraArgs); + SqoopOptions options = getSqoopOptions(args, tool); String hiveHome = options.getHiveHome(); assertNotNull("hive.home was not set", hiveHome); Path testDataPath = new Path(new Path(hiveHome), "scripts/" + verificationScript); System.setProperty("expected.script", testDataPath.toString()); // verify that we can import it correctly into hive. - runImport(getArgv(true, extraArgs)); + runImport(tool, args); } /** Test that we can generate a file containing the DDL and not import. */ @Test public void testGenerateOnly() throws IOException { final String TABLE_NAME = "GenerateOnly"; - String [] extraArgs = { "--generate-only" }; + setCurTableName(TABLE_NAME); + setNumCols(1); // Figure out where our target generated .q file is going to be. - SqoopOptions options = getSqoopOptions(extraArgs); + String [] emptyArgs = new String[0]; + SqoopOptions options = getSqoopOptions(getArgv(false, null), + new ImportTool()); Path ddlFile = new Path(new Path(options.getCodeOutputDir()), TABLE_NAME + ".q"); FileSystem fs = FileSystem.getLocal(new Configuration()); @@ -128,7 +191,8 @@ public void testGenerateOnly() throws IOException { // Run a basic import, but specify that we're just generating definitions. String [] types = { "INTEGER" }; String [] vals = { "42" }; - runImportTest(TABLE_NAME, types, vals, null, extraArgs); + runImportTest(TABLE_NAME, types, vals, null, getCodeGenArgs(), + new CodeGenTool()); // Test that the generated definition file exists. assertTrue("Couldn't find expected ddl file", fs.exists(ddlFile)); @@ -142,43 +206,64 @@ public void testGenerateOnly() throws IOException { /** Test that strings and ints are handled in the normal fashion */ @Test public void testNormalHiveImport() throws IOException { + final String TABLE_NAME = "NORMAL_HIVE_IMPORT"; + setCurTableName(TABLE_NAME); + setNumCols(3); String [] types = { "VARCHAR(32)", "INTEGER", "CHAR(64)" }; String [] vals = { "'test'", "42", "'somestring'" }; - runImportTest("NORMAL_HIVE_IMPORT", types, vals, "normalImport.q", null); + runImportTest(TABLE_NAME, types, vals, "normalImport.q", + getArgv(false, null), new ImportTool()); } /** Test that table is created in hive with no data import */ @Test public void testCreateOnlyHiveImport() throws IOException { + final String TABLE_NAME = "CREATE_ONLY_HIVE_IMPORT"; + setCurTableName(TABLE_NAME); + setNumCols(3); String [] types = { "VARCHAR(32)", "INTEGER", "CHAR(64)" }; String [] vals = { "'test'", "42", "'somestring'" }; - String [] extraArgs = {"--hive-create-only"}; - runImportTest("CREATE_ONLY_HIVE_IMPORT", types, vals, "createOnlyImport.q", extraArgs); + runImportTest(TABLE_NAME, types, vals, + "createOnlyImport.q", getCreateHiveTableArgs(null), + new CreateHiveTableTool()); } /** Test that table is created in hive and replaces the existing table if any */ @Test public void testCreateOverwriteHiveImport() throws IOException { + final String TABLE_NAME = "CREATE_OVERWRITE_HIVE_IMPORT"; + setCurTableName(TABLE_NAME); + setNumCols(3); String [] types = { "VARCHAR(32)", "INTEGER", "CHAR(64)" }; String [] vals = { "'test'", "42", "'somestring'" }; - String [] extraArgs = {"--hive-create-only", "--hive-overwrite"}; - runImportTest("CREATE_OVERWRITE_HIVE_IMPORT", types, vals, "createOverwriteImport.q", extraArgs); + String [] extraArgs = {"--hive-overwrite"}; + runImportTest(TABLE_NAME, types, vals, + "createOverwriteImport.q", getCreateHiveTableArgs(extraArgs), + new CreateHiveTableTool()); } /** Test that dates are coerced properly to strings */ @Test public void testDate() throws IOException { + final String TABLE_NAME = "DATE_HIVE_IMPORT"; + setCurTableName(TABLE_NAME); + setNumCols(2); String [] types = { "VARCHAR(32)", "DATE" }; String [] vals = { "'test'", "'2009-05-12'" }; - runImportTest("DATE_HIVE_IMPORT", types, vals, "dateImport.q", null); + runImportTest(TABLE_NAME, types, vals, "dateImport.q", + getArgv(false, null), new ImportTool()); } /** Test that NUMERICs are coerced to doubles */ @Test public void testNumeric() throws IOException { + final String TABLE_NAME = "NUMERIC_HIVE_IMPORT"; + setCurTableName(TABLE_NAME); + setNumCols(2); String [] types = { "NUMERIC", "CHAR(64)" }; String [] vals = { "3.14159", "'foo'" }; - runImportTest("NUMERIC_HIVE_IMPORT", types, vals, "numericImport.q", null); + runImportTest(TABLE_NAME, types, vals, "numericImport.q", + getArgv(false, null), new ImportTool()); } /** If bin/hive returns an error exit status, we should get an IOException */ @@ -186,10 +271,14 @@ public void testNumeric() throws IOException { public void testHiveExitFails() { // The expected script is different than the one which would be generated // by this, so we expect an IOException out. + final String TABLE_NAME = "FAILING_HIVE_IMPORT"; + setCurTableName(TABLE_NAME); + setNumCols(2); String [] types = { "NUMERIC", "CHAR(64)" }; String [] vals = { "3.14159", "'foo'" }; try { - runImportTest("FAILING_HIVE_IMPORT", types, vals, "failingImport.q", null); + runImportTest(TABLE_NAME, types, vals, "failingImport.q", + getArgv(false, null), new ImportTool()); // If we get here, then the run succeeded -- which is incorrect. fail("FAILING_HIVE_IMPORT test should have thrown IOException"); } catch (IOException ioe) { @@ -200,11 +289,15 @@ public void testHiveExitFails() { /** Test that we can set delimiters how we want them */ @Test public void testCustomDelimiters() throws IOException { + final String TABLE_NAME = "CUSTOM_DELIM_IMPORT"; + setCurTableName(TABLE_NAME); + setNumCols(3); String [] types = { "VARCHAR(32)", "INTEGER", "CHAR(64)" }; String [] vals = { "'test'", "42", "'somestring'" }; - String [] extraArgs = { "--fields-terminated-by", ",", "--lines-terminated-by", "|" }; - runImportTest("CUSTOM_DELIM_IMPORT", types, vals, "customDelimImport.q", extraArgs); + String [] extraArgs = { "--fields-terminated-by", ",", + "--lines-terminated-by", "|" }; + runImportTest(TABLE_NAME, types, vals, "customDelimImport.q", + getArgv(false, extraArgs), new ImportTool()); } - } diff --git a/src/test/org/apache/hadoop/sqoop/manager/DirectMySQLExportTest.java b/src/test/org/apache/hadoop/sqoop/manager/DirectMySQLExportTest.java index d285a25a..ad976c45 100644 --- a/src/test/org/apache/hadoop/sqoop/manager/DirectMySQLExportTest.java +++ b/src/test/org/apache/hadoop/sqoop/manager/DirectMySQLExportTest.java @@ -135,6 +135,22 @@ public void tearDown() { } } + @Override + protected String [] getCodeGenArgv(String... extraArgs) { + + String [] moreArgs = new String[extraArgs.length + 2]; + int i = 0; + for (i = 0; i < extraArgs.length; i++) { + moreArgs[i] = extraArgs[i]; + } + + // Add username argument for mysql. + moreArgs[i++] = "--username"; + moreArgs[i++] = MySQLTestUtils.getCurrentUser(); + + return super.getCodeGenArgv(moreArgs); + } + @Override protected String [] getArgv(boolean includeHadoopFlags, String... additionalArgv) { diff --git a/src/test/org/apache/hadoop/sqoop/manager/DirectMySQLTest.java b/src/test/org/apache/hadoop/sqoop/manager/DirectMySQLTest.java index 49973dae..b94a9b46 100644 --- a/src/test/org/apache/hadoop/sqoop/manager/DirectMySQLTest.java +++ b/src/test/org/apache/hadoop/sqoop/manager/DirectMySQLTest.java @@ -240,7 +240,7 @@ public void testWithExtraParams() throws IOException { "3,Fred,2009-01-23,15,marketing" }; - String [] extraArgs = { "-", "--lock-tables" }; + String [] extraArgs = { "--", "--lock-tables" }; doImport(false, true, getTableName(), expectedResults, extraArgs); } diff --git a/src/test/org/apache/hadoop/sqoop/mapreduce/TestImportJob.java b/src/test/org/apache/hadoop/sqoop/mapreduce/TestImportJob.java index d4b3b4c9..fa7e6634 100644 --- a/src/test/org/apache/hadoop/sqoop/mapreduce/TestImportJob.java +++ b/src/test/org/apache/hadoop/sqoop/mapreduce/TestImportJob.java @@ -52,7 +52,7 @@ import org.apache.hadoop.sqoop.testutil.ImportJobTestCase; import org.apache.hadoop.sqoop.testutil.InjectableManagerFactory; import org.apache.hadoop.sqoop.testutil.InjectableConnManager; -import org.apache.hadoop.util.ToolRunner; +import org.apache.hadoop.sqoop.tool.ImportTool; /** * Test aspects of the DataDrivenImportJob class @@ -77,9 +77,9 @@ public void testFailedImportDueToIOException() throws IOException { String [] argv = getArgv(true, new String [] { "DATA_COL0" }, conf); - Sqoop importer = new Sqoop(); + Sqoop importer = new Sqoop(new ImportTool()); try { - ToolRunner.run(importer, argv); + Sqoop.runSqoop(importer, argv); fail("Expected IOException running this job."); } catch (Exception e) { // In debug mode, IOException is wrapped in RuntimeException. @@ -152,9 +152,9 @@ public void testFailedImportDueToJobFail() throws IOException { DummyImportJob.class, ImportJobBase.class); - Sqoop importer = new Sqoop(conf); + Sqoop importer = new Sqoop(new ImportTool(), conf); try { - ToolRunner.run(importer, argv); + Sqoop.runSqoop(importer, argv); fail("Expected ImportException running this job."); } catch (Exception e) { // In debug mode, ImportException is wrapped in RuntimeException. diff --git a/src/test/org/apache/hadoop/sqoop/orm/TestClassWriter.java b/src/test/org/apache/hadoop/sqoop/orm/TestClassWriter.java index 1d2fa72f..301140ef 100644 --- a/src/test/org/apache/hadoop/sqoop/orm/TestClassWriter.java +++ b/src/test/org/apache/hadoop/sqoop/orm/TestClassWriter.java @@ -42,6 +42,7 @@ import org.apache.hadoop.sqoop.testutil.DirUtil; import org.apache.hadoop.sqoop.testutil.HsqldbTestServer; import org.apache.hadoop.sqoop.testutil.ImportJobTestCase; +import org.apache.hadoop.sqoop.tool.ImportTool; /** * Test that the ClassWriter generates Java classes based on the given table, @@ -123,9 +124,10 @@ private void runGenerationTest(String [] argv, String classNameToCheck) { File classGenDirFile = new File(JAR_GEN_DIR); try { - options.parse(argv); - } catch (InvalidOptionsException ioe) { - LOG.error("Could not parse options: " + ioe.toString()); + options = new ImportTool().parseArguments(argv, + null, options, true); + } catch (Exception e) { + LOG.error("Could not parse options: " + e.toString()); } CompilationManager compileMgr = new CompilationManager(options); diff --git a/src/test/org/apache/hadoop/sqoop/orm/TestParseMethods.java b/src/test/org/apache/hadoop/sqoop/orm/TestParseMethods.java index 2e82df1f..02e8a7b9 100644 --- a/src/test/org/apache/hadoop/sqoop/orm/TestParseMethods.java +++ b/src/test/org/apache/hadoop/sqoop/orm/TestParseMethods.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.ArrayList; +import org.apache.commons.cli.ParseException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IOUtils; @@ -41,6 +42,7 @@ import org.apache.hadoop.sqoop.testutil.HsqldbTestServer; import org.apache.hadoop.sqoop.testutil.ImportJobTestCase; import org.apache.hadoop.sqoop.testutil.ReparseMapper; +import org.apache.hadoop.sqoop.tool.ImportTool; import org.apache.hadoop.sqoop.util.ClassLoaderStack; /** @@ -98,12 +100,12 @@ public void runParseTest(String fieldTerminator, String lineTerminator, String e encloseRequired); runImport(argv); try { - SqoopOptions opts = new SqoopOptions(); - String tableClassName = getTableName(); - opts.parse(getArgv(false, fieldTerminator, lineTerminator, encloser, escape, - encloseRequired)); + argv = getArgv(false, fieldTerminator, lineTerminator, encloser, escape, + encloseRequired); + SqoopOptions opts = new ImportTool().parseArguments(argv, null, null, + true); CompilationManager compileMgr = new CompilationManager(opts); String jarFileName = compileMgr.getJarFilename(); @@ -137,6 +139,8 @@ public void runParseTest(String fieldTerminator, String lineTerminator, String e JobClient.runJob(job); } catch (InvalidOptionsException ioe) { fail(ioe.toString()); + } catch (ParseException pe) { + fail(pe.toString()); } finally { if (null != prevClassLoader) { ClassLoaderStack.setCurrentClassLoader(prevClassLoader); diff --git a/src/test/org/apache/hadoop/sqoop/testutil/BaseSqoopTestCase.java b/src/test/org/apache/hadoop/sqoop/testutil/BaseSqoopTestCase.java index 349b71de..ec4778ab 100644 --- a/src/test/org/apache/hadoop/sqoop/testutil/BaseSqoopTestCase.java +++ b/src/test/org/apache/hadoop/sqoop/testutil/BaseSqoopTestCase.java @@ -24,6 +24,7 @@ import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; +import java.util.Arrays; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -105,6 +106,14 @@ protected String getWarehouseDir() { return colNames; } + protected void setColNames(String [] cols) { + if (null == cols) { + this.colNames = null; + } else { + this.colNames = Arrays.copyOf(cols, cols.length); + } + } + protected HsqldbTestServer getTestServer() { return testServer; } diff --git a/src/test/org/apache/hadoop/sqoop/testutil/ExportJobTestCase.java b/src/test/org/apache/hadoop/sqoop/testutil/ExportJobTestCase.java index 9c40077a..6599f1c9 100644 --- a/src/test/org/apache/hadoop/sqoop/testutil/ExportJobTestCase.java +++ b/src/test/org/apache/hadoop/sqoop/testutil/ExportJobTestCase.java @@ -28,9 +28,10 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.util.ToolRunner; +import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.sqoop.Sqoop; +import org.apache.hadoop.sqoop.tool.ExportTool; /** * Class that implements common methods required for tests which export data @@ -82,20 +83,7 @@ protected Connection getConnection() { } } - args.add("--table"); - args.add(getTableName()); - args.add("--export-dir"); - args.add(getTablePath().toString()); - args.add("--connect"); - args.add(getConnectString()); - args.add("--fields-terminated-by"); - args.add("\\t"); - args.add("--lines-terminated-by"); - args.add("\\n"); - args.add("-m"); - args.add("1"); - - // The rest of the additional args are appended. + // The sqoop-specific additional args are then added. if (null != additionalArgv) { boolean prevIsFlag = false; for (String arg : additionalArgv) { @@ -112,6 +100,24 @@ protected Connection getConnection() { } } + args.add("--table"); + args.add(getTableName()); + args.add("--export-dir"); + args.add(getTablePath().toString()); + args.add("--connect"); + args.add(getConnectString()); + args.add("--fields-terminated-by"); + args.add("\\t"); + args.add("--lines-terminated-by"); + args.add("\\n"); + args.add("-m"); + args.add("1"); + + LOG.debug("args:"); + for (String a : args) { + LOG.debug(" " + a); + } + return args.toArray(new String[0]); } @@ -250,12 +256,13 @@ protected List runExport(String [] argv) throws IOException { int ret; List generatedJars = null; try { - Sqoop exporter = new Sqoop(); - ret = ToolRunner.run(exporter, argv); + ExportTool exporter = new ExportTool(); + Sqoop sqoop = new Sqoop(exporter); + ret = Sqoop.runSqoop(sqoop, argv); generatedJars = exporter.getGeneratedJarFiles(); } catch (Exception e) { - LOG.error("Got exception running Sqoop: " + e.toString()); - e.printStackTrace(); + LOG.error("Got exception running Sqoop: " + + StringUtils.stringifyException(e)); ret = 1; } diff --git a/src/test/org/apache/hadoop/sqoop/testutil/ImportJobTestCase.java b/src/test/org/apache/hadoop/sqoop/testutil/ImportJobTestCase.java index ca13fb49..96b56f47 100644 --- a/src/test/org/apache/hadoop/sqoop/testutil/ImportJobTestCase.java +++ b/src/test/org/apache/hadoop/sqoop/testutil/ImportJobTestCase.java @@ -27,11 +27,12 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.sqoop.SqoopOptions; import org.apache.hadoop.sqoop.Sqoop; import org.apache.hadoop.sqoop.SqoopOptions.InvalidOptionsException; import org.apache.hadoop.sqoop.orm.CompilationManager; +import org.apache.hadoop.sqoop.tool.SqoopTool; +import org.apache.hadoop.sqoop.tool.ImportTool; import org.apache.hadoop.sqoop.util.ClassLoaderStack; import org.junit.Test; @@ -119,8 +120,8 @@ protected void verifyImport(String expectedVal, String [] importCols) { // run the tool through the normal entry-point. int ret; try { - Sqoop importer = new Sqoop(conf, opts); - ret = ToolRunner.run(importer, getArgv(true, importCols, conf)); + Sqoop importer = new Sqoop(new ImportTool(), conf, opts); + ret = Sqoop.runSqoop(importer, getArgv(true, importCols, conf)); } catch (Exception e) { LOG.error("Got exception running Sqoop: " + e.toString()); throw new RuntimeException(e); @@ -131,10 +132,13 @@ protected void verifyImport(String expectedVal, String [] importCols) { opts = getSqoopOptions(conf); try { - opts.parse(getArgv(false, importCols, conf)); - } catch (InvalidOptionsException ioe) { - fail(ioe.toString()); + ImportTool importTool = new ImportTool(); + opts = importTool.parseArguments(getArgv(false, importCols, conf), conf, + opts, true); + } catch (Exception e) { + fail(e.toString()); } + CompilationManager compileMgr = new CompilationManager(opts); String jarFileName = compileMgr.getJarFilename(); ClassLoader prevClassLoader = null; @@ -166,7 +170,7 @@ protected void verifyImport(String expectedVal, String [] importCols) { /** * Run a MapReduce-based import (using the argv provided to control execution). */ - protected void runImport(String [] argv) throws IOException { + protected void runImport(SqoopTool tool, String [] argv) throws IOException { removeTableDir(); // run the tool through the normal entry-point. @@ -174,8 +178,8 @@ protected void runImport(String [] argv) throws IOException { try { Configuration conf = getConf(); SqoopOptions opts = getSqoopOptions(conf); - Sqoop importer = new Sqoop(conf, opts); - ret = ToolRunner.run(importer, argv); + Sqoop sqoop = new Sqoop(tool, conf, opts); + ret = Sqoop.runSqoop(sqoop, argv); } catch (Exception e) { LOG.error("Got exception running Sqoop: " + e.toString()); e.printStackTrace(); @@ -188,4 +192,9 @@ protected void runImport(String [] argv) throws IOException { } } + /** run an import using the default ImportTool */ + protected void runImport(String [] argv) throws IOException { + runImport(new ImportTool(), argv); + } + } diff --git a/src/test/org/apache/hadoop/sqoop/testutil/ManagerCompatTestCase.java b/src/test/org/apache/hadoop/sqoop/testutil/ManagerCompatTestCase.java index a9f95cc1..76d22f21 100644 --- a/src/test/org/apache/hadoop/sqoop/testutil/ManagerCompatTestCase.java +++ b/src/test/org/apache/hadoop/sqoop/testutil/ManagerCompatTestCase.java @@ -30,7 +30,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.sqoop.SqoopOptions; import org.apache.hadoop.sqoop.Sqoop;