Prechádzať zdrojové kódy

HADOOP-5815. Sqoop: A database import tool for Hadoop. Contributed by Aaron Kimball.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@778646 13f79535-47bb-0310-9956-ffa450edef68
Thomas White 16 rokov pred
rodič
commit
ae95d95d64
35 zmenil súbory, kde vykonal 5487 pridanie a 0 odobranie
  1. 3 0
      CHANGES.txt
  2. 91 0
      src/contrib/sqoop/build.xml
  3. 60 0
      src/contrib/sqoop/ivy.xml
  4. 21 0
      src/contrib/sqoop/ivy/libraries.properties
  5. 186 0
      src/contrib/sqoop/readme.html
  6. 81 0
      src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/ConnFactory.java
  7. 396 0
      src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/ImportOptions.java
  8. 175 0
      src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/Sqoop.java
  9. 84 0
      src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/lib/BigDecimalSerializer.java
  10. 203 0
      src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/lib/JdbcWritableBridge.java
  11. 106 0
      src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/ConnManager.java
  12. 70 0
      src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/GenericJdbcManager.java
  13. 53 0
      src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/HsqldbManager.java
  14. 68 0
      src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/MySQLManager.java
  15. 371 0
      src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/SqlManager.java
  16. 149 0
      src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapred/ImportJob.java
  17. 52 0
      src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapred/TextImportMapper.java
  18. 550 0
      src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/orm/ClassWriter.java
  19. 313 0
      src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/orm/CompilationManager.java
  20. 84 0
      src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ClassLoaderStack.java
  21. 104 0
      src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/FileListing.java
  22. 44 0
      src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java
  23. 152 0
      src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ResultSetPrinter.java
  24. 52 0
      src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/AllTests.java
  25. 128 0
      src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestAllTables.java
  26. 295 0
      src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestColumnTypes.java
  27. 214 0
      src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestMultiCols.java
  28. 162 0
      src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestOrderBy.java
  29. 83 0
      src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/TestHsqldbManager.java
  30. 220 0
      src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/TestSqlManager.java
  31. 148 0
      src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/orm/TestClassWriter.java
  32. 57 0
      src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/DirUtil.java
  33. 243 0
      src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/HsqldbTestServer.java
  34. 391 0
      src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/ImportJobTestCase.java
  35. 78 0
      src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/SeqFileReader.java

+ 3 - 0
CHANGES.txt

@@ -125,6 +125,9 @@ Trunk (unreleased changes)
     HADOOP-4829. Allow FileSystem shutdown hook to be disabled.
     (Todd Lipcon via tomwhite)
 
+    HADOOP-5815. Sqoop: A database import tool for Hadoop.
+    (Aaron Kimball via tomwhite)
+
   IMPROVEMENTS
 
     HADOOP-4565. Added CombineFileInputFormat to use data locality information

+ 91 - 0
src/contrib/sqoop/build.xml

@@ -0,0 +1,91 @@
+<?xml version="1.0"?>
+
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<!--
+Before you can run these subtargets directly, you need
+to call at top-level: ant deploy-contrib compile-core-test
+-->
+<project name="sqoop" default="jar">
+
+  <import file="../build-contrib.xml"/>
+  <property environment="env"/>
+
+  <!-- ================================================================== -->
+  <!-- Run unit tests                                                     -->
+  <!-- Override with our own version so we can set hadoop.alt.classpath   -->
+  <!-- ================================================================== -->
+  <target name="test" depends="compile-test, compile" if="test.available">
+    <echo message="contrib: ${name}"/>
+    <delete dir="${hadoop.log.dir}"/>
+    <mkdir dir="${hadoop.log.dir}"/>
+    <delete dir="${build.test}/data"/>
+    <mkdir dir="${build.test}/data" />
+    <junit
+      printsummary="yes" showoutput="${test.output}"
+      haltonfailure="no" fork="yes" maxmemory="256m"
+      errorProperty="tests.failed" failureProperty="tests.failed"
+      timeout="${test.timeout}"
+      dir="${build.test}/data">
+
+      <sysproperty key="test.build.data" value="${build.test}/data"/>
+      <sysproperty key="build.test" value="${build.test}"/>
+      <sysproperty key="contrib.name" value="${name}"/>
+
+      <!--
+           Added property needed to use the .class files for compilation
+           instead of depending on hadoop-*-core.jar
+      -->
+      <sysproperty key="hadoop.alt.classpath"
+        value="${hadoop.root}/build/classes" />
+
+      <!-- requires fork=yes for:
+        relative File paths to use the specified user.dir
+        classpath to use build/contrib/*.jar
+      -->
+      <sysproperty key="user.dir" value="${build.test}/data"/>
+
+      <!-- Setting the user.dir property is actually meaningless as it
+          is read-only in the Linux Sun JDK. Provide an alternate sysprop
+          to specify where generated code should go.
+      -->
+      <sysproperty key="sqoop.src.dir" value="${build.test}/data"/>
+
+      <sysproperty key="fs.default.name" value="${fs.default.name}"/>
+      <sysproperty key="hadoop.test.localoutputfile" value="${hadoop.test.localoutputfile}"/>
+      <sysproperty key="hadoop.log.dir" value="${hadoop.log.dir}"/>
+
+      <!-- tools.jar from Sun JDK also required to invoke javac. -->
+      <classpath>
+        <path refid="test.classpath"/>
+        <path refid="contrib-classpath"/>
+        <pathelement path="${env.JAVA_HOME}/lib/tools.jar" />
+      </classpath>
+      <formatter type="${test.junit.output.format}" />
+      <batchtest todir="${build.test}" unless="testcase">
+        <fileset dir="${src.test}"
+                 includes="**/Test*.java" excludes="**/${test.exclude}.java" />
+      </batchtest>
+      <batchtest todir="${build.test}" if="testcase">
+        <fileset dir="${src.test}" includes="**/${testcase}.java"/>
+      </batchtest>
+    </junit>
+    <fail if="tests.failed">Tests failed!</fail>
+  </target>
+
+</project>

+ 60 - 0
src/contrib/sqoop/ivy.xml

@@ -0,0 +1,60 @@
+<?xml version="1.0" ?>
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+  
+       http://www.apache.org/licenses/LICENSE-2.0
+    
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+<ivy-module version="1.0">
+  <info organisation="org.apache.hadoop" module="${ant.project.name}">
+    <license name="Apache 2.0"/>
+    <ivyauthor name="Apache Hadoop Team" url="http://hadoop.apache.org"/>
+    <description>
+        Apache Hadoop
+    </description>
+  </info>
+  <configurations defaultconfmapping="default">
+    <!--these match the Maven configurations-->
+    <conf name="default" extends="master,runtime"/>
+    <conf name="master" description="contains the artifact but no dependencies"/>
+    <conf name="runtime" description="runtime but not the artifact" />
+
+    <conf name="common" visibility="private"
+      extends="runtime"
+      description="artifacts needed to compile/test the application"/>
+    <conf name="test" visibility="private" extends="runtime"/>
+  </configurations>
+
+  <publications>
+    <!--get the artifact from our module name-->
+    <artifact conf="master"/>
+  </publications>
+  <dependencies>
+    <dependency org="commons-logging"
+      name="commons-logging"
+      rev="${commons-logging.version}"
+      conf="common->default"/>
+    <dependency org="commons-httpclient"
+      name="commons-httpclient"
+      rev="${commons-httpclient.version}"
+      conf="common->default"/>
+    <dependency org="junit"
+      name="junit"
+      rev="${junit.version}"
+      conf="common->default"/>
+    <dependency org="log4j"
+      name="log4j"
+      rev="${log4j.version}"
+      conf="common->master"/>
+    </dependencies>
+</ivy-module>

+ 21 - 0
src/contrib/sqoop/ivy/libraries.properties

@@ -0,0 +1,21 @@
+#   Licensed to the Apache Software Foundation (ASF) under one or more
+#   contributor license agreements.  See the NOTICE file distributed with
+#   this work for additional information regarding copyright ownership.
+#   The ASF licenses this file to You under the Apache License, Version 2.0
+#   (the "License"); you may not use this file except in compliance with
+#   the License.  You may obtain a copy of the License at
+#  
+#       http://www.apache.org/licenses/LICENSE-2.0
+#    
+#   Unless required by applicable law or agreed to in writing, software
+#   distributed under the License is distributed on an "AS IS" BASIS,
+#   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#   See the License for the specific language governing permissions and
+#   limitations under the License.
+
+#This properties file lists the versions of the various artifacts used by streaming.
+#It drives ivy and the generation of a maven POM
+
+#Please list the dependencies name with version if they are different from the ones
+#listed in the global libraries.properties file (in alphabetical order)
+

+ 186 - 0
src/contrib/sqoop/readme.html

@@ -0,0 +1,186 @@
+<html>
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+  
+       http://www.apache.org/licenses/LICENSE-2.0
+    
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+<head>
+<title>Sqoop User's Guide</title>
+</head>
+<body>
+    <h1><a name="SqoopUsersGuide-Sqoop"></a>Sqoop</h1>
+
+<h2><a name="SqoopUsersGuide-Overview"></a>Overview</h2>
+
+<p>Sqoop is a tool designed to help users of large data import existing relational databases into their Hadoop clusters. Sqoop uses JDBC to connect to a database, examine the schema for tables, and auto-generate the necessary classes to import data into HDFS. It then instantiates a MapReduce job to read the table from the database via the DBInputFormat (JDBC-based InputFormat). The table is read into a set of files loaded into HDFS. Both SequenceFile and text-based targets are supported.</p>
+
+<p>Longer term, Sqoop will support automatic connectivity to Hive, with the ability to load data files directly into the Hive warehouse directory, and also to inject the appropriate table definition into the metastore.</p>
+
+<h2><a name="SqoopUsersGuide-GettingStarted"></a>Getting Started</h2>
+
+<p><b>Getting Sqoop</b> Sqoop is distributed as a "contrib" jar with Hadoop. It is built in the <tt>contrib/sqoop/</tt> directory.</p>
+
+<p>You can run Sqoop by running:</p>
+
+<div class="code panel" style="border-width: 1px;"><div class="codeContent panelContent">
+<pre class="code-java">
+$ hadoop jar /path/to/sqoop.jar org.apache.hadoop.sqoop.Sqoop (options)
+</pre>
+</div></div>
+
+
+<p>This does nothing of interest without any options. The <tt>&#45;&#45;help</tt> option displays the full usage instructions.</p>
+
+<h3><a name="SqoopUsersGuide-ConnectingtoaDatabaseServer"></a>Connecting to a Database Server</h3>
+
+<p>Sqoop is designed to import tables from a database into HDFS. As such, it requires a <em>connect string</em> that describes how to connect to the database. The <em>connect string</em> looks like a URL, and is communicated to Sqoop with the <tt>&#45;&#45;connect</tt> argument. This describes the server and database to connect to; it may also specify the port. e.g.: </p>
+
+<div class="code panel" style="border-width: 1px;"><div class="codeContent panelContent">
+
+<pre class="code-java">
+$ hadoop jar /path/to/sqoop.jar org.apache.hadoop.sqoop.Sqoop --connect jdbc:mysql:<span class="code-comment">//database.example.com/employees</span>
+</pre>
+</div></div>
+
+<p>This string will connect to a MySQL database named <tt>employees</tt> on the host <tt>database.example.com</tt>. It's important that you <b>do not</b> use the URL <tt>localhost</tt> if you intend to use Sqoop with a distributed Hadoop cluster. The connect string you supply will be used on all the TaskTracker nodes in your MapReduce cluster; if they're told to connect to the literal name <tt>localhost</tt>, they'll each reach a different database (or more likely, no database at all)! Instead, you should use the full DNS or IP address of the database host that can be seen by all your remote nodes.</p>
+
+<p>You may need to authenticate against the database before you can access it. The <tt>&#45;&#45;username</tt> and <tt>&#45;&#45;password</tt> parameters can be used to supply a username and a password to the database. (Note: password access currently requires passing the password on the command-line, which is insecure.) e.g.:</p>
+
+<div class="code panel" style="border-width: 1px;"><div class="codeContent panelContent">
+<pre class="code-java">
+$ hadoop jar /path/to/sqoop.jar org.apache.hadoop.sqoop.Sqoop --connect jdbc:mysql:<span class="code-comment">//database.example.com/employees --username aaron --password 12345</span>
+</pre>
+</div></div>
+
+<p>Sqoop automatically supports <span class="nobr"><a href="http://www.mysql.com" rel="nofollow">MySQL</a></span> and <span class="nobr"><a href="http://hsqldb.org/" rel="nofollow">HSQLDB</a></span>. Connect strings beginning with <tt>jdbc:mysql://</tt> and <tt>jdbc:hsqldb:hsql://</tt> automatically inform Sqoop of the correct JDBC driver class to load. HSQLDB's JDBC driver is bundled with Hadoop, and so will work "out of the box." If you install <a href="http://dev.mysql.com/downloads/connector/j/5.1.html">MySQL's Connector/J driver</a> in Hadoop's <tt>lib/</tt> directory, Sqoop will also automatically take advantage of this for any <tt>jdbc:mysql://</tt> connect strings you use. You can use Sqoop with any other JDBC-compliant database as well. First, download the appropriate JDBC driver for the database you want to import from, and install the <tt>.jar</tt> file in the <tt>$HADOOP_HOME/lib</tt> directory on all machines in your Hadoop cluster, or some other directory which is in the classpath on all nodes. Each driver jar also has a specific <em>driver class</em> which defines the entry-point to the driver. For example, MySQL's Connector/J library has a driver class of <tt>com.mysql.jdbc.Driver</tt>. Refer to your database vendor-specific documentation to determine the main driver class. This class must be provided as an argument to Sqoop with <tt>&#45;&#45;driver</tt>.</p>
+
+<p>For example, to connect to a postgres database, first download the driver from <span class="nobr"><a href="http://jdbc.postgresql.org" rel="nofollow">http://jdbc.postgresql.org</a></span> and install it in your Hadoop lib path. Then run Sqoop with something like:</p>
+
+<div class="code panel" style="border-width: 1px;"><div class="codeContent panelContent">
+<pre class="code-java">
+$ hadoop jar /path/to/sqoop.jar org.apache.hadoop.sqoop.Sqoop --connect jdbc:postgresql:<span class="code-comment">//postgres-server.example.com/employees --driver org.postgresql.Driver</span>
+</pre>
+</div></div>
+
+<p>Note: Sqoop uses the JDBC specification to connect to databases; this should provide a versatile client that interoperates with many different databases. That having been said, we have only thoroughly tested this tool with HSQLDB and MySQL.</p>
+
+<h3><a name="SqoopUsersGuide-ListingAvailableDatabases"></a>Listing Available Databases</h3>
+
+<p>Once connected to a database server, you can list the available databases with the <tt>&#45;&#45;list-databases</tt> parameter. This currently is supported only by HSQLDB and MySQL. Note that in this case, the connect string does not include a database name, just a server address.</p>
+
+<div class="code panel" style="border-width: 1px;"><div class="codeContent panelContent">
+<pre class="code-java">
+$ hadoop jar /path/to/sqoop.jar org.apache.hadoop.sqoop.Sqoop --connect jdbc:mysql:<span class="code-comment">//database.example.com/ --list-databases
+</span>information_schema
+employees
+</pre>
+</div></div>
+
+<p><em>This only works with HSQLDB and MySQL. A vendor-agnostic implementation of this function has not yet been implemented.</em></p>
+
+<h3><a name="SqoopUsersGuide-ListingAvailableTables"></a>Listing Available Tables</h3>
+
+<p>Within a database, you can list the tables available for import with the <tt>&#45;&#45;list-tables</tt> command. The following example shows four tables available within the "employees" example database:</p>
+
+<div class="code panel" style="border-width: 1px;"><div class="codeContent panelContent">
+<pre class="code-java">
+$ hadoop jar /path/to/sqoop.jar org.apache.hadoop.sqoop.Sqoop --connect jdbc:mysql:<span class="code-comment">//database.example.com/employees --list-tables
+
+</span>employee_names
+payroll_checks
+job_descriptions
+office_supplies
+</pre>
+</div></div>
+
+<h2><a name="SqoopUsersGuide-AutomaticFulldatabaseImport"></a>Automatic Full-database Import</h2>
+
+<p>If you want to import all the tables in a database, you can use the <tt>&#45;&#45;all-tables</tt> command to do so:</p>
+
+<div class="code panel" style="border-width: 1px;"><div class="codeContent panelContent">
+<pre class="code-java">
+$ hadoop jar /path/to/sqoop.jar org.apache.hadoop.sqoop.Sqoop --connect jdbc:mysql:<span class="code-comment">//database.example.com/employees --all-tables</span>
+
+</pre>
+</div></div>
+
+<p>This will query the database for the available tables, generate an ORM class for each table, and run a MapReduce job to import each one. Hadoop uses the <span class="nobr"><a href="http://issues.apache.org/jira/browse/HADOOP-2536" rel="nofollow">DBInputFormat</a></span> to read from a database into a Mapper instance. To read a table into a MapReduce program requires creating a class to hold the fields of one row of the table. One of the benefits of Sqoop is that it generates this class definition for you, based on the table definition in the database. </p>
+
+<p>The generated <tt>.java</tt> files are, by default, placed in the current directory. You can supply a different directory with the <tt>&#45;&#45;outdir</tt> parameter. These are then compiled into <tt>.class</tt> and <tt>.jar</tt> files for use by the MapReduce job that it launches. These files are created in a temporary directory. You can redirect this target with <tt>&#45;&#45;bindir</tt>.</p>
+
+<p>Each table will be imported into a separate directory in HDFS, with the same name as the table. For instance, if my Hadoop username is <tt>aaron</tt>, the above command would have generated the following directories in HDFS:</p>
+
+<div class="code panel" style="border-width: 1px;"><div class="codeContent panelContent">
+<pre class="code-java">
+/user/aaron/employee_names
+/user/aaron/payroll_checks
+/user/aaron/job_descriptions
+/user/aaron/office_supplies
+</pre>
+</div></div>
+
+<p>You can change the base directory under which the tables are loaded with the <tt>&#45;&#45;warehouse-dir</tt> parameter. For example:</p>
+
+<div class="code panel" style="border-width: 1px;"><div class="codeContent panelContent">
+<pre class="code-java">
+$ hadoop jar /path/to/sqoop.jar org.apache.hadoop.sqoop.Sqoop --connect jdbc:mysql:<span class="code-comment">//database.example.com/employees --all-tables --warehouse-dir /common/warehouse</span>
+</pre>
+</div></div>
+
+<p>This would create the following directories instead:</p>
+<div class="code panel" style="border-width: 1px;"><div class="codeContent panelContent">
+<pre class="code-java">
+/common/warehouse/employee_names
+/common/warehouse/payroll_checks
+/common/warehouse/job_descriptions
+/common/warehouse/office_supplies
+</pre>
+</div></div>
+
+<p>By default the data will be read into text files in HDFS. Each of the columns will be represented as comma-delimited text. Each row is terminated by a newline. There is currently no mechanism to quote or escape commas or newlines inside of <tt>CHAR</tt> or <tt>VARCHAR</tt> columns of the database. Applications which depend on comma-delimited parsing of the output files must be careful if commas or newlines may be present in the database. </p>
+
+<p>If you expect commas or newlines to appear in text columns of the database, or you want to leverage compression and binary file formats, the <tt>&#45;&#45;as-sequencefile</tt> argument to Sqoop will import the table to a set of SequenceFiles instead. As this uses a separate object for each field of each database record, no quoting or escaping of values is necessary. This representation is also likely to be higher performance when used as an input to subsequent MapReduce programs. For completeness, Sqoop provides an <tt>&#45;&#45;as-textfile</tt> option, which is implied by default. An <tt>&#45;&#45;as-textfile</tt> on the command-line will override a previous <tt>&#45;&#45;as-sequencefile</tt> argument.</p>
+
+<p>The SequenceFile format will embed the records from the database as objects using the code generated by Sqoop. It is important that you retain the <tt>.java file</tt> for this class, as you will need to be able to instantiate the same type to read the objects back later, in other user-defined applications.</p>
+
+<h2><a name="SqoopUsersGuide-ImportingIndividualTables"></a>Importing Individual Tables</h2>
+
+<p>In addition to full-database imports, Sqoop will allow you to import individual tables. Instead of using <tt>&#45;&#45;all-tables</tt>, specify the name of a particular table with the <tt>&#45;&#45;table</tt> argument:</p>
+
+<div class="code panel" style="border-width: 1px;"><div class="codeContent panelContent">
+<pre class="code-java">
+$ hadoop jar /path/to/sqoop.jar org.apache.hadoop.sqoop.Sqoop --connect jdbc:mysql:<span class="code-comment">//database.example.com/employees --table employee_names </span>
+</pre>
+</div></div>
+
+<p>You can further specify a subset of the columns in a table by using the <tt>&#45;&#45;columns</tt> argument. This takes a list of column names, delimited by commas, with no spaces in between. e.g.:</p>
+
+<div class="code panel" style="border-width: 1px;"><div class="codeContent panelContent">
+<pre class="code-java">
+
+$ hadoop jar /path/to/sqoop.jar org.apache.hadoop.sqoop.Sqoop --connect jdbc:mysql:<span class="code-comment">//database.example.com/employees --table employee_names --columns employee_id,first_name,last_name,dept_id</span>
+</pre>
+</div></div>
+
+<p>Sqoop will use a MapReduce job to read sections of the table in parallel. For the MapReduce tasks to divide the table space, the results returned by the database must be orderable. Sqoop will automatically detect the primary key for a table and use that to order the results. If no primary key is available, or (less likely) you want to order the results along a different column, you can specify the column name with <tt>&#45;&#45;order-by</tt>. <b>Important:</b> To guarantee correctness of your input, you must select an ordering column for which each row has a unique value. If duplicate values appear in the ordering column, the results of the import are undefined, and Sqoop will not be able to detect the error.</p>
+
+<p>The <tt>&#45;&#45;columns</tt> and <tt>&#45;&#45;order-by</tt> arguments are incompatible with <tt>&#45;&#45;all-tables</tt>. If you require special handling for some of the tables, then you must manually run a separate import job for each table.</p>
+
+<h2><a name="SqoopUsersGuide-MiscellaneousAdditionalArguments"></a>Miscellaneous Additional Arguments</h2>
+
+<p>If you want to generate the Java classes to represent tables without actually performing an import, supply a connect string and (optionally) credentials as above, as well as <tt>&#45;&#45;all-tables</tt> or <tt>&#45;&#45;table</tt>, but also use the <b><tt>&#45;&#45;generate-only</tt></b> argument. This will generate the classes and cease further operation.</p>
+
+<p>You can override the <tt>$HADOOP_HOME</tt> environment variable within Sqoop with the <tt>&#45;&#45;hadoop-home</tt> argument. </p>
+
+</body></html>

+ 81 - 0
src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/ConnFactory.java

@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop;
+
+import org.apache.hadoop.sqoop.manager.ConnManager;
+import org.apache.hadoop.sqoop.manager.GenericJdbcManager;
+import org.apache.hadoop.sqoop.manager.HsqldbManager;
+import org.apache.hadoop.sqoop.manager.MySQLManager;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Static factory class to create the ConnManager type required
+ * for the current import job.
+ */
+public final class ConnFactory {
+
+  public static final Log LOG = LogFactory.getLog(ConnFactory.class.getName());
+
+  private ConnFactory() { }
+
+  /**
+   * Factory method to get a ConnManager for the given JDBC connect string
+   * @param opts The parsed command-line options
+   * @return a ConnManager instance for the appropriate database
+   * @throws IOException if it cannot find a ConnManager for this schema
+   */
+  public static ConnManager getManager(ImportOptions opts) throws IOException {
+
+    String manualDriver = opts.getDriverClassName();
+    if (manualDriver != null) {
+      // User has manually specified JDBC implementation with --driver.
+      // Just use GenericJdbcManager.
+      return new GenericJdbcManager(manualDriver, opts);
+    }
+
+    String connectStr = opts.getConnectString();
+
+    int schemeStopIdx = connectStr.indexOf("//");
+    if (-1 == schemeStopIdx) {
+      // no scheme component?
+      throw new IOException("Malformed connect string: " + connectStr);
+    }
+
+    String scheme = connectStr.substring(0, schemeStopIdx);
+
+    if (null == scheme) {
+      // We don't know if this is a mysql://, hsql://, etc.
+      // Can't do anything with this.
+      throw new IOException("Null scheme associated with connect string.");
+    }
+
+    if (scheme.equals("jdbc:mysql:")) {
+      return new MySQLManager(opts);
+    } else if (scheme.equals("jdbc:hsqldb:hsql:")) {
+      return new HsqldbManager(opts);
+    } else {
+      throw new IOException("Unknown connection scheme: " + scheme);
+    }
+  }
+}
+

+ 396 - 0
src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/ImportOptions.java

@@ -0,0 +1,396 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.sqoop;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Properties;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * Command-line arguments used by Sqoop
+ */
+public class ImportOptions {
+
+  public static final Log LOG = LogFactory.getLog(ImportOptions.class.getName());
+
+  /**
+   * Thrown when invalid cmdline options are given
+   */
+  @SuppressWarnings("serial")
+  public static class InvalidOptionsException extends Exception {
+
+    private String message;
+
+    public InvalidOptionsException(final String msg) {
+      this.message = msg;
+    }
+
+    public String getMessage() {
+      return message;
+    }
+
+    public String toString() {
+      return getMessage();
+    }
+  }
+
+  // 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.
+  }
+
+  // selects in-HDFS destination file format
+  public enum FileLayout {
+    TextFile,
+    SequenceFile
+  }
+
+
+  // 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.
+  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 orderByCol;
+  private String debugSqlCmd;
+  private String driverClassName;
+  private String warehouseDir;
+  private FileLayout layout;
+
+  private static final String DEFAULT_CONFIG_FILE = "sqoop.properties";
+
+  public ImportOptions() {
+    initDefaults();
+  }
+
+  /**
+   * Alternate ImportOptions interface used mostly for unit testing
+   * @param connect JDBC connect string to use
+   * @param database Database to read
+   * @param table Table to read
+   */
+  public ImportOptions(final String connect, final String table) {
+    initDefaults();
+
+    this.connectString = connect;
+    this.tableName = table;
+  }
+
+  private void loadFromProperties() {
+    File configFile = new File(DEFAULT_CONFIG_FILE);
+    if (!configFile.canRead()) {
+      return; //can't do this.
+    }
+
+    Properties props = new Properties();
+    InputStream istream = null;
+    try {
+      LOG.info("Loading properties from " + configFile.getAbsolutePath());
+      istream = new FileInputStream(configFile);
+      props.load(istream);
+
+      this.hadoopHome = props.getProperty("hadoop.home", this.hadoopHome);
+      this.codeOutputDir = props.getProperty("out.dir", this.codeOutputDir);
+      this.jarOutputDir = props.getProperty("bin.dir", this.jarOutputDir);
+      this.username = props.getProperty("db.username", this.username);
+      this.password = props.getProperty("db.password", this.password);
+      this.tableName = props.getProperty("db.table", this.tableName);
+      this.connectString = props.getProperty("db.connect.url", this.connectString);
+      this.orderByCol = props.getProperty("db.sort.column", this.orderByCol);
+      this.driverClassName = props.getProperty("jdbc.driver", this.driverClassName);
+      this.warehouseDir = props.getProperty("hdfs.warehouse.dir", this.warehouseDir);
+
+    } catch (IOException ioe) {
+      LOG.error("Could not read properties file " + DEFAULT_CONFIG_FILE + ": " + ioe.toString());
+    } finally {
+      if (null != istream) {
+        try {
+          istream.close();
+        } catch (IOException ioe) {
+          // ignore this; we're closing.
+        }
+      }
+    }
+  }
+
+  private void initDefaults() {
+    // 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");
+    this.codeOutputDir = System.getProperty("sqoop.src.dir", ".");
+
+    String tmpDir = System.getProperty("test.build.data", "/tmp/");
+    if (!tmpDir.endsWith(File.separator)) {
+      tmpDir = tmpDir + File.separator;
+    }
+
+    this.jarOutputDir = tmpDir + "sqoop/compile";
+    this.layout = FileLayout.TextFile;
+
+    loadFromProperties();
+  }
+
+  /**
+   * 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("");
+    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("--order-by (column-name)     Column of the table used to order results");
+    System.out.println("--hadoop-home (dir)          Override $HADOOP_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 --order-by)");
+    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("");
+    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("");
+    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.");
+  }
+
+  /**
+   * 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 {
+    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("--order-by")) {
+          this.orderByCol = 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("--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")) {
+          this.password = args[++i];
+        } else if (args[i].equals("--hadoop-home")) {
+          this.hadoopHome = 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("--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("--help")) {
+          printUsage();
+          throw new InvalidOptionsException("");
+        } 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.");
+    }
+  }
+
+  /**
+   * 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."
+          + "\nTry --help for usage instructions.");
+    } else if (this.allTables && this.orderByCol != null) {
+      // If we're reading all tables in a database, can't set pkey
+      throw new InvalidOptionsException("--order-by and --all-tables are incompatible options."
+          + "\nTry --help for usage instructions.");
+    } else if (this.connectString == null) {
+      throw new InvalidOptionsException("Error: Required argument --connect is missing."
+          + "\nTry --help for usage instructions.");
+    }
+  }
+
+  public String getConnectString() {
+    return connectString;
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  public String[] getColumns() {
+    if (null == columns) {
+      return null;
+    } else {
+      return Arrays.copyOf(columns, columns.length);
+    }
+  }
+
+  public String getOrderByCol() {
+    return orderByCol;
+  }
+
+  public ControlAction getAction() {
+    return action;
+  }
+
+  public boolean isAllTables() {
+    return allTables;
+  }
+
+  public String getUsername() {
+    return username;
+  }
+
+  public String getPassword() {
+    return password;
+  }
+
+  /**
+   * @return location where .java files go; guaranteed to end with '/'
+   */
+  public String getCodeOutputDir() {
+    if (codeOutputDir.endsWith(File.separator)) {
+      return codeOutputDir;
+    } else {
+      return codeOutputDir + File.separator;
+    }
+  }
+
+  /**
+   * @return location where .jar and .class files go; guaranteed to end with '/'
+   */
+  public String getJarOutputDir() {
+    if (jarOutputDir.endsWith(File.separator)) {
+      return jarOutputDir;
+    } else {
+      return jarOutputDir + File.separator;
+    }
+  }
+
+  /**
+   * Return the value of $HADOOP_HOME
+   * @return $HADOOP_HOME, or null if it's not set.
+   */
+  public String getHadoopHome() {
+    return hadoopHome;
+  }
+
+  /**
+   * @return a sql command to execute and exit with.
+   */
+  public String getDebugSqlCmd() {
+    return debugSqlCmd;
+  }
+
+  /**
+   * @return The JDBC driver class name specified with --driver
+   */
+  public String getDriverClassName() {
+    return driverClassName;
+  }
+
+  /**
+   * @return the base destination path for table uploads.
+   */
+  public String getWarehouseDir() {
+    return warehouseDir;
+  }
+
+  /**
+   * @return the destination file format
+   */
+  public FileLayout getFileLayout() {
+    return this.layout;
+  }
+}

+ 175 - 0
src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/Sqoop.java

@@ -0,0 +1,175 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import org.apache.hadoop.sqoop.manager.ConnManager;
+import org.apache.hadoop.sqoop.orm.ClassWriter;
+import org.apache.hadoop.sqoop.orm.CompilationManager;
+import org.apache.hadoop.sqoop.util.ImportError;
+
+/**
+ * Main entry-point for Sqoop
+ * Usage: hadoop jar (this_jar_name) org.apache.hadoop.sqoop.Sqoop (options)
+ * See the ImportOptions class for options.
+ */
+public class Sqoop extends Configured implements Tool {
+
+  public static final Log LOG = LogFactory.getLog(Sqoop.class.getName());
+
+  private ImportOptions options;
+  private ConnManager manager;
+
+  public Sqoop() {
+  }
+
+  public ImportOptions getOptions() {
+    return options;
+  }
+
+  /**
+   * Generate the .class and .jar files
+   * @return the filename of the emitted jar file.
+   * @throws IOException
+   */
+  private String generateORM(String tableName) throws IOException {
+    LOG.info("Beginning code generation");
+    CompilationManager compileMgr = new CompilationManager(options);
+    ClassWriter classWriter = new ClassWriter(options, manager, tableName, compileMgr);
+    classWriter.generate();
+    compileMgr.compile();
+    compileMgr.jar();
+    return compileMgr.getJarFilename();
+  }
+
+  private void importTable(String tableName) throws IOException, ImportError {
+    String jarFile = null;
+
+    // Generate the ORM code for the tables.
+    // TODO(aaron): Allow this to be bypassed if the user has already generated code
+    jarFile = generateORM(tableName);
+
+    if (options.getAction() == ImportOptions.ControlAction.FullImport) {
+      // Proceed onward to do the import.
+      manager.importTable(tableName, jarFile, getConf());
+    }
+  }
+
+
+  /**
+   * Actual main entry-point for the program
+   */
+  public int run(String [] args) {
+    options = new ImportOptions();
+    try {
+      options.parse(args);
+      options.validate();
+    } catch (ImportOptions.InvalidOptionsException e) {
+      // display the error msg
+      System.err.println(e.getMessage());
+      return 1; // exit on exception here
+    }
+
+    // Get the connection to the database
+    try {
+      manager = ConnFactory.getManager(options);
+    } catch (Exception e) {
+      LOG.error("Got error creating database manager: " + e.toString());
+      return 1;
+    }
+
+    ImportOptions.ControlAction action = options.getAction();
+    if (action == ImportOptions.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 == ImportOptions.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 == ImportOptions.ControlAction.DebugExec) {
+      // just run a SQL statement for debugging purposes.
+      manager.execAndPrint(options.getDebugSqlCmd());
+      return 0;
+    } 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());
+        return 1;
+      } catch (ImportError ie) {
+        LOG.error("Error during import: " + ie.toString());
+        return 1;
+      }
+    }
+
+    return 0;
+  }
+
+  public static void main(String [] args) {
+    int ret;
+    try {
+      Sqoop importer = new Sqoop();
+      ret = ToolRunner.run(importer, args);
+    } catch (Exception e) {
+      LOG.error("Got exception running Sqoop: " + e.toString());
+      e.printStackTrace();
+      ret = 1;
+    }
+
+    System.exit(ret);
+  }
+}

+ 84 - 0
src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/lib/BigDecimalSerializer.java

@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.lib;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+
+import org.apache.hadoop.io.Text;
+
+/**
+ * Serialize BigDecimal classes to/from DataInput and DataOutput objects.
+ *
+ * BigDecimal is comprised of a BigInteger with an integer 'scale' field.
+ * The BigDecimal/BigInteger can also return itself as a 'long' value.
+ *
+ * We serialize in one of two formats:
+ *
+ *  First, check whether the BigInt can fit in a long:
+ *  boolean b = BigIntegerPart > LONG_MAX || BigIntegerPart < LONG_MIN
+ *
+ *  [int: scale][boolean: b == false][long: BigInt-part]
+ *  [int: scale][boolean: b == true][string: BigInt-part.toString()]
+ *
+ *
+ * 
+ *
+ * TODO(aaron): Get this to work with Hadoop's Serializations framework.
+ */
+public final class BigDecimalSerializer {
+
+  private BigDecimalSerializer() { }
+
+  static final BigInteger LONG_MAX_AS_BIGINT = BigInteger.valueOf(Long.MAX_VALUE);
+  static final BigInteger LONG_MIN_AS_BIGINT = BigInteger.valueOf(Long.MIN_VALUE);
+
+  public static void write(BigDecimal d, DataOutput out) throws IOException {
+    int scale = d.scale();
+    BigInteger bigIntPart = d.unscaledValue();
+    boolean fastpath = bigIntPart.compareTo(LONG_MAX_AS_BIGINT) < 0
+        && bigIntPart .compareTo(LONG_MIN_AS_BIGINT) > 0;
+
+    out.writeInt(scale);
+    out.writeBoolean(fastpath);
+    if (fastpath) {
+      out.writeLong(bigIntPart.longValue());
+    } else {
+      Text.writeString(out, bigIntPart.toString());
+    }
+  }
+
+  public static BigDecimal readFields(DataInput in) throws IOException {
+    int scale = in.readInt();
+    boolean fastpath = in.readBoolean();
+    BigInteger unscaledIntPart;
+    if (fastpath) {
+      long unscaledValue = in.readLong();
+      unscaledIntPart = BigInteger.valueOf(unscaledValue);
+    } else {
+      String unscaledValueStr = Text.readString(in);
+      unscaledIntPart = new BigInteger(unscaledValueStr);
+    }
+
+    return new BigDecimal(unscaledIntPart, scale);
+  }
+}

+ 203 - 0
src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/lib/JdbcWritableBridge.java

@@ -0,0 +1,203 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.lib;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+
+/**
+ * Contains a set of methods which can read db columns from a ResultSet into
+ * Java types, and do serialization of these types to/from DataInput/DataOutput
+ * for use with Hadoop's Writable implementation. This supports null values
+ * for all types.
+ *
+ * 
+ *
+ */
+public final class JdbcWritableBridge {
+
+  private JdbcWritableBridge() {
+  }
+
+  public static Integer readInteger(int colNum, ResultSet r) throws SQLException {
+    int val;
+    val = r.getInt(colNum);
+    if (r.wasNull()) {
+      return null;
+    } else {
+      return Integer.valueOf(val);
+    }
+  }
+
+  public static Long readLong(int colNum, ResultSet r) throws SQLException {
+    long val;
+    val = r.getLong(colNum);
+    if (r.wasNull()) {
+      return null;
+    } else {
+      return Long.valueOf(val);
+    }
+  }
+
+  public static String readString(int colNum, ResultSet r) throws SQLException {
+    return r.getString(colNum);
+  }
+
+  public static Float readFloat(int colNum, ResultSet r) throws SQLException {
+    float val;
+    val = r.getFloat(colNum);
+    if (r.wasNull()) {
+      return null;
+    } else {
+      return Float.valueOf(val);
+    }
+  }
+
+  public static Double readDouble(int colNum, ResultSet r) throws SQLException {
+    double val;
+    val = r.getDouble(colNum);
+    if (r.wasNull()) {
+      return null;
+    } else {
+      return Double.valueOf(val);
+    }
+  }
+
+  public static Boolean readBoolean(int colNum, ResultSet r) throws SQLException {
+    boolean val;
+    val = r.getBoolean(colNum);
+    if (r.wasNull()) {
+      return null;
+    } else {
+      return Boolean.valueOf(val);
+    }
+  }
+
+  public static Time readTime(int colNum, ResultSet r) throws SQLException {
+    return r.getTime(colNum);
+  }
+
+  public static Timestamp readTimestamp(int colNum, ResultSet r) throws SQLException {
+    return r.getTimestamp(colNum);
+  }
+
+  public static Date readDate(int colNum, ResultSet r) throws SQLException {
+    return r.getDate(colNum);
+  }
+
+  public static BigDecimal readBigDecimal(int colNum, ResultSet r) throws SQLException {
+    return r.getBigDecimal(colNum);
+  }
+
+  public static void writeInteger(Integer val, int paramIdx, int sqlType, PreparedStatement s)
+      throws SQLException {
+    if (null == val) {
+      s.setNull(paramIdx, sqlType);
+    } else {
+      s.setInt(paramIdx, val);
+    }
+  }
+
+  public static void writeLong(Long val, int paramIdx, int sqlType, PreparedStatement s)
+      throws SQLException {
+    if (null == val) {
+      s.setNull(paramIdx, sqlType);
+    } else {
+      s.setLong(paramIdx, val);
+    }
+  }
+
+  public static void writeDouble(Double val, int paramIdx, int sqlType, PreparedStatement s)
+      throws SQLException {
+    if (null == val) {
+      s.setNull(paramIdx, sqlType);
+    } else {
+      s.setDouble(paramIdx, val);
+    }
+  }
+
+  public static void writeBoolean(Boolean val, int paramIdx, int sqlType, PreparedStatement s)
+      throws SQLException {
+    if (null == val) {
+      s.setNull(paramIdx, sqlType);
+    } else {
+      s.setBoolean(paramIdx, val);
+    }
+  }
+
+  public static void writeFloat(Float val, int paramIdx, int sqlType, PreparedStatement s)
+      throws SQLException {
+    if (null == val) {
+      s.setNull(paramIdx, sqlType);
+    } else {
+      s.setFloat(paramIdx, val);
+    }
+  }
+
+  public static void writeString(String val, int paramIdx, int sqlType, PreparedStatement s)
+      throws SQLException {
+    if (null == val) {
+      s.setNull(paramIdx, sqlType);
+    } else {
+      s.setString(paramIdx, val);
+    }
+  }
+
+  public static void writeTimestamp(Timestamp val, int paramIdx, int sqlType, PreparedStatement s)
+      throws SQLException {
+    if (null == val) {
+      s.setNull(paramIdx, sqlType);
+    } else {
+      s.setTimestamp(paramIdx, val);
+    }
+  }
+
+  public static void writeTime(Time val, int paramIdx, int sqlType, PreparedStatement s)
+      throws SQLException {
+    if (null == val) {
+      s.setNull(paramIdx, sqlType);
+    } else {
+      s.setTime(paramIdx, val);
+    }
+  }
+
+  public static void writeDate(Date val, int paramIdx, int sqlType, PreparedStatement s)
+      throws SQLException {
+    if (null == val) {
+      s.setNull(paramIdx, sqlType);
+    } else {
+      s.setDate(paramIdx, val);
+    }
+  }
+
+  public static void writeBigDecimal(BigDecimal val, int paramIdx, int sqlType, PreparedStatement s)
+      throws SQLException {
+    if (null == val) {
+      s.setNull(paramIdx, sqlType);
+    } else {
+      s.setBigDecimal(paramIdx, val);
+    }
+  }
+
+}

+ 106 - 0
src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/ConnManager.java

@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.manager;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.hadoop.sqoop.util.ImportError;
+
+/**
+ * Abstract interface that manages connections to a database.
+ * The implementations of this class drive the actual discussion with
+ * the database about table formats, etc.
+ */
+public interface ConnManager {
+
+  /**
+   * Return a list of all databases on a server
+   */
+  String [] listDatabases();
+
+  /**
+   * Return a list of all tables in a database
+   */
+  String [] listTables();
+
+  /**
+   * Return a list of column names in a table in the order returned by the db.
+   */
+  String [] getColumnNames(String tableName);
+
+  /**
+   * Return the name of the primary key for a table, or null if there is none.
+   */
+  String getPrimaryKey(String tableName);
+
+  /**
+   * Return an unordered mapping from colname to sqltype for
+   * all columns in a table.
+   *
+   * The Integer type id is a constant from java.sql.Types
+   */
+  Map<String, Integer> getColumnTypes(String tableName);
+
+  /**
+   * Execute a SQL statement to read the named set of columns from a table.
+   * If columns is null, all columns from the table are read. This is a local
+   * (non-parallelized) read of the table back to the current client.
+   */
+  ResultSet readTable(String tableName, String [] columns) throws SQLException;
+
+  /**
+   * @return the actual database connection
+   */
+  Connection getConnection() throws SQLException;
+
+  /**
+   * Resolve a database-specific type to the Java type that should contain it.
+   * @param sqlType
+   * @return the name of a Java type to hold the sql datatype, or null if none.
+   */
+  String toJavaType(int sqlType);
+
+  /**
+   * @return a string identifying the driver class to load for this JDBC connection type.
+   */
+  String getDriverClass();
+
+  /**
+   * Execute a SQL statement 's' and print its results to stdout
+   */
+  void execAndPrint(String s);
+
+  /**
+   * Perform an import of a table from the database into HDFS
+   */
+  void importTable(String tableName, String jarFile, Configuration conf)
+      throws IOException, ImportError;
+
+  /**
+   * Perform any shutdown operations on the connection.
+   */
+  void close() throws SQLException;
+}
+

+ 70 - 0
src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/GenericJdbcManager.java

@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.manager;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.sqoop.ImportOptions;
+
+/**
+ * Database manager that is connects to a generic JDBC-compliant
+ * database; its constructor is parameterized on the JDBC Driver
+ * class to load.
+ *
+ * 
+ *
+ */
+public class GenericJdbcManager extends SqlManager {
+
+  public static final Log LOG = LogFactory.getLog(GenericJdbcManager.class.getName());
+
+  private String jdbcDriverClass;
+  private Connection connection;
+
+  public GenericJdbcManager(final String driverClass, final ImportOptions opts) {
+    super(opts);
+
+    this.jdbcDriverClass = driverClass;
+  }
+
+  @Override
+  public Connection getConnection() throws SQLException {
+    if (null == this.connection) {
+      this.connection = makeConnection();
+    }
+
+    return this.connection;
+  }
+
+  public void close() throws SQLException {
+    super.close();
+    if (null != this.connection) {
+      this.connection.close();
+    }
+  }
+
+  public String getDriverClass() {
+    return jdbcDriverClass;
+  }
+}
+

+ 53 - 0
src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/HsqldbManager.java

@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.manager;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.sqoop.ImportOptions;
+
+/**
+ * Manages connections to hsqldb databases.
+ * Extends generic SQL manager.
+ */
+public class HsqldbManager extends GenericJdbcManager implements ConnManager {
+
+  public static final Log LOG = LogFactory.getLog(HsqldbManager.class.getName());
+
+  // driver class to ensure is loaded when making db connection.
+  private static final String DRIVER_CLASS = "org.hsqldb.jdbcDriver";
+
+  // HsqlDb doesn't have a notion of multiple "databases"; the user's database is always called
+  // "PUBLIC";
+  private static final String HSQL_SCHEMA_NAME = "PUBLIC";
+
+  public HsqldbManager(final ImportOptions opts) {
+    super(DRIVER_CLASS, opts);
+  }
+
+  /**
+   * Note: HSqldb only supports a single schema named "PUBLIC"
+   */
+  @Override
+  public String[] listDatabases() {
+    String [] databases = {HSQL_SCHEMA_NAME};
+    return databases;
+  }
+}

+ 68 - 0
src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/MySQLManager.java

@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.manager;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.sqoop.ImportOptions;
+
+/**
+ * Manages connections to MySQL databases
+ * 
+ *
+ */
+public class MySQLManager extends GenericJdbcManager {
+
+  public static final Log LOG = LogFactory.getLog(MySQLManager.class.getName());
+
+  // driver class to ensure is loaded when making db connection.
+  private static final String DRIVER_CLASS = "com.mysql.jdbc.Driver";
+
+  public MySQLManager(final ImportOptions opts) {
+    super(DRIVER_CLASS, opts);
+  }
+
+  @Override
+  public String[] listDatabases() {
+    // TODO(aaron): Add an automated unit test for this.
+
+    ResultSet results = execute("SHOW DATABASES");
+    if (null == results) {
+      return null;
+    }
+
+    try {
+      ArrayList<String> databases = new ArrayList<String>();
+      while (results.next()) {
+        String dbName = results.getString(1);
+        databases.add(dbName);
+      }
+
+      return databases.toArray(new String[0]);
+    } catch (SQLException sqlException) {
+      LOG.error("Error reading from database: " + sqlException.toString());
+      return null;
+    }
+  }
+}

+ 371 - 0
src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/manager/SqlManager.java

@@ -0,0 +1,371 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.manager;
+
+import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.mapred.ImportJob;
+import org.apache.hadoop.sqoop.util.ImportError;
+import org.apache.hadoop.sqoop.util.ResultSetPrinter;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * ConnManager implementation for generic SQL-compliant database.
+ * This is an abstract class; it requires a database-specific
+ * ConnManager implementation to actually create the connection.
+ *
+ * 
+ *
+ */
+public abstract class SqlManager implements ConnManager {
+
+  public static final Log LOG = LogFactory.getLog(SqlManager.class.getName());
+
+  protected ImportOptions options;
+
+  /**
+   * Constructs the SqlManager
+   * @param opts
+   * @param specificMgr
+   */
+  public SqlManager(final ImportOptions opts) {
+    this.options = opts;
+  }
+
+  @Override
+  public String[] getColumnNames(String tableName) {
+    String stmt = "SELECT t.* FROM " + tableName + " AS t WHERE 1 = 1";
+
+    ResultSet results = execute(stmt);
+    if (null == results) {
+      return null;
+    }
+
+    try {
+      int cols = results.getMetaData().getColumnCount();
+      ArrayList<String> columns = new ArrayList<String>();
+      ResultSetMetaData metadata = results.getMetaData();
+      for (int i = 1; i < cols + 1; i++) {
+        String colName = metadata.getColumnName(i);
+        if (colName == null || colName.equals("")) {
+          colName = metadata.getColumnLabel(i);
+        }
+        columns.add(colName);
+      }
+      return columns.toArray(new String[0]);
+    } catch (SQLException sqlException) {
+      LOG.error("Error reading from database: " + sqlException.toString());
+      return null;
+    }
+  }
+
+  @Override
+  public Map<String, Integer> getColumnTypes(String tableName) {
+    String stmt = "SELECT t.* FROM " + tableName + " AS t WHERE 1 = 1";
+
+    ResultSet results = execute(stmt);
+    if (null == results) {
+      return null;
+    }
+
+    try {
+      Map<String, Integer> colTypes = new HashMap<String, Integer>();
+
+      int cols = results.getMetaData().getColumnCount();
+      ResultSetMetaData metadata = results.getMetaData();
+      for (int i = 1; i < cols + 1; i++) {
+        int typeId = metadata.getColumnType(i);
+        String colName = metadata.getColumnName(i);
+        if (colName == null || colName.equals("")) {
+          colName = metadata.getColumnLabel(i);
+        }
+
+        colTypes.put(colName, Integer.valueOf(typeId));
+      }
+
+      return colTypes;
+    } catch (SQLException sqlException) {
+      LOG.error("Error reading from database: " + sqlException.toString());
+      return null;
+    }
+  }
+
+  @Override
+  public ResultSet readTable(String tableName, String[] columns) throws SQLException {
+    if (columns == null) {
+      columns = getColumnNames(tableName);
+    }
+
+    StringBuilder sb = new StringBuilder();
+    sb.append("SELECT ");
+    boolean first = true;
+    for (String col : columns) {
+      if (!first) {
+        sb.append(", ");
+      }
+      sb.append(col);
+      first = false;
+    }
+    sb.append(" FROM ");
+    sb.append(tableName);
+    sb.append(" AS ");   // needed for hsqldb; doesn't hurt anyone else.
+    sb.append(tableName);
+
+    return execute(sb.toString());
+  }
+
+  @Override
+  public String[] listDatabases() {
+    // TODO(aaron): Implement this!
+    LOG.error("Generic SqlManager.listDatabases() not implemented.");
+    return null;
+  }
+
+  @Override
+  public String[] listTables() {
+    ResultSet results = null;
+    String [] tableTypes = {"TABLE"};
+    try {
+      DatabaseMetaData metaData = this.getConnection().getMetaData();
+      results = metaData.getTables(null, null, null, tableTypes);
+    } catch (SQLException sqlException) {
+      LOG.error("Error reading database metadata: " + sqlException.toString());
+      return null;
+    }
+
+    if (null == results) {
+      return null;
+    }
+
+    try {
+      ArrayList<String> tables = new ArrayList<String>();
+      while (results.next()) {
+        String tableName = results.getString("TABLE_NAME");
+        tables.add(tableName);
+      }
+
+      return tables.toArray(new String[0]);
+    } catch (SQLException sqlException) {
+      LOG.error("Error reading from database: " + sqlException.toString());
+      return null;
+    }
+  }
+
+  @Override
+  public String getPrimaryKey(String tableName) {
+    try {
+      DatabaseMetaData metaData = this.getConnection().getMetaData();
+      ResultSet results = metaData.getPrimaryKeys(null, null, tableName);
+      if (null == results) {
+        return null;
+      }
+
+      if (results.next()) {
+        return results.getString("COLUMN_NAME");
+      }
+    } catch (SQLException sqlException) {
+      LOG.error("Error reading primary key metadata: " + sqlException.toString());
+      return null;
+    }
+
+    return null;
+  }
+
+  /**
+   * Retrieve the actual connection from the outer ConnManager
+   */
+  public abstract Connection getConnection() throws SQLException;
+
+  /**
+   * Default implementation of importTable() is to launch a MapReduce job
+   * via ImportJob to read the table with DBInputFormat.
+   */
+  public void importTable(String tableName, String jarFile, Configuration conf)
+      throws IOException, ImportError {
+    ImportJob importer = new ImportJob(options);
+    String orderCol = options.getOrderByCol();
+    if (null == orderCol) {
+      // If the user didn't specify an ordering column, try to infer one.
+      orderCol = getPrimaryKey(tableName);
+    }
+
+    if (null == orderCol) {
+      // Can't infer a primary key.
+      throw new ImportError("No primary key could be found for table " + tableName
+          + ". Please specify one with --order-by.");
+    }
+
+    importer.runImport(tableName, jarFile, orderCol, conf);
+  }
+
+  /**
+   * executes an arbitrary SQL statement
+   * @param stmt The SQL statement to execute
+   * @return A ResultSet encapsulating the results or null on error
+   */
+  protected ResultSet execute(String stmt, Object... args) {
+    if (null == stmt) {
+      LOG.error("Null statement sent to SqlManager.execute()");
+      return null;
+    }
+
+    PreparedStatement statement = null;
+    try {
+      statement = this.getConnection().prepareStatement(stmt,
+          ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+      if (null != args) {
+        for (int i = 0; i < args.length; i++) {
+          statement.setObject(i + 1, args[i]);
+        }
+      }
+
+      LOG.info("Executing SQL statement: " + stmt);
+      return statement.executeQuery();
+    } catch (SQLException sqlException) {
+      LOG.error("Error returned by SQL database: " + sqlException.toString());
+      return null;
+    }
+
+    // TODO(aaron): Is calling ResultSet.close() sufficient?
+    // Or must statement.close() be called too?
+  }
+
+  public String toJavaType(int sqlType) {
+    // mappings from http://java.sun.com/j2se/1.3/docs/guide/jdbc/getstart/mapping.html
+    if (sqlType == Types.INTEGER) {
+      return "Integer";
+    } else if (sqlType == Types.VARCHAR) {
+      return "String";
+    } else if (sqlType == Types.CHAR) {
+      return "String";
+    } else if (sqlType == Types.LONGVARCHAR) {
+      return "String";
+    } else if (sqlType == Types.NUMERIC) {
+      return "java.math.BigDecimal";
+    } else if (sqlType == Types.DECIMAL) {
+      return "java.math.BigDecimal";
+    } else if (sqlType == Types.BIT) {
+      return "Boolean";
+    } else if (sqlType == Types.BOOLEAN) {
+      return "Boolean";
+    } else if (sqlType == Types.TINYINT) {
+      return "Integer";
+    } else if (sqlType == Types.SMALLINT) {
+      return "Integer";
+    } else if (sqlType == Types.BIGINT) {
+      return "Long";
+    } else if (sqlType == Types.REAL) {
+      return "Float";
+    } else if (sqlType == Types.FLOAT) {
+      return "Double";
+    } else if (sqlType == Types.DOUBLE) {
+      return "Double";
+    } else if (sqlType == Types.DATE) {
+      return "java.sql.Date";
+    } else if (sqlType == Types.TIME) {
+      return "java.sql.Time";
+    } else if (sqlType == Types.TIMESTAMP) {
+      return "java.sql.Timestamp";
+    } else {
+      // TODO(aaron): Support BINARY, VARBINARY, LONGVARBINARY, DISTINCT, CLOB, BLOB, ARRAY,
+      // STRUCT, REF, JAVA_OBJECT.
+      return null;
+    }
+  }
+
+
+  public void close() throws SQLException {
+  }
+
+  /**
+   * Poor man's SQL query interface; used for debugging.
+   * @param s
+   */
+  public void execAndPrint(String s) {
+    System.out.println("Executing statement: " + s);
+    ResultSet results = execute(s);
+    if (null == results) {
+      LOG.info("Got null results back!");
+      return;
+    }
+
+    try {
+      int cols = results.getMetaData().getColumnCount();
+      System.out.println("Got " + cols + " columns back");
+      if (cols > 0) {
+        System.out.println("Schema: " + results.getMetaData().getSchemaName(1));
+        System.out.println("Table: " + results.getMetaData().getTableName(1));
+      }
+    } catch (SQLException sqlE) {
+      LOG.error("SQLException reading result metadata: " + sqlE.toString());
+    }
+
+    try {
+      new ResultSetPrinter().printResultSet(System.out, results);
+    } catch (IOException ioe) {
+      LOG.error("IOException writing results to stdout: " + ioe.toString());
+      return;
+    }
+  }
+
+  /**
+   * Create a connection to the database; usually used only from within
+   * getConnection(), which enforces a singleton guarantee around the
+   * Connection object.
+   */
+  protected Connection makeConnection() throws SQLException {
+
+    Connection connection;
+    String driverClass = getDriverClass();
+
+    try {
+      Class.forName(driverClass);
+    } catch (ClassNotFoundException cnfe) {
+      throw new RuntimeException("Could not load db driver class: " + driverClass);
+    }
+
+    String username = options.getUsername();
+    String password = options.getPassword();
+    if (null == username) {
+      connection = DriverManager.getConnection(options.getConnectString());
+    } else {
+      connection = DriverManager.getConnection(options.getConnectString(), username, password);
+    }
+
+    connection.setAutoCommit(false);
+    connection.setTransactionIsolation(Connection.TRANSACTION_SERIALIZABLE);
+
+    return connection;
+  }
+}

+ 149 - 0
src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapred/ImportJob.java

@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.mapred;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.SequenceFileOutputFormat;
+import org.apache.hadoop.mapred.lib.db.DBConfiguration;
+import org.apache.hadoop.mapred.lib.db.DBInputFormat;
+import org.apache.hadoop.mapred.lib.db.DBWritable;
+
+import org.apache.hadoop.sqoop.ConnFactory;
+import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.manager.ConnManager;
+import org.apache.hadoop.sqoop.util.ClassLoaderStack;
+
+/**
+ * Actually runs a jdbc import job using the ORM files generated by the sqoop.orm package.
+ *
+ * 
+ *
+ */
+public class ImportJob {
+
+  public static final Log LOG = LogFactory.getLog(ImportJob.class.getName());
+
+  private ImportOptions options;
+
+  public ImportJob(final ImportOptions opts) {
+    this.options = opts;
+  }
+
+  /**
+   * Run an import job to read a table in to HDFS
+   *
+   * @param tableName  the database table to read
+   * @param ormJarFile the Jar file to insert into the dcache classpath. (may be null)
+   * @param orderByCol the column of the database table to use to order the import
+   * @param conf A fresh Hadoop Configuration to use to build an MR job.
+   */
+  public void runImport(String tableName, String ormJarFile, String orderByCol,
+      Configuration conf) throws IOException {
+
+    LOG.info("Beginning data import of " + tableName);
+
+    // TODO(aaron): If we add packages, the tableName will not be the class name.
+    String tableClassName = tableName;
+
+    boolean isLocal = "local".equals(conf.get("mapred.job.tracker"));
+    ClassLoader prevClassLoader = null;
+    if (isLocal) {
+      // If we're using the LocalJobRunner, then instead of using the compiled jar file
+      // as the job source, we're running in the current thread. Push on another classloader
+      // that loads from that jar in addition to everything currently on the classpath.
+
+      // take advantage of the fact that table name = class name.
+      prevClassLoader = ClassLoaderStack.addJarFile(ormJarFile, tableClassName);
+    }
+
+    try {
+      JobConf job = new JobConf(conf);
+      job.setJar(ormJarFile);
+
+      String hdfsWarehouseDir = options.getWarehouseDir();
+      Path outputPath;
+
+      if (null != hdfsWarehouseDir) {
+        Path hdfsWarehousePath = new Path(hdfsWarehouseDir);
+        hdfsWarehousePath.makeQualified(FileSystem.get(job));
+        outputPath = new Path(hdfsWarehousePath, tableName);
+      } else {
+        outputPath = new Path(tableName);
+      }
+
+      if (options.getFileLayout() == ImportOptions.FileLayout.TextFile) {
+        job.setMapperClass(TextImportMapper.class);
+        job.setOutputKeyClass(Text.class);
+        job.setOutputValueClass(NullWritable.class);
+      } else if (options.getFileLayout() == ImportOptions.FileLayout.SequenceFile) {
+        job.setOutputFormat(SequenceFileOutputFormat.class);
+        SequenceFileOutputFormat.setCompressOutput(job, true);
+        SequenceFileOutputFormat.setOutputCompressionType(job, CompressionType.BLOCK);
+        job.set("mapred.output.value.class", tableClassName);
+      } else {
+        LOG.warn("Unknown file layout specified: " + options.getFileLayout() + "; using text.");
+      }
+
+      job.setNumReduceTasks(0);
+      job.setInputFormat(DBInputFormat.class);
+
+      FileOutputFormat.setOutputPath(job, outputPath);
+
+      ConnManager mgr = ConnFactory.getManager(options);
+      String username = options.getUsername();
+      if (null == username || username.length() == 0) {
+        DBConfiguration.configureDB(job, mgr.getDriverClass(), options.getConnectString());
+      } else {
+        DBConfiguration.configureDB(job, mgr.getDriverClass(), options.getConnectString(),
+            username, options.getPassword());
+      }
+
+      String [] colNames = options.getColumns();
+      if (null == colNames) {
+        colNames = mgr.getColumnNames(tableName);
+      }
+
+      // We can't set the class properly in here, because we may not have the
+      // jar loaded in this JVM. So we start by calling setInput() with DBWritable,
+      // and then overriding the string manually.
+      DBInputFormat.setInput(job, DBWritable.class, tableName, null,
+          orderByCol, colNames);
+      job.set(DBConfiguration.INPUT_CLASS_PROPERTY, tableClassName);
+
+      JobClient.runJob(job);
+    } finally {
+      if (isLocal && null != prevClassLoader) {
+        // unload the special classloader for this jar.
+        ClassLoaderStack.setCurrentClassLoader(prevClassLoader);
+      }
+    }
+  }
+}

+ 52 - 0
src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/mapred/TextImportMapper.java

@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.mapred;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.lib.db.DBWritable;
+
+/**
+ * Converts an input record into a string representation and emit it.
+ * 
+ *
+ */
+public class TextImportMapper extends MapReduceBase
+    implements Mapper<LongWritable, DBWritable, Text, NullWritable> {
+
+  private Text outkey;
+
+  public TextImportMapper() {
+    outkey = new Text();
+  }
+
+  public void map(LongWritable key, DBWritable val, OutputCollector<Text, NullWritable> output,
+      Reporter reporter) throws IOException {
+
+    outkey.set(val.toString());
+    output.collect(outkey, NullWritable.get());
+  }
+}

+ 550 - 0
src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/orm/ClassWriter.java

@@ -0,0 +1,550 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.orm;
+
+import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.manager.ConnManager;
+import org.apache.hadoop.sqoop.lib.BigDecimalSerializer;
+import org.apache.hadoop.sqoop.lib.JdbcWritableBridge;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Creates an ORM class to represent a table from a database
+ *
+ * 
+ *
+ */
+public class ClassWriter {
+
+  public static final Log LOG = LogFactory.getLog(ClassWriter.class.getName());
+
+  /**
+   * This version number is injected into all generated Java classes to denote
+   * which version of the ClassWriter's output format was used to generate the
+   * class.
+   *
+   *  If the way that we generate classes, bump this number.
+   */
+  public static final int CLASS_WRITER_VERSION = 1;
+
+  private ImportOptions options;
+  private ConnManager connManager;
+  private String tableName;
+  private CompilationManager compileManager;
+
+  /**
+   * Creates a new ClassWriter to generate an ORM class for a table.
+   * @param opts program-wide options
+   * @param connMgr the connection manager used to describe the table.
+   * @param table the name of the table to read.
+   */
+  public ClassWriter(final ImportOptions opts, final ConnManager connMgr,
+      final String table, final CompilationManager compMgr) {
+    this.options = opts;
+    this.connManager = connMgr;
+    this.tableName = table;
+    this.compileManager = compMgr;
+  }
+
+
+  /**
+   * @param javaType
+   * @return the name of the method of JdbcWritableBridge to read an entry with a given java type.
+   */
+  private String dbGetterForType(String javaType) {
+    // All Class-based types (e.g., java.math.BigDecimal) are handled with
+    // "readBar" where some.package.foo.Bar is the canonical class name.
+    // Turn the javaType string into the getter type string.
+
+    String [] parts = javaType.split("\\.");
+    if (parts.length == 0) {
+      LOG.error("No ResultSet method for Java type " + javaType);
+      return null;
+    }
+
+    String lastPart = parts[parts.length - 1];
+    try {
+      String getter = "read" + Character.toUpperCase(lastPart.charAt(0)) + lastPart.substring(1);
+      return getter;
+    } catch (StringIndexOutOfBoundsException oob) {
+      // lastPart.*() doesn't work on empty strings.
+      LOG.error("Could not infer JdbcWritableBridge getter for Java type " + javaType);
+      return null;
+    }
+  }
+
+  /**
+   * @param javaType
+   * @return the name of the method of JdbcWritableBridge to write an entry with a given java type.
+   */
+  private String dbSetterForType(String javaType) {
+    // TODO(aaron): Lots of unit tests needed here.
+    // See dbGetterForType() for the logic used here; it's basically the same.
+
+    String [] parts = javaType.split("\\.");
+    if (parts.length == 0) {
+      LOG.error("No PreparedStatement Set method for Java type " + javaType);
+      return null;
+    }
+
+    String lastPart = parts[parts.length - 1];
+    try {
+      String setter = "write" + Character.toUpperCase(lastPart.charAt(0)) + lastPart.substring(1);
+      return setter;
+    } catch (StringIndexOutOfBoundsException oob) {
+      // lastPart.*() doesn't work on empty strings.
+      LOG.error("Could not infer PreparedStatement setter for Java type " + javaType);
+      return null;
+    }
+  }
+
+  private String stringifierForType(String javaType, String colName) {
+    if (javaType.equals("String")) {
+      return colName;
+    } else {
+      // this is an object type -- just call its toString() in a null-safe way.
+      return "\"\" + " + colName;
+    }
+  }
+
+  /**
+   * @param javaType the type to read
+   * @param inputObj the name of the DataInput to read from
+   * @param colName the column name to read
+   * @return the line of code involving a DataInput object to read an entry with a given java type.
+   */
+  private String rpcGetterForType(String javaType, String inputObj, String colName) {
+    if (javaType.equals("Integer")) {
+      return "    this." + colName + " = Integer.valueOf(" + inputObj + ".readInt());\n";
+    } else if (javaType.equals("Long")) {
+      return "    this." + colName + " = Long.valueOf(" + inputObj + ".readLong());\n";
+    } else if (javaType.equals("Float")) {
+      return "    this." + colName + " = Float.valueOf(" + inputObj + ".readFloat());\n";
+    } else if (javaType.equals("Double")) {
+      return "    this." + colName + " = Double.valueOf(" + inputObj + ".readDouble());\n";
+    } else if (javaType.equals("Boolean")) {
+      return "    this." + colName + " = Boolean.valueOf(" + inputObj + ".readBoolean());\n";
+    } else if (javaType.equals("String")) {
+      return "    this." + colName + " = Text.readString(" + inputObj + ");\n";
+    } else if (javaType.equals("java.sql.Date")) {
+      return "    this." + colName + " = new Date(" + inputObj + ".readLong());\n";
+    } else if (javaType.equals("java.sql.Time")) {
+      return "    this." + colName + " = new Time(" + inputObj + ".readLong());\n";
+    } else if (javaType.equals("java.sql.Timestamp")) {
+      return "    this." + colName + " = new Timestamp(" + inputObj + ".readLong());\n"
+          + "    this." + colName + ".setNanos(" + inputObj + ".readInt());\n";
+    } else if (javaType.equals("java.math.BigDecimal")) {
+      return "    this." + colName + " = " + BigDecimalSerializer.class.getCanonicalName()
+          + ".readFields(" + inputObj + ");\n";
+    } else {
+      LOG.error("No ResultSet method for Java type " + javaType);
+      return null;
+    }
+  }
+
+  /**
+   * Deserialize a possibly-null value from the DataInput stream
+   * @param javaType name of the type to deserialize if it's not null.
+   * @param inputObj name of the DataInput to read from
+   * @param colName the column name to read.
+   * @return
+   */
+  private String rpcGetterForMaybeNull(String javaType, String inputObj, String colName) {
+    return "    if (" + inputObj + ".readBoolean()) { \n"
+        + "        this." + colName + " = null;\n"
+        + "    } else {\n"
+        + rpcGetterForType(javaType, inputObj, colName)
+        + "    }\n";
+  }
+
+  /**
+   * @param javaType the type to write
+   * @param inputObj the name of the DataOutput to write to
+   * @param colName the column name to write
+   * @return the line of code involving a DataOutput object to write an entry with
+   *         a given java type.
+   */
+  private String rpcSetterForType(String javaType, String outputObj, String colName) {
+    if (javaType.equals("Integer")) {
+      return "    " + outputObj + ".writeInt(this." + colName + ");\n";
+    } else if (javaType.equals("Long")) {
+      return "    " + outputObj + ".writeLong(this." + colName + ");\n";
+    } else if (javaType.equals("Boolean")) {
+      return "    " + outputObj + ".writeBoolean(this." + colName + ");\n";
+    } else if (javaType.equals("Float")) {
+      return "    " + outputObj + ".writeFloat(this." + colName + ");\n";
+    } else if (javaType.equals("Double")) {
+      return "    " + outputObj + ".writeDouble(this." + colName + ");\n";
+    } else if (javaType.equals("String")) {
+      return "    Text.writeString(" + outputObj + ", " + colName + ");\n";
+    } else if (javaType.equals("java.sql.Date")) {
+      return "    " + outputObj + ".writeLong(this." + colName + ".getTime());\n";
+    } else if (javaType.equals("java.sql.Time")) {
+      return "    " + outputObj + ".writeLong(this." + colName + ".getTime());\n";
+    } else if (javaType.equals("java.sql.Timestamp")) {
+      return "    " + outputObj + ".writeLong(this." + colName + ".getTime());\n"
+          + "    " + outputObj + ".writeInt(this." + colName + ".getNanos());\n";
+    } else if (javaType.equals("java.math.BigDecimal")) {
+      return "    " + BigDecimalSerializer.class.getCanonicalName()
+          + ".write(this." + colName + ", " + outputObj + ");\n";
+    } else {
+      LOG.error("No ResultSet method for Java type " + javaType);
+      return null;
+    }
+  }
+
+  /**
+   * Serialize a possibly-null value to the DataOutput stream. First a boolean
+   * isNull is written, followed by the contents itself (if not null).
+   * @param javaType name of the type to deserialize if it's not null.
+   * @param inputObj name of the DataInput to read from
+   * @param colName the column name to read.
+   * @return
+   */
+  private String rpcSetterForMaybeNull(String javaType, String outputObj, String colName) {
+    return "    if (null == this." + colName + ") { \n"
+        + "        " + outputObj + ".writeBoolean(true);\n"
+        + "    } else {\n"
+        + "        " + outputObj + ".writeBoolean(false);\n"
+        + rpcSetterForType(javaType, outputObj, colName)
+        + "    }\n";
+  }
+
+  /**
+   * Generate a member field and getter method for each column
+   * @param columnTypes - mapping from column names to sql types
+   * @param colNames - ordered list of column names for table.
+   * @param sb - StringBuilder to append code to
+   */
+  private void generateFields(Map<String, Integer> columnTypes, String [] colNames,
+      StringBuilder sb) {
+
+    for (String col : colNames) {
+      int sqlType = columnTypes.get(col);
+      String javaType = connManager.toJavaType(sqlType);
+      if (null == javaType) {
+        LOG.error("Cannot resolve SQL type " + sqlType);
+        continue;
+      }
+
+      sb.append("  private " + javaType + " " + col + ";\n");
+      sb.append("  public " + javaType + " get_" + col + "() {\n");
+      sb.append("    return " + col + ";\n");
+      sb.append("  }\n");
+    }
+  }
+
+  /**
+   * Generate the readFields() method used by the database
+   * @param columnTypes - mapping from column names to sql types
+   * @param colNames - ordered list of column names for table.
+   * @param sb - StringBuilder to append code to
+   */
+  private void generateDbRead(Map<String, Integer> columnTypes, String [] colNames,
+      StringBuilder sb) {
+
+    sb.append("  public void readFields(ResultSet __dbResults) throws SQLException {\n");
+
+    int fieldNum = 0;
+
+    for (String col : colNames) {
+      fieldNum++;
+
+      int sqlType = columnTypes.get(col);
+      String javaType = connManager.toJavaType(sqlType);
+      if (null == javaType) {
+        LOG.error("No Java type for SQL type " + sqlType);
+        continue;
+      }
+
+      String getterMethod = dbGetterForType(javaType);
+      if (null == getterMethod) {
+        LOG.error("No db getter method for Java type " + javaType);
+        continue;
+      }
+
+      sb.append("    this." + col + " = JdbcWritableBridge." +  getterMethod
+          + "(" + fieldNum + ", __dbResults);\n");
+    }
+
+    sb.append("  }\n");
+  }
+
+
+  /**
+   * Generate the write() method used by the database
+   * @param columnTypes - mapping from column names to sql types
+   * @param colNames - ordered list of column names for table.
+   * @param sb - StringBuilder to append code to
+   */
+  private void generateDbWrite(Map<String, Integer> columnTypes, String [] colNames,
+      StringBuilder sb) {
+
+    sb.append("  public void write(PreparedStatement __dbStmt) throws SQLException {\n");
+
+    int fieldNum = 0;
+
+    for (String col : colNames) {
+      fieldNum++;
+
+      int sqlType = columnTypes.get(col);
+      String javaType = connManager.toJavaType(sqlType);
+      if (null == javaType) {
+        LOG.error("No Java type for SQL type " + sqlType);
+        continue;
+      }
+
+      String setterMethod = dbSetterForType(javaType);
+      if (null == setterMethod) {
+        LOG.error("No db setter method for Java type " + javaType);
+        continue;
+      }
+
+      sb.append("    JdbcWritableBridge." + setterMethod + "(" + col + ", "
+          + fieldNum + ", " + sqlType + ", __dbStmt);\n");
+    }
+
+    sb.append("  }\n");
+  }
+
+
+  /**
+   * Generate the readFields() method used by the Hadoop RPC system
+   * @param columnTypes - mapping from column names to sql types
+   * @param colNames - ordered list of column names for table.
+   * @param sb - StringBuilder to append code to
+   */
+  private void generateHadoopRead(Map<String, Integer> columnTypes, String [] colNames,
+      StringBuilder sb) {
+
+    sb.append("  public void readFields(DataInput __dataIn) throws IOException {\n");
+
+    for (String col : colNames) {
+      int sqlType = columnTypes.get(col);
+      String javaType = connManager.toJavaType(sqlType);
+      if (null == javaType) {
+        LOG.error("No Java type for SQL type " + sqlType);
+        continue;
+      }
+
+      String getterMethod = rpcGetterForMaybeNull(javaType, "__dataIn", col);
+      if (null == getterMethod) {
+        LOG.error("No RPC getter method for Java type " + javaType);
+        continue;
+      }
+
+      sb.append(getterMethod);
+    }
+
+    sb.append("  }\n");
+  }
+
+  /**
+   * Generate the toString() method
+   * @param columnTypes - mapping from column names to sql types
+   * @param colNames - ordered list of column names for table.
+   * @param sb - StringBuilder to append code to
+   */
+  private void generateToString(Map<String, Integer> columnTypes, String [] colNames,
+      StringBuilder sb) {
+
+    sb.append("  public String toString() {\n");
+    sb.append("    StringBuilder sb = new StringBuilder();\n");
+
+    boolean first = true;
+    for (String col : colNames) {
+      int sqlType = columnTypes.get(col);
+      String javaType = connManager.toJavaType(sqlType);
+      if (null == javaType) {
+        LOG.error("No Java type for SQL type " + sqlType);
+        continue;
+      }
+
+      if (!first) {
+        // TODO(aaron): Support arbitrary record delimiters
+        sb.append("    sb.append(\",\");\n");
+      }
+
+      first = false;
+
+      String stringExpr = stringifierForType(javaType, col);
+      if (null == stringExpr) {
+        LOG.error("No toString method for Java type " + javaType);
+        continue;
+      }
+
+      sb.append("    sb.append(" + stringExpr + ");\n");
+
+    }
+
+    sb.append("    return sb.toString();\n");
+    sb.append("  }\n");
+  }
+
+  /**
+   * Generate the write() method used by the Hadoop RPC system
+   * @param columnTypes - mapping from column names to sql types
+   * @param colNames - ordered list of column names for table.
+   * @param sb - StringBuilder to append code to
+   */
+  private void generateHadoopWrite(Map<String, Integer> columnTypes, String [] colNames,
+      StringBuilder sb) {
+
+    sb.append("  public void write(DataOutput __dataOut) throws IOException {\n");
+
+    for (String col : colNames) {
+      int sqlType = columnTypes.get(col);
+      String javaType = connManager.toJavaType(sqlType);
+      if (null == javaType) {
+        LOG.error("No Java type for SQL type " + sqlType);
+        continue;
+      }
+
+      String setterMethod = rpcSetterForMaybeNull(javaType, "__dataOut", col);
+      if (null == setterMethod) {
+        LOG.error("No RPC setter method for Java type " + javaType);
+        continue;
+      }
+
+      sb.append(setterMethod);
+    }
+
+    sb.append("  }\n");
+  }
+  /**
+   * Generate the ORM code for the class.
+   */
+  public void generate() throws IOException {
+    Map<String, Integer> columnTypes = connManager.getColumnTypes(tableName);
+
+    String [] colNames = options.getColumns();
+    if (null == colNames) {
+      colNames = connManager.getColumnNames(tableName);
+    }
+
+    // Generate the Java code
+    StringBuilder sb = generateClassForColumns(columnTypes, colNames);
+
+    // Write this out to a file.
+    String codeOutDir = options.getCodeOutputDir();
+
+    // TODO(aaron): Allow package subdirectory (that goes in sourceFilename).
+    String sourceFilename = tableName + ".java";
+    String filename = codeOutDir + sourceFilename;
+
+    LOG.debug("Writing source file: " + filename);
+    LOG.debug("Table name: " + tableName);
+    StringBuilder sbColTypes = new StringBuilder();
+    for (String col : colNames) {
+      Integer colType = columnTypes.get(col);
+      sbColTypes.append(col + ":" + colType + ", ");
+    }
+    String colTypeStr = sbColTypes.toString();
+    LOG.debug("Columns: " + colTypeStr);
+
+    compileManager.addSourceFile(sourceFilename);
+
+    // Create any missing parent directories.
+    File file = new File(filename);
+    String dirname = file.getParent();
+    if (null != dirname) {
+      boolean mkdirSuccess = new File(dirname).mkdirs();
+      if (!mkdirSuccess) {
+        LOG.debug("Could not create directory tree for " + dirname);
+      }
+    }
+
+    OutputStream ostream = null;
+    Writer writer = null;
+    try {
+      ostream = new FileOutputStream(filename);
+      writer = new OutputStreamWriter(ostream);
+      writer.append(sb.toString());
+    } finally {
+      if (null != writer) {
+        try {
+          writer.close();
+        } catch (IOException ioe) {
+          // ignored because we're closing.
+        }
+      }
+
+      if (null != ostream) {
+        try {
+          ostream.close();
+        } catch (IOException ioe) {
+          // ignored because we're closing.
+        }
+      }
+    }
+  }
+
+  /**
+   * Generate the ORM code for a table object containing the named columns
+   * @param columnTypes - mapping from column names to sql types
+   * @param colNames - ordered list of column names for table.
+   * @return - A StringBuilder that contains the text of the class code.
+   */
+  public StringBuilder generateClassForColumns(Map<String, Integer> columnTypes,
+      String [] colNames) {
+    StringBuilder sb = new StringBuilder();
+    // TODO(aaron): Emit package name.
+    sb.append("// ORM class for " + tableName + "\n");
+    sb.append("// WARNING: This class is AUTO-GENERATED. Modify at your own risk.\n");
+
+    sb.append("import org.apache.hadoop.io.Text;\n");
+    sb.append("import org.apache.hadoop.io.Writable;\n");
+    sb.append("import org.apache.hadoop.mapred.lib.db.DBWritable;\n");
+    sb.append("import " + JdbcWritableBridge.class.getCanonicalName() + ";\n");
+    sb.append("import java.sql.PreparedStatement;\n");
+    sb.append("import java.sql.ResultSet;\n");
+    sb.append("import java.sql.SQLException;\n");
+    sb.append("import java.io.DataInput;\n");
+    sb.append("import java.io.DataOutput;\n");
+    sb.append("import java.io.IOException;\n");
+    sb.append("import java.sql.Date;\n");
+    sb.append("import java.sql.Time;\n");
+    sb.append("import java.sql.Timestamp;\n");
+
+    // TODO(aaron): Allow different table/class names.
+    sb.append("public class " + tableName + " implements DBWritable, Writable {\n");
+    sb.append("  public static final int PROTOCOL_VERSION = " + CLASS_WRITER_VERSION + ";\n");
+    generateFields(columnTypes, colNames, sb);
+    generateDbRead(columnTypes, colNames, sb);
+    generateDbWrite(columnTypes, colNames, sb);
+    generateHadoopRead(columnTypes, colNames, sb);
+    generateHadoopWrite(columnTypes, colNames, sb);
+    generateToString(columnTypes, colNames, sb);
+    // TODO(aaron): Generate hashCode(), compareTo(), equals() so it can be a WritableComparable
+
+    sb.append("}\n");
+
+    return sb;
+  }
+}

+ 313 - 0
src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/orm/CompilationManager.java

@@ -0,0 +1,313 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.orm;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.URL;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.List;
+import java.util.jar.JarOutputStream;
+import java.util.zip.ZipEntry;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.mapred.JobConf;
+
+import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.util.FileListing;
+
+/**
+ * Manages the compilation of a bunch of .java files into .class files
+ * and eventually a jar.
+ *
+ * Also embeds this program's jar into the lib/ directory inside the compiled jar
+ * to ensure that the job runs correctly.
+ *
+ * 
+ *
+ */
+public class CompilationManager {
+
+  public static final Log LOG = LogFactory.getLog(CompilationManager.class.getName());
+
+  private ImportOptions options;
+  private List<String> sources;
+
+  public CompilationManager(final ImportOptions opts) {
+    options = opts;
+    sources = new ArrayList<String>();
+  }
+
+  public void addSourceFile(String sourceName) {
+    sources.add(sourceName);
+  }
+
+  /**
+   * locate the hadoop-*-core.jar in $HADOOP_HOME or --hadoop-home.
+   * If that doesn't work, check our classpath.
+   * @return the filename of the hadoop-*-core.jar file.
+   */
+  private String findHadoopCoreJar() {
+    String hadoopHome = options.getHadoopHome();
+
+    if (null == hadoopHome) {
+      LOG.info("$HADOOP_HOME is not set");
+      return findJarForClass(JobConf.class);
+    }
+
+    if (!hadoopHome.endsWith(File.separator)) {
+      hadoopHome = hadoopHome + File.separator;
+    }
+
+    File hadoopHomeFile = new File(hadoopHome);
+    LOG.info("HADOOP_HOME is " + hadoopHomeFile.getAbsolutePath());
+    File [] entries = hadoopHomeFile.listFiles();
+
+    if (null == entries) {
+      LOG.warn("HADOOP_HOME appears empty or missing");
+      return findJarForClass(JobConf.class);
+    }
+
+    for (File f : entries) {
+      if (f.getName().startsWith("hadoop-") && f.getName().endsWith("-core.jar")) {
+        LOG.info("Found hadoop core jar at: " + f.getAbsolutePath());
+        return f.getAbsolutePath();
+      }
+    }
+
+    return findJarForClass(JobConf.class);
+  }
+
+  /**
+   * Compile the .java files into .class files via embedded javac call.
+   */
+  public void compile() throws IOException {
+    List<String> args = new ArrayList<String>();
+
+    // ensure that the jar output dir exists.
+    String jarOutDir = options.getJarOutputDir();
+    boolean mkdirSuccess = new File(jarOutDir).mkdirs();
+    if (!mkdirSuccess) {
+      LOG.debug("Warning: Could not make directories for " + jarOutDir);
+    }
+
+    // find hadoop-*-core.jar for classpath.
+    String coreJar = findHadoopCoreJar();
+    if (null == coreJar) {
+      // Couldn't find a core jar to insert into the CP for compilation.
+      // If, however, we're running this from a unit test, then the path
+      // to the .class files might be set via the hadoop.alt.classpath property
+      // instead. Check there first.
+      String coreClassesPath = System.getProperty("hadoop.alt.classpath");
+      if (null == coreClassesPath) {
+        // no -- we're out of options. Fail.
+        throw new IOException("Could not find hadoop core jar!");
+      } else {
+        coreJar = coreClassesPath;
+      }
+    }
+
+    String curClasspath = System.getProperty("java.class.path");
+
+    args.add("-sourcepath");
+    String srcOutDir = options.getCodeOutputDir();
+    args.add(srcOutDir);
+
+    args.add("-d");
+    args.add(jarOutDir);
+
+    args.add("-classpath");
+    args.add(curClasspath + File.pathSeparator + coreJar);
+
+    // add all the source files
+    for (String srcfile : sources) {
+      args.add(srcOutDir + srcfile);
+    }
+
+    StringBuilder sb = new StringBuilder();
+    for (String arg : args) {
+      sb.append(arg + " ");
+    }
+
+    // NOTE(aaron): Usage is at http://java.sun.com/j2se/1.5.0/docs/tooldocs/solaris/javac.html
+    LOG.info("Invoking javac with args: " + sb.toString());
+    int javacRet = com.sun.tools.javac.Main.compile(args.toArray(new String[0]));
+    if (javacRet != 0) {
+      throw new IOException("javac exited with status " + javacRet);
+    }
+  }
+
+  public String getJarFilename() {
+    String jarOutDir = options.getJarOutputDir();
+    String tableName = options.getTableName();
+    if (null != tableName && tableName.length() > 0) {
+      return jarOutDir + tableName + ".jar";
+    } else if (this.sources.size() == 1) {
+      // if we only have one source file, find it's base name,
+      // turn "foo.java" into "foo", and then return jarDir + "foo" + ".jar"
+      String srcFileName = this.sources.get(0);
+      String basename = new File(srcFileName).getName();
+      String [] parts = basename.split("\\.");
+      String preExtPart = parts[0];
+      return jarOutDir + preExtPart + ".jar";
+    } else {
+      return jarOutDir + "sqoop.jar";
+    }
+  }
+
+  /**
+   * Create an output jar file to use when executing MapReduce jobs
+   */
+  public void jar() throws IOException {
+    String jarOutDir = options.getJarOutputDir();
+    List<File> outDirEntries =  FileListing.getFileListing(new File(jarOutDir));
+
+    String jarFilename = getJarFilename();
+
+    LOG.info("Writing jar file: " + jarFilename);
+
+    findThisJar();
+    File jarFileObj = new File(jarFilename);
+    if (jarFileObj.exists()) {
+      if (!jarFileObj.delete()) {
+        LOG.warn("Could not remove existing jar file: " + jarFilename);
+      }
+    }
+
+    FileOutputStream fstream = null;
+    JarOutputStream jstream = null;
+    try {
+      fstream = new FileOutputStream(jarFilename);
+      jstream = new JarOutputStream(fstream);
+
+      // for each input class file, create a zipfile entry for it,
+      // read the file into a buffer, and write it to the jar file.
+
+      for (File entry : outDirEntries) {
+        if (entry.equals(jarFileObj)) {
+          // don't include our own jar!
+          continue;
+        } else if (entry.isDirectory()) {
+          // don't write entries for directories
+          continue;
+        } else {
+          String fileName = entry.getName();
+
+          boolean include = fileName.endsWith(".class")
+              && sources.contains(
+              fileName.substring(0, fileName.length() - ".class".length()) + ".java");
+
+          if (include) {
+            // include this file.
+
+            // chomp off the portion of the full path that is shared
+            // with the base directory where class files were put;
+            // we only record the subdir parts in the zip entry.
+            String fullPath = entry.getAbsolutePath();
+            String chompedPath = fullPath.substring(jarOutDir.length());
+
+            LOG.debug("Got classfile: " + entry.getPath() + " -> " + chompedPath);
+            ZipEntry ze = new ZipEntry(chompedPath);
+            jstream.putNextEntry(ze);
+            copyFileToStream(entry, jstream);
+            jstream.closeEntry();
+          }
+        }
+      }
+
+      // put our own jar in there in its lib/ subdir
+      String thisJarFile = findThisJar();
+      if (null != thisJarFile) {
+        File thisJarFileObj = new File(thisJarFile);
+        String thisJarBasename = thisJarFileObj.getName();
+        String thisJarEntryName = "lib" + File.separator + thisJarBasename;
+        ZipEntry ze = new ZipEntry(thisJarEntryName);
+        jstream.putNextEntry(ze);
+        copyFileToStream(thisJarFileObj, jstream);
+        jstream.closeEntry();
+      } else {
+        // couldn't find our own jar (we were running from .class files?)
+        LOG.warn("Could not find jar for Sqoop; MapReduce jobs may not run correctly.");
+      }
+    } finally {
+      IOUtils.closeStream(jstream);
+      IOUtils.closeStream(fstream);
+    }
+  }
+
+
+  private static final int BUFFER_SZ = 4096;
+
+  /**
+   * utility method to copy a .class file into the jar stream.
+   * @param f
+   * @param ostream
+   * @throws IOException
+   */
+  private void copyFileToStream(File f, OutputStream ostream) throws IOException {
+    FileInputStream fis = new FileInputStream(f);
+    byte [] buffer = new byte[BUFFER_SZ];
+    try {
+      while (true) {
+        int bytesReceived = fis.read(buffer);
+        if (bytesReceived < 1) {
+          break;
+        }
+
+        ostream.write(buffer, 0, bytesReceived);
+      }
+    } finally {
+      fis.close();
+    }
+  }
+
+  private String findThisJar() {
+    return findJarForClass(CompilationManager.class);
+  }
+
+  // method mostly cloned from o.a.h.mapred.JobConf.findContainingJar()
+  private String findJarForClass(Class<? extends Object> classObj) {
+    ClassLoader loader = classObj.getClassLoader();
+    String classFile = classObj.getName().replaceAll("\\.", "/") + ".class";
+    try {
+      for (Enumeration<URL> itr = loader.getResources(classFile);
+          itr.hasMoreElements();) {
+        URL url = (URL) itr.nextElement();
+        if ("jar".equals(url.getProtocol())) {
+          String toReturn = url.getPath();
+          if (toReturn.startsWith("file:")) {
+            toReturn = toReturn.substring("file:".length());
+          }
+          toReturn = URLDecoder.decode(toReturn, "UTF-8");
+          return toReturn.replaceAll("!.*$", "");
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    return null;
+  }
+}

+ 84 - 0
src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ClassLoaderStack.java

@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLClassLoader;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Allows you to add and remove jar-files from the running JVM by
+ * instantiating classloaders for them.
+ *
+ * 
+ *
+ */
+public final class ClassLoaderStack {
+
+  public static final Log LOG = LogFactory.getLog(ClassLoaderStack.class.getName());
+
+  private ClassLoaderStack() {
+  }
+
+  /**
+   * Sets the classloader for the current thread
+   */
+  public static void setCurrentClassLoader(ClassLoader cl) {
+    LOG.info("Restoring classloader: " + cl.toString());
+    Thread.currentThread().setContextClassLoader(cl);
+  }
+
+  /**
+   * Adds a ClassLoader to the top of the stack that will load from the Jar file
+   * of your choice. Returns the previous classloader so you can restore it
+   * if need be, later.
+   *
+   * @param jarFile The filename of a jar file that you want loaded into this JVM
+   * @param tableClassName The name of the class to load immediately (optional)
+   */
+  public static ClassLoader addJarFile(String jarFile, String testClassName)
+      throws IOException {
+
+    // load the classes from the ORM JAR file into the current VM
+    ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader();
+    String urlPath = "jar:file://" + new File(jarFile).getAbsolutePath() + "!/";
+    LOG.debug("Attempting to load jar through URL: " + urlPath);
+    LOG.debug("Previous classloader is " + prevClassLoader);
+    URL [] jarUrlArray = {new URL(urlPath)};
+    URLClassLoader cl = URLClassLoader.newInstance(jarUrlArray, prevClassLoader);
+    try {
+      if (null != testClassName) {
+        // try to load a class from the jar to force loading now.
+        Class.forName(testClassName, true, cl);
+      }
+      LOG.info("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);
+    Thread.currentThread().setContextClassLoader(cl);
+    return prevClassLoader;
+  }
+}

+ 104 - 0
src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/FileListing.java

@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.util;
+
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.io.File;
+import java.io.FileNotFoundException;
+
+/**
+* Recursive file listing under a specified directory.
+*
+* Taken from http://www.javapractices.com/topic/TopicAction.do?Id=68
+* Used under the terms of the CC Attribution license:
+* http://creativecommons.org/licenses/by/3.0/
+*
+* Method by Alex Wong (javapractices.com)
+*/
+public final class FileListing {
+
+  private FileListing() { }
+
+  /**
+  * Demonstrate use.
+  *
+  * @param aArgs - <tt>aArgs[0]</tt> is the full name of an existing
+  * directory that can be read.
+  */
+  public static void main(String... aArgs) throws FileNotFoundException {
+    File startingDirectory = new File(aArgs[0]);
+    List<File> files = FileListing.getFileListing(startingDirectory);
+
+    //print out all file names, in the the order of File.compareTo()
+    for (File file : files) {
+      System.out.println(file);
+    }
+  }
+
+  /**
+  * Recursively walk a directory tree and return a List of all
+  * Files found; the List is sorted using File.compareTo().
+  *
+  * @param aStartingDir is a valid directory, which can be read.
+  */
+  public static List<File> getFileListing(File aStartingDir) throws FileNotFoundException {
+    validateDirectory(aStartingDir);
+    List<File> result = getFileListingNoSort(aStartingDir);
+    Collections.sort(result);
+    return result;
+  }
+
+  // PRIVATE //
+  private static List<File> getFileListingNoSort(File aStartingDir) throws FileNotFoundException {
+    List<File> result = new ArrayList<File>();
+    File[] filesAndDirs = aStartingDir.listFiles();
+    List<File> filesDirs = Arrays.asList(filesAndDirs);
+    for (File file : filesDirs) {
+      result.add(file); //always add, even if directory
+      if (!file.isFile()) {
+        //must be a directory
+        //recursive call!
+        List<File> deeperList = getFileListingNoSort(file);
+        result.addAll(deeperList);
+      }
+    }
+    return result;
+  }
+
+  /**
+  * Directory is valid if it exists, does not represent a file, and can be read.
+  */
+  private static void validateDirectory(File aDirectory) throws FileNotFoundException {
+    if (aDirectory == null) {
+      throw new IllegalArgumentException("Directory should not be null.");
+    }
+    if (!aDirectory.exists()) {
+      throw new FileNotFoundException("Directory does not exist: " + aDirectory);
+    }
+    if (!aDirectory.isDirectory()) {
+      throw new IllegalArgumentException("Is not a directory: " + aDirectory);
+    }
+    if (!aDirectory.canRead()) {
+      throw new IllegalArgumentException("Directory cannot be read: " + aDirectory);
+    }
+  }
+}

+ 44 - 0
src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ImportError.java

@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.util;
+
+/**
+ * General error during import process.
+ *
+ * 
+ */
+@SuppressWarnings("serial")
+public class ImportError extends Exception {
+
+  public ImportError() {
+    super("ImportError");
+  }
+
+  public ImportError(final String message) {
+    super(message);
+  }
+
+  public ImportError(final Throwable cause) {
+    super(cause);
+  }
+
+  public ImportError(final String message, final Throwable cause) {
+    super(message, cause);
+  }
+}

+ 152 - 0
src/contrib/sqoop/src/java/org/apache/hadoop/sqoop/util/ResultSetPrinter.java

@@ -0,0 +1,152 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.util;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Utility methods to format and print ResultSet objects
+ * 
+ *
+ */
+public class ResultSetPrinter {
+
+  public static final Log LOG = LogFactory.getLog(ResultSetPrinter.class.getName());
+
+  // 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
+   */
+  private static void printPadded(StringBuilder sb, String str, int width) {
+    int numPad;
+    if (null == str) {
+      sb.append("(null)");
+      numPad = width - "(null)".length();
+    } else {
+      sb.append(str);
+      numPad = width - str.length();
+    }
+
+    for (int i = 0; i < numPad; i++) {
+      sb.append(' ');
+    }
+  }
+
+
+  /**
+   * 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 {
+    try {
+      StringBuilder sbNames = new StringBuilder();
+      int cols = results.getMetaData().getColumnCount();
+
+      int [] colWidths = new int[cols];
+      ResultSetMetaData metadata = results.getMetaData();
+      for (int i = 1; i < cols + 1; i++) {
+        String colName = metadata.getColumnName(i);
+        colWidths[i - 1] = Math.min(metadata.getColumnDisplaySize(i), MAX_COL_WIDTH);
+        if (colName == null || colName.equals("")) {
+          colName = metadata.getColumnLabel(i) + "*";
+        }
+        printPadded(sbNames, colName, colWidths[i - 1]);
+        sbNames.append(COL_SEPARATOR);
+      }
+      sbNames.append('\n');
+
+      StringBuilder sbPad = new StringBuilder();
+      for (int i = 0; i < cols; i++) {
+        for (int j = 0; j < COL_SEPARATOR.length() + colWidths[i]; j++) {
+          sbPad.append('-');
+        }
+      }
+      sbPad.append('\n');
+
+      sendToStream(sbPad, os);
+      sendToStream(sbNames, os);
+      sendToStream(sbPad, os);
+
+      while (results.next())  {
+        StringBuilder sb = new StringBuilder();
+        for (int i = 1; i < cols + 1; i++) {
+          printPadded(sb, results.getString(i), colWidths[i - 1]);
+          sb.append(COL_SEPARATOR);
+        }
+        sb.append('\n');
+        sendToStream(sb, os);
+      }
+
+      sendToStream(sbPad, os);
+    } catch (SQLException sqlException) {
+      LOG.error("Error reading from database: " + sqlException.toString());
+    }
+  }
+
+}
+

+ 52 - 0
src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/AllTests.java

@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop;
+
+import org.apache.hadoop.sqoop.manager.TestHsqldbManager;
+import org.apache.hadoop.sqoop.manager.TestSqlManager;
+import org.apache.hadoop.sqoop.orm.TestClassWriter;
+
+import junit.framework.Test;
+import junit.framework.TestSuite;
+
+/**
+ * All tests for Sqoop (org.apache.hadoop.sqoop)
+ *
+ * 
+ */
+public final class AllTests  {
+
+  private AllTests() { }
+
+  public static Test suite() {
+    TestSuite suite = new TestSuite("Tests for org.apache.hadoop.sqoop");
+
+    suite.addTestSuite(TestAllTables.class);
+    suite.addTestSuite(TestHsqldbManager.class);
+    suite.addTestSuite(TestSqlManager.class);
+    suite.addTestSuite(TestClassWriter.class);
+    suite.addTestSuite(TestColumnTypes.class);
+    suite.addTestSuite(TestMultiCols.class);
+    suite.addTestSuite(TestOrderBy.class);
+
+    return suite;
+  }
+
+}
+

+ 128 - 0
src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestAllTables.java

@@ -0,0 +1,128 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.Before;
+
+import org.apache.hadoop.sqoop.testutil.HsqldbTestServer;
+import org.apache.hadoop.sqoop.testutil.ImportJobTestCase;
+
+/**
+ * Test the --all-tables functionality that can import multiple tables.
+ * ;
+ * 
+ *
+ */
+public class TestAllTables extends ImportJobTestCase {
+
+  /**
+   * Create the argv to pass to Sqoop
+   * @return the argv as an array of strings.
+   */
+  private String [] getArgv(boolean includeHadoopFlags) {
+    ArrayList<String> args = new ArrayList<String>();
+
+    if (includeHadoopFlags) {
+      args.add("-D");
+      args.add("mapred.job.tracker=local");
+      args.add("-D");
+      args.add("mapred.map.tasks=1");
+      args.add("-D");
+      args.add("fs.default.name=file:///");
+    }
+
+    args.add("--all-tables");
+    args.add("--warehouse-dir");
+    args.add(getWarehouseDir());
+    args.add("--connect");
+    args.add(HsqldbTestServer.getUrl());
+
+    return args.toArray(new String[0]);
+  }
+
+  /** the names of the tables we're creating. */
+  private List<String> tableNames;
+
+  /** The strings to inject in the (ordered) tables */
+  private List<String> expectedStrings;
+
+  @Before
+  public void setUp() {
+    // start the server
+    super.setUp();
+
+    // throw away TWOINTTABLE and things we don't care about.
+    try {
+      this.getTestServer().dropExistingSchema();
+    } catch (SQLException sqlE) {
+      fail(sqlE.toString());
+    }
+
+    this.tableNames = new ArrayList<String>();
+    this.expectedStrings = new ArrayList<String>();
+
+    // create two tables.
+    this.expectedStrings.add("A winner");
+    this.expectedStrings.add("is you!");
+
+    for (String expectedStr: this.expectedStrings) {
+      this.createTableForColType("VARCHAR(32) PRIMARY KEY", "'" + expectedStr + "'");
+      this.tableNames.add(this.getTableName());
+      this.removeTableDir();
+      incrementTableNum();
+    }
+  }
+
+  public void testMultiTableImport() throws IOException {
+    String [] argv = getArgv(true);
+    runImport(argv);
+
+    Path warehousePath = new Path(this.getWarehouseDir());
+    for (String tableName : this.tableNames) {
+      Path tablePath = new Path(warehousePath, tableName);
+      Path filePath = new Path(tablePath, "part-00000");
+
+      // dequeue the expected value for this table. This
+      // list has the same order as the tableNames list.
+      String expectedVal = this.expectedStrings.get(0);
+      this.expectedStrings.remove(0);
+
+      BufferedReader reader = new BufferedReader(
+          new InputStreamReader(new FileInputStream(new File(filePath.toString()))));
+      try {
+        String line = reader.readLine();
+        assertEquals("Table " + tableName + " expected a different string",
+            expectedVal, line);
+      } finally {
+        IOUtils.closeStream(reader);
+      }
+    }
+  }
+}

+ 295 - 0
src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestColumnTypes.java

@@ -0,0 +1,295 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.Test;
+
+import org.apache.hadoop.sqoop.testutil.ImportJobTestCase;
+
+/**
+ * Test that each of the different SQL Column types that we support
+ * can, in fact, be imported into HDFS. Test that the writable
+ * that we expect to work, does.
+ *
+ * This requires testing:
+ * - That we can pull from the database into HDFS:
+ *    readFields(ResultSet), toString()
+ * - That we can pull from mapper to reducer:
+ *    write(DataOutput), readFields(DataInput)
+ * - And optionally, that we can push to the database:
+ *    write(PreparedStatement)
+ *
+ * 
+ *
+ */
+public class TestColumnTypes extends ImportJobTestCase {
+
+  public static final Log LOG = LogFactory.getLog(TestColumnTypes.class.getName());
+
+  /**
+   * Do a full verification test on the singleton value of a given type.
+   * @param colType  The SQL type to instantiate the column
+   * @param insertVal The SQL text to insert a value into the database
+   * @param returnVal The string representation of the value as extracted from the db
+   */
+  private void verifyType(String colType, String insertVal, String returnVal) {
+    verifyType(colType, insertVal, returnVal, returnVal);
+  }
+
+  /**
+   * Do a full verification test on the singleton value of a given type.
+   * @param colType  The SQL type to instantiate the column
+   * @param insertVal The SQL text to insert a value into the database
+   * @param returnVal The string representation of the value as extracted from the db
+   * @param seqFileVal The string representation of the value as extracted through
+   *        the DBInputFormat, serialized, and injected into a SequenceFile and put
+   *        through toString(). This may be slightly different than what ResultSet.getString()
+   *        returns, which is used by returnVal.
+   */
+  private void verifyType(String colType, String insertVal, String returnVal, String seqFileVal) {
+    createTableForColType(colType, insertVal);
+    verifyReadback(1, returnVal);
+    verifyImport(seqFileVal, null);
+  }
+
+  static final String STRING_VAL_IN = "'this is a short string'";
+  static final String STRING_VAL_OUT = "this is a short string";
+
+  @Test
+  public void testStringCol1() {
+    verifyType("VARCHAR(32)", STRING_VAL_IN, STRING_VAL_OUT);
+  }
+
+  @Test
+  public void testStringCol2() {
+    verifyType("CHAR(32)", STRING_VAL_IN, STRING_VAL_OUT);
+  }
+
+  @Test
+  public void testEmptyStringCol() {
+    verifyType("VARCHAR(32)", "''", "");
+  }
+
+  @Test
+  public void testNullStringCol() {
+    verifyType("VARCHAR(32)", "NULL", null);
+  }
+
+  @Test
+  public void testInt() {
+    verifyType("INTEGER", "42", "42");
+  }
+
+  @Test
+  public void testNullInt() {
+    verifyType("INTEGER", "NULL", null);
+  }
+
+  @Test
+  public void testBit1() {
+    verifyType("BIT", "1", "true");
+  }
+
+  @Test
+  public void testBit2() {
+    verifyType("BIT", "0", "false");
+  }
+
+  @Test
+  public void testBit3() {
+    verifyType("BIT", "false", "false");
+  }
+
+  @Test
+  public void testTinyInt1() {
+    verifyType("TINYINT", "0", "0");
+  }
+
+  @Test
+  public void testTinyInt2() {
+    verifyType("TINYINT", "42", "42");
+  }
+
+  @Test
+  public void testSmallInt1() {
+    verifyType("SMALLINT", "-1024", "-1024");
+  }
+
+  @Test
+  public void testSmallInt2() {
+    verifyType("SMALLINT", "2048", "2048");
+  }
+
+  @Test
+  public void testBigInt1() {
+    verifyType("BIGINT", "10000000000", "10000000000");
+  }
+
+  @Test
+  public void testReal1() {
+    verifyType("REAL", "256", "256.0");
+  }
+
+  @Test
+  public void testReal2() {
+    verifyType("REAL", "256.45", "256.45");
+  }
+
+  @Test
+  public void testFloat1() {
+    verifyType("FLOAT", "256", "256.0");
+  }
+
+  @Test
+  public void testFloat2() {
+    verifyType("FLOAT", "256.45", "256.45");
+  }
+
+  @Test
+  public void testDouble1() {
+    verifyType("DOUBLE", "-256", "-256.0");
+  }
+
+  @Test
+  public void testDouble2() {
+    verifyType("DOUBLE", "256.45", "256.45");
+  }
+
+  @Test
+  public void testDate1() {
+    verifyType("DATE", "'2009-1-12'", "2009-01-12");
+  }
+
+  @Test
+  public void testDate2() {
+    verifyType("DATE", "'2009-01-12'", "2009-01-12");
+  }
+
+  @Test
+  public void testDate3() {
+    verifyType("DATE", "'2009-04-24'", "2009-04-24");
+  }
+
+  @Test
+  public void testTime1() {
+    verifyType("TIME", "'12:24:00'", "12:24:00");
+  }
+
+  @Test
+  public void testTime2() {
+    verifyType("TIME", "'06:24:00'", "06:24:00");
+  }
+
+  @Test
+  public void testTime3() {
+    verifyType("TIME", "'6:24:00'", "06:24:00");
+  }
+
+  @Test
+  public void testTime4() {
+    verifyType("TIME", "'18:24:00'", "18:24:00");
+  }
+
+  @Test
+  public void testTimestamp1() {
+    verifyType("TIMESTAMP", "'2009-04-24 18:24:00'",
+        "2009-04-24 18:24:00.000000000",
+        "2009-04-24 18:24:00.0");
+  }
+
+  @Test
+  public void testTimestamp2() {
+    verifyType("TIMESTAMP", "'2009-04-24 18:24:00.0002'",
+        "2009-04-24 18:24:00.000200000",
+        "2009-04-24 18:24:00.0002");
+  }
+
+  @Test
+  public void testTimestamp3() {
+    verifyType("TIMESTAMP", "null", null);
+  }
+
+  @Test
+  public void testNumeric1() {
+    verifyType("NUMERIC", "1", "1");
+  }
+
+  @Test
+  public void testNumeric2() {
+    verifyType("NUMERIC", "-10", "-10");
+  }
+
+  @Test
+  public void testNumeric3() {
+    verifyType("NUMERIC", "3.14159", "3.14159");
+  }
+
+  @Test
+  public void testNumeric4() {
+    verifyType("NUMERIC", "30000000000000000000000000.14159", "30000000000000000000000000.14159");
+  }
+
+  @Test
+  public void testNumeric5() {
+    verifyType("NUMERIC", "999999999999999999999999999999.14159", "999999999999999999999999999999.14159");
+  }
+
+  @Test
+  public void testNumeric6() {
+    verifyType("NUMERIC", "-999999999999999999999999999999.14159", "-999999999999999999999999999999.14159");
+  }
+
+  @Test
+  public void testDecimal1() {
+    verifyType("DECIMAL", "1", "1");
+  }
+
+  @Test
+  public void testDecimal2() {
+    verifyType("DECIMAL", "-10", "-10");
+  }
+
+  @Test
+  public void testDecimal3() {
+    verifyType("DECIMAL", "3.14159", "3.14159");
+  }
+
+  @Test
+  public void testDecimal4() {
+    verifyType("DECIMAL", "30000000000000000000000000.14159", "30000000000000000000000000.14159");
+  }
+
+  @Test
+  public void testDecimal5() {
+    verifyType("DECIMAL", "999999999999999999999999999999.14159", "999999999999999999999999999999.14159");
+  }
+
+  @Test
+  public void testDecimal6() {
+    verifyType("DECIMAL", "-999999999999999999999999999999.14159", "-999999999999999999999999999999.14159");
+  }
+
+  @Test
+  public void testLongVarChar() {
+    verifyType("LONGVARCHAR", "'this is a long varchar'", "this is a long varchar");
+  }
+
+}

+ 214 - 0
src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestMultiCols.java

@@ -0,0 +1,214 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.sqoop.testutil.ImportJobTestCase;
+
+/**
+ * Test cases that import rows containing multiple columns,
+ * some of which may contain null values.
+ *
+ * Also test loading only selected columns from the db.
+ * 
+ *
+ */
+public class TestMultiCols extends ImportJobTestCase {
+
+  public static final Log LOG = LogFactory.getLog(TestMultiCols.class.getName());
+
+  /**
+   * Do a full import verification test on a table containing one row
+   * @param types the types of the columns to insert
+   * @param insertVals the SQL text to use to insert each value
+   * @param validateVals the text to expect when retrieving each value from
+   * the db
+   * @param validateLine the text to expect as a toString() of the entire row,
+   * as imported by the tool
+   * @param importColumns The list of columns to import
+   */
+  private void verifyTypes(String [] types , String [] insertVals,
+      String validateVals [], String validateLine) {
+    verifyTypes(types, insertVals, validateVals, validateLine, null);
+  }
+
+  private void verifyTypes(String [] types , String [] insertVals,
+      String validateVals [], String validateLine, String [] importColumns) {
+
+    createTableWithColTypes(types, insertVals);
+
+    int i = 0;
+    for (String val : validateVals) {
+      verifyReadback(++i, val);
+      LOG.debug("Verified column " + i + " as value: " + val);
+    }
+
+    verifyImport(validateLine, importColumns);
+    LOG.debug("Verified input line as " + validateLine + " -- ok!");
+  }
+
+  public void testThreeStrings() {
+    String [] types = { "VARCHAR(32)", "VARCHAR(32)", "VARCHAR(32)" };
+    String [] insertVals = { "'foo'", "'bar'", "'baz'" };
+    String [] validateVals = { "foo", "bar", "baz" };
+    String validateLine = "foo,bar,baz";
+
+    verifyTypes(types, insertVals, validateVals, validateLine);
+  }
+
+  public void testStringsWithNull1() {
+    String [] types = { "VARCHAR(32)", "VARCHAR(32)", "VARCHAR(32)" };
+    String [] insertVals = { "'foo'", "null", "'baz'" };
+    String [] validateVals = { "foo", null, "baz" };
+    String validateLine = "foo,null,baz";
+
+    verifyTypes(types, insertVals, validateVals, validateLine);
+  }
+
+  public void testStringsWithNull2() {
+    String [] types = { "VARCHAR(32)", "VARCHAR(32)", "VARCHAR(32)" };
+    String [] insertVals = { "null", "'foo'", "'baz'" };
+    String [] validateVals = { null, "foo", "baz" };
+    String validateLine = "null,foo,baz";
+
+    verifyTypes(types, insertVals, validateVals, validateLine);
+  }
+
+  public void testStringsWithNull3() {
+    String [] types = { "VARCHAR(32)", "VARCHAR(32)", "VARCHAR(32)" };
+    String [] insertVals = { "'foo'", "'baz'", "null"};
+    String [] validateVals = { "foo", "baz", null };
+    String validateLine = "foo,baz,null";
+
+    verifyTypes(types, insertVals, validateVals, validateLine);
+  }
+
+  public void testThreeInts() {
+    String [] types = { "INTEGER", "INTEGER", "INTEGER" };
+    String [] insertVals = { "1", "2", "3" };
+    String [] validateVals = { "1", "2", "3" };
+    String validateLine = "1,2,3";
+
+    verifyTypes(types, insertVals, validateVals, validateLine);
+  }
+
+  public void testIntsWithNulls() {
+    String [] types = { "INTEGER", "INTEGER", "INTEGER" };
+    String [] insertVals = { "1", "null", "3" };
+    String [] validateVals = { "1", null, "3" };
+    String validateLine = "1,null,3";
+
+    verifyTypes(types, insertVals, validateVals, validateLine);
+  }
+
+  public void testMixed1() {
+    String [] types = { "INTEGER", "VARCHAR(32)", "DATE" };
+    String [] insertVals = { "1", "'meep'", "'2009-12-31'" };
+    String [] validateVals = { "1", "meep", "2009-12-31" };
+    String validateLine = "1,meep,2009-12-31";
+
+    verifyTypes(types, insertVals, validateVals, validateLine);
+  }
+
+  public void testMixed2() {
+    String [] types = { "INTEGER", "VARCHAR(32)", "DATE" };
+    String [] insertVals = { "null", "'meep'", "'2009-12-31'" };
+    String [] validateVals = { null, "meep", "2009-12-31" };
+    String validateLine = "null,meep,2009-12-31";
+
+    verifyTypes(types, insertVals, validateVals, validateLine);
+  }
+
+  public void testMixed3() {
+    String [] types = { "INTEGER", "VARCHAR(32)", "DATE" };
+    String [] insertVals = { "1", "'meep'", "null" };
+    String [] validateVals = { "1", "meep", null };
+    String validateLine = "1,meep,null";
+
+    verifyTypes(types, insertVals, validateVals, validateLine);
+  }
+
+  public void testMixed4() {
+    String [] types = { "NUMERIC", "INTEGER", "NUMERIC" };
+    String [] insertVals = { "-42", "17", "33333333333333333333333.1714" };
+    String [] validateVals = { "-42", "17", "33333333333333333333333.1714" };
+    String validateLine = "-42,17,33333333333333333333333.1714";
+
+    verifyTypes(types, insertVals, validateVals, validateLine);
+  }
+
+  public void testMixed5() {
+    String [] types = { "NUMERIC", "INTEGER", "NUMERIC" };
+    String [] insertVals = { "null", "17", "33333333333333333333333.0" };
+    String [] validateVals = { null, "17", "33333333333333333333333.0" };
+    String validateLine = "null,17,33333333333333333333333.0";
+
+    verifyTypes(types, insertVals, validateVals, validateLine);
+  }
+
+  public void testMixed6() {
+    String [] types = { "NUMERIC", "INTEGER", "NUMERIC" };
+    String [] insertVals = { "33333333333333333333333", "17", "-42"};
+    String [] validateVals = { "33333333333333333333333", "17", "-42" };
+    String validateLine = "33333333333333333333333,17,-42";
+
+    verifyTypes(types, insertVals, validateVals, validateLine);
+  }
+
+  //////////////////////////////////////////////////////////////////////////
+  // the tests below here test the --columns parameter and ensure that
+  // we can selectively import only certain columns.
+  //////////////////////////////////////////////////////////////////////////
+
+  public void testSkipFirstCol() {
+    String [] types = { "NUMERIC", "INTEGER", "NUMERIC" };
+    String [] insertVals = { "33333333333333333333333", "17", "-42"};
+    String [] validateVals = { "33333333333333333333333", "17", "-42" };
+    String validateLine = "17,-42";
+
+    String [] loadCols = {"DATA_COL1", "DATA_COL2"};
+
+    verifyTypes(types, insertVals, validateVals, validateLine, loadCols);
+  }
+
+  public void testSkipSecondCol() {
+    String [] types = { "NUMERIC", "INTEGER", "NUMERIC" };
+    String [] insertVals = { "33333333333333333333333", "17", "-42"};
+    String [] validateVals = { "33333333333333333333333", "17", "-42" };
+    String validateLine = "33333333333333333333333,-42";
+
+    String [] loadCols = {"DATA_COL0", "DATA_COL2"};
+
+    verifyTypes(types, insertVals, validateVals, validateLine, loadCols);
+  }
+
+  public void testSkipThirdCol() {
+    String [] types = { "NUMERIC", "INTEGER", "NUMERIC" };
+    String [] insertVals = { "33333333333333333333333", "17", "-42"};
+    String [] validateVals = { "33333333333333333333333", "17", "-42" };
+    String validateLine = "33333333333333333333333,17";
+
+    String [] loadCols = {"DATA_COL0", "DATA_COL1"};
+
+    verifyTypes(types, insertVals, validateVals, validateLine, loadCols);
+  }
+
+}

+ 162 - 0
src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/TestOrderBy.java

@@ -0,0 +1,162 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import org.apache.hadoop.sqoop.ImportOptions.InvalidOptionsException;
+import org.apache.hadoop.sqoop.orm.CompilationManager;
+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.util.ClassLoaderStack;
+
+/**
+ * Test that --order-by works
+ * 
+ *
+ */
+public class TestOrderBy extends ImportJobTestCase {
+
+  /**
+   * Create the argv to pass to Sqoop
+   * @return the argv as an array of strings.
+   */
+  private String [] getArgv(boolean includeHadoopFlags, String [] colNames, String orderByCol) {
+    String columnsString = "";
+    for (String col : colNames) {
+      columnsString += col + ",";
+    }
+
+    ArrayList<String> args = new ArrayList<String>();
+
+    if (includeHadoopFlags) {
+      args.add("-D");
+      args.add("mapred.job.tracker=local");
+      args.add("-D");
+      args.add("mapred.map.tasks=1");
+      args.add("-D");
+      args.add("fs.default.name=file:///");
+    }
+
+    args.add("--table");
+    args.add(HsqldbTestServer.getTableName());
+    args.add("--columns");
+    args.add(columnsString);
+    args.add("--order-by");
+    args.add(orderByCol);
+    args.add("--warehouse-dir");
+    args.add(getWarehouseDir());
+    args.add("--connect");
+    args.add(HsqldbTestServer.getUrl());
+    args.add("--as-sequencefile");
+
+    return args.toArray(new String[0]);
+  }
+
+  // this test just uses the two int table.
+  protected String getTableName() {
+    return HsqldbTestServer.getTableName();
+  }
+
+
+  /**
+   * Given a comma-delimited list of integers, grab and parse the first int
+   * @param str a comma-delimited list of values, the first of which is an int.
+   * @return the first field in the string, cast to int
+   */
+  private int getFirstInt(String str) {
+    String [] parts = str.split(",");
+    return Integer.parseInt(parts[0]);
+  }
+
+  public void runOrderByTest(String orderByCol, String firstValStr, int expectedSum)
+      throws IOException {
+
+    String [] columns = HsqldbTestServer.getFieldNames();
+    ClassLoader prevClassLoader = null;
+    SequenceFile.Reader reader = null;
+
+    String [] argv = getArgv(true, columns, orderByCol);
+    runImport(argv);
+    try {
+      ImportOptions opts = new ImportOptions();
+      opts.parse(getArgv(false, columns, orderByCol));
+
+      CompilationManager compileMgr = new CompilationManager(opts);
+      String jarFileName = compileMgr.getJarFilename();
+
+      prevClassLoader = ClassLoaderStack.addJarFile(jarFileName, getTableName());
+
+      reader = SeqFileReader.getSeqFileReader(getDataFilePath().toString());
+
+      // here we can actually instantiate (k, v) pairs.
+      Configuration conf = new Configuration();
+      Object key = ReflectionUtils.newInstance(reader.getKeyClass(), conf);
+      Object val = ReflectionUtils.newInstance(reader.getValueClass(), conf);
+
+      if (reader.next(key) == null) {
+        fail("Empty SequenceFile during import");
+      }
+
+      // make sure that the value we think should be at the top, is.
+      reader.getCurrentValue(val);
+      assertEquals("Invalid ordering within sorted SeqFile", firstValStr, val.toString());
+
+      // We know that these values are two ints separated by a ',' character.
+      // Since this is all dynamic, though, we don't want to actually link against
+      // the class and use its methods. So we just parse this back into int fields manually.
+      // Sum them up and ensure that we get the expected total for the first column, to
+      // verify that we got all the results from the db into the file.
+      int curSum = getFirstInt(val.toString());
+
+      // now sum up everything else in the file.
+      while (reader.next(key) != null) {
+        reader.getCurrentValue(val);
+        curSum += getFirstInt(val.toString());
+      }
+
+      assertEquals("Total sum of first db column mismatch", expectedSum, curSum);
+    } catch (InvalidOptionsException ioe) {
+      fail(ioe.toString());
+    } finally {
+      IOUtils.closeStream(reader);
+
+      if (null != prevClassLoader) {
+        ClassLoaderStack.setCurrentClassLoader(prevClassLoader);
+      }
+    }
+  }
+
+  public void testOrderByFirstCol() throws IOException {
+    String orderByCol = "INTFIELD1";
+    runOrderByTest(orderByCol, "1,8", HsqldbTestServer.getFirstColSum());
+  }
+
+  public void testOrderBySecondCol() throws IOException {
+    String orderByCol = "INTFIELD2";
+    runOrderByTest(orderByCol, "7,2", HsqldbTestServer.getFirstColSum());
+  }
+}

+ 83 - 0
src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/TestHsqldbManager.java

@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.manager;
+
+import java.sql.SQLException;
+
+import junit.framework.TestCase;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.hadoop.sqoop.testutil.HsqldbTestServer;
+
+/**
+ * Test HsqldbManager-specific functionality that overrides SqlManager behavior
+ *
+ * 
+ */
+public class TestHsqldbManager extends TestCase {
+
+  public static final Log LOG = LogFactory.getLog(TestHsqldbManager.class.getName());
+
+  // instance variables populated during setUp, used during tests
+  private HsqldbTestServer testServer;
+  private ConnManager manager;
+
+  @Before
+  public void setUp() {
+    testServer = new HsqldbTestServer();
+    try {
+      testServer.resetServer();
+    } catch (SQLException sqlE) {
+      LOG.error("Got SQLException: " + sqlE.toString());
+      fail("Got SQLException: " + sqlE.toString());
+    } catch (ClassNotFoundException cnfe) {
+      LOG.error("Could not find class for db driver: " + cnfe.toString());
+      fail("Could not find class for db driver: " + cnfe.toString());
+    }
+
+    manager = testServer.getManager();
+  }
+
+  @After
+  public void tearDown() {
+    try {
+      manager.close();
+    } catch (SQLException sqlE) {
+      LOG.error("Got SQLException: " + sqlE.toString());
+      fail("Got SQLException: " + sqlE.toString());
+    }
+  }
+
+  // note: hsql returns only the "PUBLIC" schema name; not individual user db names.
+  @Test
+  public void testListDatabases() {
+    String [] databases = manager.listDatabases();
+
+    assertNotNull("manager returned no database list", databases);
+    assertEquals("Database list should be length 1", 1, databases.length);
+    assertEquals(HsqldbTestServer.getSchemaName(), databases[0]);
+  }
+
+}
+

+ 220 - 0
src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/manager/TestSqlManager.java

@@ -0,0 +1,220 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.manager;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.Map;
+
+import junit.framework.TestCase;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.hadoop.sqoop.testutil.HsqldbTestServer;
+
+/**
+ * Test methods of the generic SqlManager implementation.
+ *
+ * 
+ *
+ */
+public class TestSqlManager extends TestCase {
+
+  public static final Log LOG = LogFactory.getLog(TestSqlManager.class.getName());
+
+  /** the name of a table that doesn't exist. */
+  static final String MISSING_TABLE = "MISSING_TABLE";
+
+  // instance variables populated during setUp, used during tests
+  private HsqldbTestServer testServer;
+  private ConnManager manager;
+
+  @Before
+  public void setUp() {
+    testServer = new HsqldbTestServer();
+    try {
+      testServer.resetServer();
+    } catch (SQLException sqlE) {
+      LOG.error("Got SQLException: " + sqlE.toString());
+      fail("Got SQLException: " + sqlE.toString());
+    } catch (ClassNotFoundException cnfe) {
+      LOG.error("Could not find class for db driver: " + cnfe.toString());
+      fail("Could not find class for db driver: " + cnfe.toString());
+    }
+
+    manager = testServer.getManager();
+  }
+
+  @After
+  public void tearDown() {
+    try {
+      manager.close();
+    } catch (SQLException sqlE) {
+      LOG.error("Got SQLException: " + sqlE.toString());
+      fail("Got SQLException: " + sqlE.toString());
+    }
+  }
+
+  @Test
+  public void testListColNames() {
+    String [] colNames = manager.getColumnNames(HsqldbTestServer.getTableName());
+    assertNotNull("manager returned no colname list", colNames);
+    assertEquals("Table list should be length 2", 2, colNames.length);
+    String [] knownFields = HsqldbTestServer.getFieldNames();
+    for (int i = 0; i < colNames.length; i++) {
+      assertEquals(knownFields[i], colNames[i]);
+    }
+  }
+
+  @Test
+  public void testListColTypes() {
+    Map<String, Integer> types = manager.getColumnTypes(HsqldbTestServer.getTableName());
+
+    assertNotNull("manager returned no types map", types);
+    assertEquals("Map should be size=2", 2, types.size());
+    assertEquals(types.get("INTFIELD1").intValue(), Types.INTEGER);
+    assertEquals(types.get("INTFIELD2").intValue(), Types.INTEGER);
+  }
+
+  @Test
+  public void testMissingTableColNames() {
+    String [] colNames = manager.getColumnNames(MISSING_TABLE);
+    assertNull("No column names should be returned for missing table", colNames);
+  }
+
+  @Test
+  public void testMissingTableColTypes() {
+    Map<String, Integer> colTypes = manager.getColumnTypes(MISSING_TABLE);
+    assertNull("No column types should be returned for missing table", colTypes);
+  }
+
+  @Test
+  public void testListTables() {
+    String [] tables = manager.listTables();
+    for (String table : tables) {
+      System.err.println("Got table: " + table);
+    }
+    assertNotNull("manager returned no table list", tables);
+    assertEquals("Table list should be length 1", 1, tables.length);
+    assertEquals(HsqldbTestServer.getTableName(), tables[0]);
+  }
+
+  // constants related to testReadTable()
+  final static int EXPECTED_NUM_ROWS = 4;
+  final static int EXPECTED_COL1_SUM = 16;
+  final static int EXPECTED_COL2_SUM = 20;
+
+  @Test
+  public void testReadTable() {
+    try {
+      ResultSet results = manager.readTable(HsqldbTestServer.getTableName(),
+          HsqldbTestServer.getFieldNames());
+
+      assertNotNull("ResultSet from readTable() is null!", results);
+
+      ResultSetMetaData metaData = results.getMetaData();
+      assertNotNull("ResultSetMetadata is null in readTable()", metaData);
+
+      // ensure that we get the correct number of columns back
+      assertEquals("Number of returned columns was unexpected!", metaData.getColumnCount(),
+          HsqldbTestServer.getFieldNames().length);
+
+      // should get back 4 rows. They are:
+      // 1 2
+      // 3 4
+      // 5 6
+      // 7 8
+      // .. so while order isn't guaranteed, we should get back 16 on the left and 20 on the right.
+      int sumCol1 = 0, sumCol2 = 0, rowCount = 0;
+      while (results.next()) {
+        rowCount++;
+        sumCol1 += results.getInt(1);
+        sumCol2 += results.getInt(2);
+      }
+
+      assertEquals("Expected 4 rows back", EXPECTED_NUM_ROWS, rowCount);
+      assertEquals("Expected left sum of 16", EXPECTED_COL1_SUM, sumCol1);
+      assertEquals("Expected right sum of 20", EXPECTED_COL2_SUM, sumCol2);
+    } catch (SQLException sqlException) {
+      fail("SQL Exception: " + sqlException.toString());
+    }
+  }
+
+  @Test
+  public void testReadMissingTable() {
+    try {
+      String [] colNames = { "*" };
+      ResultSet results = manager.readTable(MISSING_TABLE, colNames);
+      assertNull("Expected null resultset from readTable(MISSING_TABLE)", results);
+    } catch (SQLException sqlException) {
+      // we actually expect this. pass.
+    }
+  }
+
+  @Test
+  public void getPrimaryKeyFromMissingTable() {
+    String primaryKey = manager.getPrimaryKey(MISSING_TABLE);
+    assertNull("Expected null pkey for missing table", primaryKey);
+  }
+
+  @Test
+  public void getPrimaryKeyFromTableWithoutKey() {
+    String primaryKey = manager.getPrimaryKey(HsqldbTestServer.getTableName());
+    assertNull("Expected null pkey for table without key", primaryKey);
+  }
+
+  // constants for getPrimaryKeyFromTable()
+  static final String TABLE_WITH_KEY = "TABLE_WITH_KEY";
+  static final String KEY_FIELD_NAME = "KEYFIELD";
+
+  @Test
+  public void getPrimaryKeyFromTable() {
+    // first, create a table with a primary key
+    Connection conn = null;
+    try {
+      conn = testServer.getConnection();
+      PreparedStatement statement = conn.prepareStatement(
+          "CREATE TABLE " + TABLE_WITH_KEY + "(" + KEY_FIELD_NAME
+          + " INT NOT NULL PRIMARY KEY, foo INT)",
+          ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+      statement.executeUpdate();
+    } catch (SQLException sqlException) {
+      fail("Could not create table with primary key: " + sqlException.toString());
+    } finally {
+      if (null != conn) {
+        try {
+          conn.close();
+        } catch (SQLException sqlE) {
+          LOG.warn("Got SQLException during close: " + sqlE.toString());
+        }
+      }
+    }
+
+    String primaryKey = manager.getPrimaryKey(TABLE_WITH_KEY);
+    assertEquals("Expected null pkey for table without key", primaryKey, KEY_FIELD_NAME);
+  }
+}

+ 148 - 0
src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/orm/TestClassWriter.java

@@ -0,0 +1,148 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.orm;
+
+import java.io.File;
+import java.io.IOException;
+import java.sql.SQLException;
+
+import junit.framework.TestCase;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.ImportOptions.InvalidOptionsException;
+import org.apache.hadoop.sqoop.manager.ConnManager;
+import org.apache.hadoop.sqoop.testutil.DirUtil;
+import org.apache.hadoop.sqoop.testutil.HsqldbTestServer;
+import org.apache.hadoop.sqoop.testutil.ImportJobTestCase;
+
+/**
+ * Test that the ClassWriter generates Java classes based on the given table,
+ * which compile.
+ *
+ * 
+ */
+public class TestClassWriter extends TestCase {
+
+  public static final Log LOG =
+      LogFactory.getLog(TestClassWriter.class.getName());
+
+  // instance variables populated during setUp, used during tests
+  private HsqldbTestServer testServer;
+  private ConnManager manager;
+  private ImportOptions options;
+
+  @Before
+  public void setUp() {
+    testServer = new HsqldbTestServer();
+    try {
+      testServer.resetServer();
+    } catch (SQLException sqlE) {
+      LOG.error("Got SQLException: " + sqlE.toString());
+      fail("Got SQLException: " + sqlE.toString());
+    } catch (ClassNotFoundException cnfe) {
+      LOG.error("Could not find class for db driver: " + cnfe.toString());
+      fail("Could not find class for db driver: " + cnfe.toString());
+    }
+
+    manager = testServer.getManager();
+    options = testServer.getImportOptions();
+  }
+
+  @After
+  public void tearDown() {
+    try {
+      manager.close();
+    } catch (SQLException sqlE) {
+      LOG.error("Got SQLException: " + sqlE.toString());
+      fail("Got SQLException: " + sqlE.toString());
+    }
+  }
+
+  static final String CODE_GEN_DIR = ImportJobTestCase.TEMP_BASE_DIR + "sqoop/test/codegen";
+  static final String JAR_GEN_DIR = ImportJobTestCase.TEMP_BASE_DIR + "sqoop/test/jargen";
+
+  /**
+   * Test that we can generate code. Test that we can redirect the --outdir and --bindir too.
+   */
+  @Test
+  public void testCodeGen() {
+
+    // sanity check: make sure we're in a tmp dir before we blow anything away.
+    assertTrue("Test generates code in non-tmp dir!",
+        CODE_GEN_DIR.startsWith(ImportJobTestCase.TEMP_BASE_DIR));
+    assertTrue("Test generates jars in non-tmp dir!",
+        JAR_GEN_DIR.startsWith(ImportJobTestCase.TEMP_BASE_DIR));
+
+    // start out by removing these directories ahead of time
+    // to ensure that this is truly generating the code.
+    File codeGenDirFile = new File(CODE_GEN_DIR);
+    File classGenDirFile = new File(JAR_GEN_DIR);
+
+    if (codeGenDirFile.exists()) {
+      DirUtil.deleteDir(codeGenDirFile);
+    }
+
+    if (classGenDirFile.exists()) {
+      DirUtil.deleteDir(classGenDirFile);
+    }
+
+    // Set the option strings in an "argv" to redirect our srcdir and bindir
+    String [] argv = {
+        "--bindir",
+        JAR_GEN_DIR,
+        "--outdir",
+        CODE_GEN_DIR
+    };
+
+    try {
+      options.parse(argv);
+    } catch (InvalidOptionsException ioe) {
+      LOG.error("Could not parse options: " + ioe.toString());
+    }
+
+    CompilationManager compileMgr = new CompilationManager(options);
+    ClassWriter writer = new ClassWriter(options, manager, HsqldbTestServer.getTableName(),
+        compileMgr);
+
+    try {
+      writer.generate();
+      compileMgr.compile();
+      compileMgr.jar();
+    } catch (IOException ioe) {
+      LOG.error("Got IOException: " + ioe.toString());
+      fail("Got IOException: " + ioe.toString());
+    }
+
+    File tableFile = new File(codeGenDirFile, HsqldbTestServer.getTableName() + ".java");
+    assertTrue("Cannot find generated source file for table!", tableFile.exists());
+
+    File tableClassFile = new File(classGenDirFile, HsqldbTestServer.getTableName() + ".class");
+    assertTrue("Cannot find generated class file for table!", tableClassFile.exists());
+
+    File jarFile = new File(compileMgr.getJarFilename());
+    assertTrue("Cannot find compiled jar", jarFile.exists());
+  }
+}
+

+ 57 - 0
src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/DirUtil.java

@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.testutil;
+
+import java.io.File;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Misc directory operations
+ * 
+ *
+ */
+public final class DirUtil {
+
+  public static final Log LOG = LogFactory.getLog(DirUtil.class.getName());
+
+  /**
+   * recursively delete a dir and its children.
+   * @param dir
+   * @return true on succesful removal of a dir
+   */
+  public static boolean deleteDir(File dir) {
+    if (dir.isDirectory()) {
+      String [] children = dir.list();
+      for (int i = 0; i < children.length; i++) {
+        File f = new File(dir, children[i]);
+        boolean success = deleteDir(f);
+        if (!success) {
+          LOG.warn("Could not delete " + f.getAbsolutePath());
+          return false;
+        }
+      }
+    }
+
+    // The directory is now empty so delete it too.
+    return dir.delete();
+  }
+
+}

+ 243 - 0
src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/HsqldbTestServer.java

@@ -0,0 +1,243 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.testutil;
+
+import java.util.Arrays;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.hsqldb.Server;
+
+import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.manager.ConnManager;
+import org.apache.hadoop.sqoop.manager.HsqldbManager;
+
+/**
+ * Create a simple hsqldb server and schema to use for testing.
+ * 
+ *
+ */
+public class HsqldbTestServer {
+  public static final Log LOG =
+    LogFactory.getLog(HsqldbTestServer.class.getName());
+
+  // singleton server instance.
+  private static Server server;
+
+  // When we create databases in HSqlDb, where do we put the files?
+  private static final String DATABASE_DATA_DIR = "./hsqldb-data";
+
+  private static final String DATABASE_NAME = "db1";
+
+  // hsqldb always capitalizes table and column names
+  private static final String DUMMY_TABLE_NAME = "TWOINTTABLE";
+  private static final String [] TWO_INT_TABLE_FIELDS = {"INTFIELD1", "INTFIELD2"};
+
+  private static final String EMPLOYEE_TABLE_NAME = "EMPLOYEES";
+
+  private static final String DB_URL = "jdbc:hsqldb:hsql://localhost/" + DATABASE_NAME;
+  private static final String DRIVER_CLASS = "org.hsqldb.jdbcDriver";
+
+  // all user-created HSQLDB tables are in the "PUBLIC" schema when connected to a database.
+  private static final String HSQLDB_SCHEMA_NAME = "PUBLIC";
+
+  public static String getSchemaName() {
+    return HSQLDB_SCHEMA_NAME;
+  }
+
+  public static String [] getFieldNames() {
+    return Arrays.copyOf(TWO_INT_TABLE_FIELDS, TWO_INT_TABLE_FIELDS.length);
+  }
+
+  public static String getUrl() {
+    return DB_URL;
+  }
+
+  public static String getTableName() {
+    return DUMMY_TABLE_NAME;
+  }
+
+  public static String getDatabaseName() {
+    return DATABASE_NAME;
+  }
+
+  /**
+   * start the server
+   */
+  public void start() {
+    if (null == server) {
+      LOG.info("Starting new hsqldb server; database=" + DATABASE_NAME + "; dir="
+          + DATABASE_DATA_DIR);
+      server = new Server();
+      server.setDatabasePath(0, DATABASE_DATA_DIR);
+      server.setDatabaseName(0, DATABASE_NAME);
+      server.start();
+    }
+  }
+
+  public Connection getConnection() throws SQLException {
+    try {
+      Class.forName(DRIVER_CLASS);
+    } catch (ClassNotFoundException cnfe) {
+      LOG.error("Could not get connection; driver class not found: " + DRIVER_CLASS);
+      return null;
+    }
+
+    Connection connection = DriverManager.getConnection(DB_URL);
+    connection.setAutoCommit(false);
+    return connection;
+  }
+
+  /**
+   * Create a table
+   */
+  public void createSchema() throws SQLException {
+
+    Connection connection = null;
+    Statement st = null;
+
+    try {
+      connection = getConnection();
+
+      st = connection.createStatement();
+      st.executeUpdate("DROP TABLE " + DUMMY_TABLE_NAME + " IF EXISTS");
+      st.executeUpdate("CREATE TABLE " + DUMMY_TABLE_NAME + "(intField1 INT, intField2 INT)");
+
+      connection.commit();
+    } finally {
+      if (null != st) {
+        st.close();
+      }
+
+      if (null != connection) {
+        connection.close();
+      }
+    }
+  }
+
+  /**
+   * @return the sum of the integers in the first column of TWOINTTABLE.
+   */
+  public static int getFirstColSum() {
+    return 1 + 3 + 5 + 7;
+  }
+
+  /**
+   * Fill the table with some data
+   */
+  public void populateData() throws SQLException {
+
+    Connection connection = null;
+    Statement st = null;
+
+    try {
+      connection = getConnection();
+
+      st = connection.createStatement();
+      st.executeUpdate("INSERT INTO " + DUMMY_TABLE_NAME + " VALUES(1, 8)");
+      st.executeUpdate("INSERT INTO " + DUMMY_TABLE_NAME + " VALUES(3, 6)");
+      st.executeUpdate("INSERT INTO " + DUMMY_TABLE_NAME + " VALUES(5, 4)");
+      st.executeUpdate("INSERT INTO " + DUMMY_TABLE_NAME + " VALUES(7, 2)");
+
+      connection.commit();
+    } finally {
+      if (null != st) {
+        st.close();
+      }
+
+      if (null != connection) {
+        connection.close();
+      }
+    }
+  }
+
+  public void createEmployeeDemo() throws SQLException, ClassNotFoundException {
+    Class.forName(DRIVER_CLASS);
+
+    Connection connection = null;
+    Statement st = null;
+
+    try {
+      connection = getConnection();
+
+      st = connection.createStatement();
+      st.executeUpdate("DROP TABLE " + EMPLOYEE_TABLE_NAME + " IF EXISTS");
+      st.executeUpdate("CREATE TABLE " + EMPLOYEE_TABLE_NAME
+          + "(emp_id INT NOT NULL PRIMARY KEY, name VARCHAR(64))");
+
+      st.executeUpdate("INSERT INTO " + EMPLOYEE_TABLE_NAME + " VALUES(1, 'Aaron')");
+      st.executeUpdate("INSERT INTO " + EMPLOYEE_TABLE_NAME + " VALUES(2, 'Joe')");
+      st.executeUpdate("INSERT INTO " + EMPLOYEE_TABLE_NAME + " VALUES(3, 'Jim')");
+      st.executeUpdate("INSERT INTO " + EMPLOYEE_TABLE_NAME + " VALUES(4, 'Lisa')");
+
+      connection.commit();
+    } finally {
+      if (null != st) {
+        st.close();
+      }
+
+      if (null != connection) {
+        connection.close();
+      }
+    }
+  }
+
+  /**
+   * Delete any existing tables.
+   */
+  public void dropExistingSchema() throws SQLException {
+    ConnManager mgr = getManager();
+    String [] tables = mgr.listTables();
+    if (null != tables) {
+      Connection conn = mgr.getConnection();
+      for (String table : tables) {
+        Statement s = conn.createStatement();
+        s.executeUpdate("DROP TABLE " + table);
+        conn.commit();
+        s.close();
+      }
+    }
+  }
+
+  /**
+   * Creates an hsqldb server, fills it with tables and data.
+   */
+  public void resetServer() throws ClassNotFoundException, SQLException {
+    start();
+    dropExistingSchema();
+    createSchema();
+    populateData();
+  }
+
+  public ImportOptions getImportOptions() {
+    return new ImportOptions(HsqldbTestServer.getUrl(),
+        HsqldbTestServer.getTableName());
+  }
+
+  public ConnManager getManager() {
+    return new HsqldbManager(getImportOptions());
+  }
+
+
+}

+ 391 - 0
src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/ImportJobTestCase.java

@@ -0,0 +1,391 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.testutil;
+
+import java.io.File;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.BasicConfigurator;
+import org.junit.After;
+import org.junit.Before;
+
+import org.apache.hadoop.sqoop.ImportOptions;
+import org.apache.hadoop.sqoop.Sqoop;
+import org.apache.hadoop.sqoop.ImportOptions.InvalidOptionsException;
+import org.apache.hadoop.sqoop.manager.ConnManager;
+import org.apache.hadoop.sqoop.orm.CompilationManager;
+import org.apache.hadoop.sqoop.util.ClassLoaderStack;
+
+import junit.framework.TestCase;
+
+/**
+ * Class that implements common methods required for tests which import data
+ * from SQL into HDFS and verify correct import.
+ *
+ * 
+ *
+ */
+public class ImportJobTestCase extends TestCase {
+
+  public static final Log LOG = LogFactory.getLog(ImportJobTestCase.class.getName());
+
+  /** Base directory for all temporary data */
+  public static final String TEMP_BASE_DIR;
+
+  /** Where to import table data to in the local filesystem for testing */
+  public static final String LOCAL_WAREHOUSE_DIR;
+
+  // Initializer for the above
+  static {
+    String tmpDir = System.getProperty("test.build.data", "/tmp/");
+    if (!tmpDir.endsWith(File.separator)) {
+      tmpDir = tmpDir + File.separator;
+    }
+
+    TEMP_BASE_DIR = tmpDir;
+    LOCAL_WAREHOUSE_DIR = TEMP_BASE_DIR + "sqoop/warehouse";
+  }
+
+  /**
+   * Because of how classloading works, we don't actually want to name
+   * all the tables the same thing -- they'll actually just use the same
+   * implementation of the Java class that was classloaded before. So we
+   * use this counter to uniquify table names.
+   */
+  private static int tableNum = 0;
+
+  /** the name of a table that we'll populate with items for each test. */
+  static final String TABLE_NAME = "IMPORT_TABLE_";
+
+  protected String getTableName() {
+    return TABLE_NAME + Integer.toString(tableNum);
+  }
+
+  protected String getWarehouseDir() {
+    return LOCAL_WAREHOUSE_DIR;
+  }
+
+  private String [] colNames;
+  protected String [] getColNames() {
+    return colNames;
+  }
+
+  protected HsqldbTestServer getTestServer() {
+    return testServer;
+  }
+
+  protected ConnManager getManager() {
+    return manager;
+  }
+
+  // instance variables populated during setUp, used during tests
+  private HsqldbTestServer testServer;
+  private ConnManager manager;
+
+  private static boolean isLog4jConfigured = false;
+
+  protected void incrementTableNum() {
+    tableNum++;
+  }
+
+  @Before
+  public void setUp() {
+
+    incrementTableNum();
+
+    if (!isLog4jConfigured) {
+      BasicConfigurator.configure();
+      isLog4jConfigured = true;
+      LOG.info("Configured log4j with console appender.");
+    }
+
+    testServer = new HsqldbTestServer();
+    try {
+      testServer.resetServer();
+    } catch (SQLException sqlE) {
+      LOG.error("Got SQLException: " + sqlE.toString());
+      fail("Got SQLException: " + sqlE.toString());
+    } catch (ClassNotFoundException cnfe) {
+      LOG.error("Could not find class for db driver: " + cnfe.toString());
+      fail("Could not find class for db driver: " + cnfe.toString());
+    }
+
+    manager = testServer.getManager();
+  }
+
+  @After
+  public void tearDown() {
+    try {
+      manager.close();
+    } catch (SQLException sqlE) {
+      LOG.error("Got SQLException: " + sqlE.toString());
+      fail("Got SQLException: " + sqlE.toString());
+    }
+  }
+
+  static final String BASE_COL_NAME = "DATA_COL";
+
+  /**
+   * Create a table with a set of columns and add a row of values.
+   * @param colTypes the types of the columns to make
+   * @param vals the SQL text for each value to insert
+   */
+  protected void createTableWithColTypes(String [] colTypes, String [] vals) {
+    Connection conn = null;
+    try {
+      conn = getTestServer().getConnection();
+      PreparedStatement statement = conn.prepareStatement(
+          "DROP TABLE " + getTableName() + " IF EXISTS",
+          ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+      statement.executeUpdate();
+      statement.close();
+
+      String columnDefStr = "";
+      String columnListStr = "";
+      String valueListStr = "";
+
+      String [] myColNames = new String[colTypes.length];
+
+      for (int i = 0; i < colTypes.length; i++) {
+        String colName = BASE_COL_NAME + Integer.toString(i);
+        columnDefStr += colName + " " + colTypes[i];
+        columnListStr += colName;
+        valueListStr += vals[i];
+        myColNames[i] = colName;
+        if (i < colTypes.length - 1) {
+          columnDefStr += ", ";
+          columnListStr += ", ";
+          valueListStr += ", ";
+        }
+      }
+
+      statement = conn.prepareStatement(
+          "CREATE TABLE " + getTableName() + "(" + columnDefStr + ")",
+          ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+      statement.executeUpdate();
+      statement.close();
+
+      statement = conn.prepareStatement(
+          "INSERT INTO " + getTableName() + "(" + columnListStr + ")"
+          + " VALUES(" + valueListStr + ")",
+          ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+      statement.executeUpdate();
+      statement.close();
+      conn.commit();
+      this.colNames = myColNames;
+    } catch (SQLException sqlException) {
+      fail("Could not create table: " + sqlException.toString());
+    } finally {
+      if (null != conn) {
+        try {
+          conn.close();
+        } catch (SQLException sqlE) {
+          LOG.warn("Got SQLException during close: " + sqlE.toString());
+        }
+      }
+    }
+  }
+
+  /**
+   * Create a table with a single column and put a data element in it.
+   * @param colType the type of the column to create
+   * @param val the value to insert (reformatted as a string)
+   */
+  protected void createTableForColType(String colType, String val) {
+    String [] types = { colType };
+    String [] vals = { val };
+
+    createTableWithColTypes(types, vals);
+  }
+
+  /**
+   * verify that the single-column single-row result can be read back from the db.
+   *
+   */
+  protected void verifyReadback(int colNum, String expectedVal) {
+    try {
+      ResultSet results = getManager().readTable(getTableName(), getColNames());
+      assertNotNull("Null results from readTable()!", results);
+      assertTrue("Expected at least one row returned", results.next());
+      String resultVal = results.getString(colNum);
+      if (null != expectedVal) {
+        assertNotNull("Expected non-null result value", resultVal);
+      }
+
+      assertEquals("Error reading inserted value back from db", expectedVal, resultVal);
+      assertFalse("Expected at most one row returned", results.next());
+      results.close();
+    } catch (SQLException sqlE) {
+      fail("Got SQLException: " + sqlE.toString());
+    }
+  }
+
+  /**
+   * Create the argv to pass to Sqoop
+   * @param includeHadoopFlags if true, then include -D various.settings=values
+   * @param colNames the columns to import. If null, all columns are used.
+   * @return the argv as an array of strings.
+   */
+  private String [] getArgv(boolean includeHadoopFlags, String [] colNames) {
+    if (null == colNames) {
+      colNames = getColNames();
+    }
+
+    String orderByCol = colNames[0];
+    String columnsString = "";
+    for (String col : colNames) {
+      columnsString += col + ",";
+    }
+
+    ArrayList<String> args = new ArrayList<String>();
+
+    if (includeHadoopFlags) {
+      args.add("-D");
+      args.add("mapred.job.tracker=local");
+      args.add("-D");
+      args.add("mapred.map.tasks=1");
+      args.add("-D");
+      args.add("fs.default.name=file:///");
+    }
+
+    args.add("--table");
+    args.add(getTableName());
+    args.add("--columns");
+    args.add(columnsString);
+    args.add("--order-by");
+    args.add(orderByCol);
+    args.add("--warehouse-dir");
+    args.add(getWarehouseDir());
+    args.add("--connect");
+    args.add(HsqldbTestServer.getUrl());
+    args.add("--as-sequencefile");
+
+    return args.toArray(new String[0]);
+  }
+
+  protected Path getTablePath() {
+    Path warehousePath = new Path(getWarehouseDir());
+    Path tablePath = new Path(warehousePath, getTableName());
+    return tablePath;
+  }
+
+  protected Path getDataFilePath() {
+    return new Path(getTablePath(), "part-00000");
+  }
+
+  protected void removeTableDir() {
+    File tableDirFile = new File(getTablePath().toString());
+    if (tableDirFile.exists()) {
+      // Remove the director where the table will be imported to,
+      // prior to running the MapReduce job.
+      if (!DirUtil.deleteDir(tableDirFile)) {
+        LOG.warn("Could not delete table directory: " + tableDirFile.getAbsolutePath());
+      }
+    }
+  }
+
+  /**
+   * Do a MapReduce-based import of the table and verify that the results
+   * were imported as expected. (tests readFields(ResultSet) and toString())
+   * @param expectedVal the value we injected into the table.
+   * @param importCols the columns to import. If null, all columns are used.
+   */
+  protected void verifyImport(String expectedVal, String [] importCols) {
+
+    // paths to where our output file will wind up.
+    Path dataFilePath = getDataFilePath();
+
+    removeTableDir();
+
+    // run the tool through the normal entry-point.
+    int ret;
+    try {
+      Sqoop importer = new Sqoop();
+      ret = ToolRunner.run(importer, getArgv(true, importCols));
+    } catch (Exception e) {
+      LOG.error("Got exception running Sqoop: " + e.toString());
+      e.printStackTrace();
+      ret = 1;
+    }
+
+    // expect a successful return.
+    assertEquals("Failure during job", 0, ret);
+
+    ImportOptions opts = new ImportOptions();
+    try {
+      opts.parse(getArgv(false, importCols));
+    } catch (InvalidOptionsException ioe) {
+      fail(ioe.toString());
+    }
+    CompilationManager compileMgr = new CompilationManager(opts);
+    String jarFileName = compileMgr.getJarFilename();
+    ClassLoader prevClassLoader = null;
+    try {
+      prevClassLoader = ClassLoaderStack.addJarFile(jarFileName, getTableName());
+
+      // now actually open the file and check it
+      File f = new File(dataFilePath.toString());
+      assertTrue("Error: " + dataFilePath.toString() + " does not exist", f.exists());
+
+      Object readValue = SeqFileReader.getFirstValue(dataFilePath.toString());
+      if (null == expectedVal) {
+        assertEquals("Error validating result from SeqFile", "null", readValue.toString());
+      } else {
+        assertEquals("Error validating result from SeqFile", expectedVal, readValue.toString());
+      }
+    } catch (IOException ioe) {
+      fail("IOException: " + ioe.toString());
+    } finally {
+      if (null != prevClassLoader) {
+        ClassLoaderStack.setCurrentClassLoader(prevClassLoader);
+      }
+    }
+  }
+
+  /**
+   * Run a MapReduce-based import (using the argv provided to control execution).
+   */
+  protected void runImport(String [] argv) throws IOException {
+    removeTableDir();
+
+    // run the tool through the normal entry-point.
+    int ret;
+    try {
+      Sqoop importer = new Sqoop();
+      ret = ToolRunner.run(importer, argv);
+    } catch (Exception e) {
+      LOG.error("Got exception running Sqoop: " + e.toString());
+      e.printStackTrace();
+      ret = 1;
+    }
+
+    // expect a successful return.
+    assertEquals("Failure during job", 0, ret);
+  }
+
+}

+ 78 - 0
src/contrib/sqoop/src/test/org/apache/hadoop/sqoop/testutil/SeqFileReader.java

@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.sqoop.testutil;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * Utility class to help with test cases. Just reads the first (k, v) pair
+ * from a SequenceFile and returns the value part.
+ * 
+ *
+ */
+public final class SeqFileReader {
+
+  public static final Log LOG = LogFactory.getLog(SeqFileReader.class.getName());
+
+  public static Reader getSeqFileReader(String filename) throws IOException {
+    // read from local filesystem
+    Configuration conf = new Configuration();
+    conf.set("fs.default.name", "file:///");
+    FileSystem fs = FileSystem.get(conf);
+    LOG.info("Opening SequenceFile " + filename);
+    return new SequenceFile.Reader(fs, new Path(filename), conf);
+  }
+
+  public static Object getFirstValue(String filename) throws IOException {
+    Reader r = null;
+    try {
+      // read from local filesystem
+      Configuration conf = new Configuration();
+      conf.set("fs.default.name", "file:///");
+      FileSystem fs = FileSystem.get(conf);
+      r = new SequenceFile.Reader(fs, new Path(filename), conf);
+      Object key = ReflectionUtils.newInstance(r.getKeyClass(), conf);
+      Object val = ReflectionUtils.newInstance(r.getValueClass(), conf);
+      LOG.info("Reading value of type " + r.getValueClassName()
+          + " from SequenceFile " + filename);
+      r.next(key);
+      r.getCurrentValue(val);
+      LOG.info("Value as string: " + val.toString());
+      return val;
+    } finally {
+      if (null != r) {
+        try {
+          r.close();
+        } catch (IOException ioe) {
+          LOG.warn("IOException during close: " + ioe.toString());
+        }
+      }
+    }
+  }
+}
+