You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ek...@apache.org on 2014/10/25 21:19:38 UTC

svn commit: r1634242 - in /hive/trunk/hcatalog: src/test/e2e/templeton/deployers/ src/test/e2e/templeton/deployers/config/webhcat/ src/test/e2e/templeton/tests/ webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/ webhcat/svr/src/main/java/org...

Author: ekoifman
Date: Sat Oct 25 19:19:37 2014
New Revision: 1634242

URL: http://svn.apache.org/r1634242
Log:
HIVE-8588 sqoop REST endpoint fails to send appropriate JDBC driver to the cluster

Modified:
    hive/trunk/hcatalog/src/test/e2e/templeton/deployers/clean_file_system.sh
    hive/trunk/hcatalog/src/test/e2e/templeton/deployers/config/webhcat/webhcat-site.xml
    hive/trunk/hcatalog/src/test/e2e/templeton/deployers/deploy_e2e_artifacts.sh
    hive/trunk/hcatalog/src/test/e2e/templeton/deployers/env.sh
    hive/trunk/hcatalog/src/test/e2e/templeton/tests/jobsubmission.conf
    hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/AppConfig.java
    hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/JarDelegator.java
    hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/LauncherDelegator.java
    hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Server.java
    hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/SqoopDelegator.java
    hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/JobSubmissionConstants.java
    hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/LaunchMapper.java
    hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/TempletonUtils.java

Modified: hive/trunk/hcatalog/src/test/e2e/templeton/deployers/clean_file_system.sh
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/templeton/deployers/clean_file_system.sh?rev=1634242&r1=1634241&r2=1634242&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/templeton/deployers/clean_file_system.sh (original)
+++ hive/trunk/hcatalog/src/test/e2e/templeton/deployers/clean_file_system.sh Sat Oct 25 19:19:37 2014
@@ -28,6 +28,6 @@
 echo "Deleting artifacts from HDFS..."
 
 ${HADOOP_HOME}/bin/hdfs dfs -rm -r       /user/hive/ /user/${USER}/ /user/templeton /apps /tmp /sqoopoutputdir
-${HADOOP_HOME}/bin/hdfs dfs -mkdir -p    /tmp/hadoop-${USER} /user/hive/warehouse /user/${USER}/ /user/templeton /apps/templeton /tmp/hadoop-yarn /tmp/templeton_test_out
+${HADOOP_HOME}/bin/hdfs dfs -mkdir -p    /tmp/hadoop-${USER} /user/hive/warehouse /user/${USER}/ /user/templeton /apps/templeton/jdbc /tmp/hadoop-yarn /tmp/templeton_test_out
 ${HADOOP_HOME}/bin/hdfs dfs -chmod -R a+rwx /user /tmp/
 ${HADOOP_HOME}/bin/hdfs dfs -chmod g+rwx   /user/hive/warehouse

Modified: hive/trunk/hcatalog/src/test/e2e/templeton/deployers/config/webhcat/webhcat-site.xml
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/templeton/deployers/config/webhcat/webhcat-site.xml?rev=1634242&r1=1634241&r2=1634242&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/templeton/deployers/config/webhcat/webhcat-site.xml (original)
+++ hive/trunk/hcatalog/src/test/e2e/templeton/deployers/config/webhcat/webhcat-site.xml Sat Oct 25 19:19:37 2014
@@ -78,10 +78,22 @@
             HCat, Hive query, etc.</description>
     </property>
     <property>
+        <name>templeton.sqoop.archive</name>
+        <value>hdfs:///apps/templeton/sqoop-1.4.5.bin__hadoop-2.0.4-alpha.tar.gz</value>
+        <description>This should point to Sqoop tar that will be shipped to target node executing
+        the actual sqoop command.  If not set, Sqoop is expected to be installed on every node of the
+         cluster.</description>
+    </property>
+    <property>
         <name>templeton.sqoop.path</name>
-        <value>${env.SQOOP_HOME}/bin/sqoop</value>
+        <value>sqoop-1.4.5.bin__hadoop-2.0.4-alpha.tar.gz/sqoop-1.4.5.bin__hadoop-2.0.4-alpha/bin/sqoop</value>
         <description>The path to the Sqoop executable.</description>
     </property>
+    <property>
+        <name>templeton.sqoop.home</name>
+        <value>sqoop-1.4.5.bin__hadoop-2.0.4-alpha.tar.gz/sqoop-1.4.5.bin__hadoop-2.0.4-alpha</value>
+        <description>The path to the Sqoop home in the exploded archive.</description>
+    </property>
 
     <property>
         <name>templeton.controller.mr.child.opts</name>

Modified: hive/trunk/hcatalog/src/test/e2e/templeton/deployers/deploy_e2e_artifacts.sh
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/templeton/deployers/deploy_e2e_artifacts.sh?rev=1634242&r1=1634241&r2=1634242&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/templeton/deployers/deploy_e2e_artifacts.sh (original)
+++ hive/trunk/hcatalog/src/test/e2e/templeton/deployers/deploy_e2e_artifacts.sh Sat Oct 25 19:19:37 2014
@@ -35,7 +35,6 @@ ${HADOOP_HOME}/bin/hdfs dfs -put ${HADOO
 ${HADOOP_HOME}/bin/hdfs dfs -put ${HADOOP_HOME}/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-${HADOOP_VERSION}.jar webhcate2e/hclient.jar
 ${HADOOP_HOME}/bin/hdfs dfs -put ${HADOOP_HOME}/share/hadoop/tools/lib/hadoop-streaming-${HADOOP_VERSION}.jar  /user/templeton/hadoop-streaming.jar
 
-
 #must match config/webhcat/webhcat-stie.xml
 ${HADOOP_HOME}/bin/hdfs dfs -put ${PROJ_HOME}/packaging/target/apache-hive-${HIVE_VERSION}-bin.tar.gz /apps/templeton/apache-hive-${HIVE_VERSION}-bin.tar.gz
 # To run against Hadoop2 cluster, you have to build Pig tar yourself with 
@@ -47,7 +46,7 @@ ${HADOOP_HOME}/bin/hadoop fs -put ${PIG_
 #${HADOOP_HOME}/bin/hadoop fs -put /Users/ekoifman/dev/data/jarsForTmplte2e/pig-0.12.0/contrib/piggybank/java/piggybank.jar  webhcate2e/
 
 
-${HADOOP_HOME}/bin/hadoop fs -put ${HIVE_HOME}/lib/zookeeper-3.4.5.jar /apps/templeton/zookeeper-3.4.5.jar
-
+${HADOOP_HOME}/bin/hadoop fs -put /Users/ekoifman/dev/sqoop-1.4.5.bin__hadoop-2.0.4-alpha.tar.gz /apps/templeton/sqoop-1.4.5.bin__hadoop-2.0.4-alpha.tar.gz
+${HADOOP_HOME}/bin/hadoop fs -put /Users/ekoifman/dev/mysql-connector-java-5.1.30/mysql-connector-java-5.1.30-bin.jar /apps/templeton/jdbc/mysql-connector-java.jar
 #check what got deployed
-${HADOOP_HOME}/bin/hdfs dfs -ls /apps/templeton webhcate2e /user/templeton /user/hive/warehouse
+${HADOOP_HOME}/bin/hdfs dfs -ls -R /apps/templeton webhcate2e /user/templeton /user/hive/warehouse

Modified: hive/trunk/hcatalog/src/test/e2e/templeton/deployers/env.sh
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/templeton/deployers/env.sh?rev=1634242&r1=1634241&r2=1634242&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/templeton/deployers/env.sh (original)
+++ hive/trunk/hcatalog/src/test/e2e/templeton/deployers/env.sh Sat Oct 25 19:19:37 2014
@@ -23,14 +23,13 @@
 # define necessary env vars here and source it in other files
 
 export HADOOP_VERSION=2.4.1-SNAPSHOT
-export HIVE_VERSION=0.14.0-SNAPSHOT
+#export HIVE_VERSION=0.14.0-SNAPSHOT
 export PIG_VERSION=0.12.2-SNAPSHOT
 
 #Root of project source tree
 export PROJ_HOME=/Users/${USER}/dev/hive
 export HIVE_HOME=${PROJ_HOME}/packaging/target/apache-hive-${HIVE_VERSION}-bin/apache-hive-${HIVE_VERSION}-bin
 export HADOOP_HOME=/Users/${USER}/dev/hwxhadoop/hadoop-dist/target/hadoop-${HADOOP_VERSION}
-export SQOOP_HOME=/Users/${USER}/dev/sqoop-1.4.4.bin__hadoop-2.0.4-alpha
 
 #Make sure Pig is built for the Hadoop version you are running
 export PIG_TAR_PATH=/Users/${USER}/dev/pig-${PIG_VERSION}-src/build

Modified: hive/trunk/hcatalog/src/test/e2e/templeton/tests/jobsubmission.conf
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/test/e2e/templeton/tests/jobsubmission.conf?rev=1634242&r1=1634241&r2=1634242&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/test/e2e/templeton/tests/jobsubmission.conf (original)
+++ hive/trunk/hcatalog/src/test/e2e/templeton/tests/jobsubmission.conf Sat Oct 25 19:19:37 2014
@@ -524,7 +524,7 @@ $cfg = 
      'num' => 1,
      'method' => 'POST',
      'url' => ':TEMPLETON_URL:/templeton/v1/sqoop?user.name=:UNAME:',
-     'post_options' => ['command=export --connect :DB_CONNECTION_STRING: --username :DB_USER_NAME: --password :DB_PASSWORD: --export-dir :INPDIR_HDFS:/sqoop --table person','statusdir=sqoop.output' ],
+     'post_options' => ['libdir=hdfs:///apps/templeton/jdbc', 'command=export --connect :DB_CONNECTION_STRING: --username :DB_USER_NAME: --password :DB_PASSWORD: --export-dir :INPDIR_HDFS:/sqoop --table person','statusdir=TestSqoop_:TNUM:' ],
      'json_field_substr_match' => { 'id' => '\d+'},
                    #results
      'status_code' => 200,
@@ -539,7 +539,7 @@ $cfg = 
      'num' => 2,
      'method' => 'POST',
      'url' => ':TEMPLETON_URL:/templeton/v1/sqoop?user.name=:UNAME:',
-     'post_options' => ['files=:INPDIR_HDFS:/sqoopcommand.txt','command=import --connect :DB_CONNECTION_STRING: --username :DB_USER_NAME: --password :DB_PASSWORD: --options-file sqoopcommand.txt','statusdir=sqoop.output' ],
+     'post_options' => ['libdir=hdfs:///apps/templeton/jdbc', 'files=:INPDIR_HDFS:/sqoopcommand.txt','command=import --connect :DB_CONNECTION_STRING: --username :DB_USER_NAME: --password :DB_PASSWORD: --options-file sqoopcommand.txt','statusdir=TestSqoop_:TNUM:' ],
      'json_field_substr_match' => { 'id' => '\d+'},
                    #results
      'status_code' => 200,
@@ -554,7 +554,7 @@ $cfg = 
      'num' => 3,
      'method' => 'POST',
      'url' => ':TEMPLETON_URL:/templeton/v1/sqoop?user.name=:UNAME:',
-     'post_options' => ['files=:INPDIR_HDFS:/sqoopcommand.txt','statusdir=sqoop.output' ],
+     'post_options' => ['files=:INPDIR_HDFS:/sqoopcommand.txt','statusdir=TestSqoop_:TNUM:' ],
      'json_field_substr_match' => { 'error' => 'Must define Sqoop command or a optionsfile contains Sqoop command to run Sqoop job.'},
                    #results
      'status_code' => 400,
@@ -564,7 +564,7 @@ $cfg = 
      'num' => 4,
      'method' => 'POST',
      'url' => ':TEMPLETON_URL:/templeton/v1/sqoop?user.name=:UNAME:',
-     'post_options' => ['optionsfile=:INPDIR_HDFS:/sqoopcommand.txt','command=import --connect :DB_CONNECTION_STRING: --username :DB_USER_NAME: --password :DB_PASSWORD: --options-file sqoopcommand.txt','statusdir=sqoop.output' ],
+     'post_options' => ['optionsfile=:INPDIR_HDFS:/sqoopcommand.txt','command=import --connect :DB_CONNECTION_STRING: --username :DB_USER_NAME: --password :DB_PASSWORD: --options-file sqoopcommand.txt','statusdir=TestSqoop_:TNUM:' ],
      'json_field_substr_match' => { 'error' => 'Cannot set command and optionsfile at the same time.'},
                    #results
      'status_code' => 400,

Modified: hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/AppConfig.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/AppConfig.java?rev=1634242&r1=1634241&r2=1634242&view=diff
==============================================================================
--- hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/AppConfig.java (original)
+++ hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/AppConfig.java Sat Oct 25 19:19:37 2014
@@ -119,6 +119,7 @@ public class AppConfig extends Configura
   public static final String HIVE_PROPS_NAME     = "templeton.hive.properties";
   public static final String SQOOP_ARCHIVE_NAME  = "templeton.sqoop.archive";
   public static final String SQOOP_PATH_NAME     = "templeton.sqoop.path";
+  public static final String SQOOP_HOME_PATH     = "templeton.sqoop.home";
   public static final String LIB_JARS_NAME       = "templeton.libjars";
   public static final String PIG_ARCHIVE_NAME    = "templeton.pig.archive";
   public static final String PIG_PATH_NAME       = "templeton.pig.path";
@@ -297,6 +298,7 @@ public class AppConfig extends Configura
   public String hiveArchive()      { return get(HIVE_ARCHIVE_NAME); }
   public String sqoopPath()        { return get(SQOOP_PATH_NAME); }
   public String sqoopArchive()     { return get(SQOOP_ARCHIVE_NAME); }
+  public String sqoopHome()        { return get(SQOOP_HOME_PATH); }
   public String streamingJar()     { return get(STREAMING_JAR_NAME); }
   public String kerberosSecret()   { return get(KERBEROS_SECRET); }
   public String kerberosPrincipal(){ return get(KERBEROS_PRINCIPAL); }

Modified: hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/JarDelegator.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/JarDelegator.java?rev=1634242&r1=1634241&r2=1634242&view=diff
==============================================================================
--- hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/JarDelegator.java (original)
+++ hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/JarDelegator.java Sat Oct 25 19:19:37 2014
@@ -85,6 +85,9 @@ public class JarDelegator extends Launch
       if (TempletonUtils.isset(libjars)) {
         String libjarsListAsString =
             TempletonUtils.hadoopFsListAsString(libjars, appConf, runAs);
+        //This will work only if the files are local files on webhcat server
+        // (which is not very useful since users might not have access to that file system).
+        //This is likely the HIVE-5188 issue
         args.add("-libjars");
         args.add(TempletonUtils.quoteForWindows(libjarsListAsString));
       }

Modified: hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/LauncherDelegator.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/LauncherDelegator.java?rev=1634242&r1=1634241&r2=1634242&view=diff
==============================================================================
--- hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/LauncherDelegator.java (original)
+++ hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/LauncherDelegator.java Sat Oct 25 19:19:37 2014
@@ -121,6 +121,8 @@ public class LauncherDelegator extends T
                      JobType jobType) {
     ArrayList<String> args = new ArrayList<String>();
 
+    //note that in ToolRunner this is expected to be a local FS path
+    //see GenericOptionsParser.getLibJars()
     args.add("-libjars");
 
     // Include shim and admin specified libjars
@@ -136,6 +138,7 @@ public class LauncherDelegator extends T
 
     // Internal vars
     addDef(args, TempletonControllerJob.STATUSDIR_NAME, statusdir);
+    //Use of ToolRunner "-files" option could be considered here
     addDef(args, TempletonControllerJob.COPY_NAME,
       TempletonUtils.encodeArray(copyFiles));
     addDef(args, TempletonControllerJob.OVERRIDE_CLASSPATH,

Modified: hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Server.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Server.java?rev=1634242&r1=1634241&r2=1634242&view=diff
==============================================================================
--- hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Server.java (original)
+++ hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/Server.java Sat Oct 25 19:19:37 2014
@@ -778,6 +778,7 @@ public class Server {
    * @param optionsFile  name of option file which contains Sqoop command to run
    * @param otherFiles   additional files to be shipped to the launcher, such as option
                          files which contain part of the Sqoop command
+   * @param libdir       dir containing JDBC jars that Sqoop will need to interact with the database
    * @param statusdir    where the stderr/stdout of templeton controller job goes
    * @param callback     URL which WebHCat will call when the sqoop job finishes
    * @param enablelog    whether to collect mapreduce log into statusdir/logs
@@ -787,12 +788,13 @@ public class Server {
   @Produces({MediaType.APPLICATION_JSON})
   public EnqueueBean sqoop(@FormParam("command") String command,
               @FormParam("optionsfile") String optionsFile,
+              @FormParam("libdir") String libdir,
               @FormParam("files") String otherFiles,
               @FormParam("statusdir") String statusdir,
               @FormParam("callback") String callback,
               @FormParam("enablelog") boolean enablelog)
     throws NotAuthorizedException, BusyException, BadParam, QueueException,
-    ExecuteException, IOException, InterruptedException {
+    IOException, InterruptedException {
     verifyUser();
     if (command == null && optionsFile == null)
       throw new BadParam("Must define Sqoop command or a optionsfile contains Sqoop command to run Sqoop job.");
@@ -805,13 +807,14 @@ public class Server {
     userArgs.put("user.name", getDoAsUser());
     userArgs.put("command", command);
     userArgs.put("optionsfile", optionsFile);
+    userArgs.put("libdir", libdir);
     userArgs.put("files", otherFiles);
     userArgs.put("statusdir", statusdir);
     userArgs.put("callback", callback);
     userArgs.put("enablelog", Boolean.toString(enablelog));
     SqoopDelegator d = new SqoopDelegator(appConf);
     return d.run(getDoAsUser(), userArgs, command, optionsFile, otherFiles,
-      statusdir, callback, getCompletedUrl(), enablelog);
+      statusdir, callback, getCompletedUrl(), enablelog, libdir);
   }
 
   /**

Modified: hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/SqoopDelegator.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/SqoopDelegator.java?rev=1634242&r1=1634241&r2=1634242&view=diff
==============================================================================
--- hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/SqoopDelegator.java (original)
+++ hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/SqoopDelegator.java Sat Oct 25 19:19:37 2014
@@ -25,7 +25,10 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.commons.exec.ExecuteException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hive.hcatalog.templeton.tool.JobSubmissionConstants;
 import org.apache.hive.hcatalog.templeton.tool.TempletonControllerJob;
 import org.apache.hive.hcatalog.templeton.tool.TempletonUtils;
 
@@ -35,6 +38,7 @@ import org.apache.hive.hcatalog.templeto
  * This is the backend of the Sqoop web service.
  */
 public class SqoopDelegator extends LauncherDelegator {
+  private static final Log LOG = LogFactory.getLog(SqoopDelegator.class);
 
   public SqoopDelegator(AppConfig appConf) {
     super(appConf);
@@ -43,24 +47,29 @@ public class SqoopDelegator extends Laun
   public EnqueueBean run(String user,
                Map<String, Object> userArgs, String command, 
                String optionsFile, String otherFiles, String statusdir, 
-               String callback, String completedUrl, boolean enablelog)
+               String callback, String completedUrl, boolean enablelog, String libdir)
   throws NotAuthorizedException, BadParam, BusyException, QueueException,
-  ExecuteException, IOException, InterruptedException
+  IOException, InterruptedException
   {
+    if(TempletonUtils.isset(appConf.sqoopArchive())) {
+      if(!TempletonUtils.isset(appConf.sqoopPath()) && !TempletonUtils.isset(appConf.sqoopHome())) {
+        throw new IllegalStateException("If '" + AppConfig.SQOOP_ARCHIVE_NAME + "' is defined, '" +
+        AppConfig.SQOOP_PATH_NAME + "' and '" + AppConfig.SQOOP_HOME_PATH + "' must be defined");
+      }
+    }
     runAs = user;
     List<String> args = makeArgs(command, optionsFile, otherFiles, statusdir,
-                   completedUrl, enablelog);
+                   completedUrl, enablelog, libdir);
 
     return enqueueController(user, userArgs, callback, args);
   }
-
-  List<String> makeArgs(String command, String optionsFile, String otherFiles,
-            String statusdir, String completedUrl, boolean enablelog)
+  private List<String> makeArgs(String command, String optionsFile, String otherFiles,
+            String statusdir, String completedUrl, boolean enablelog, String libdir)
     throws BadParam, IOException, InterruptedException
   {
     ArrayList<String> args = new ArrayList<String>();
     try {
-      args.addAll(makeBasicArgs(optionsFile, otherFiles, statusdir, completedUrl, enablelog));
+      args.addAll(makeBasicArgs(optionsFile, otherFiles, statusdir, completedUrl, enablelog, libdir));
       args.add("--");
       TempletonUtils.addCmdForWindows(args);
       args.add(appConf.sqoopPath());
@@ -89,7 +98,7 @@ public class SqoopDelegator extends Laun
   }
 
   private List<String> makeBasicArgs(String optionsFile, String otherFiles,
-            String statusdir, String completedUrl, boolean enablelog)
+            String statusdir, String completedUrl, boolean enablelog, String libdir)
     throws URISyntaxException, FileNotFoundException, IOException,
                           InterruptedException
   {
@@ -101,9 +110,34 @@ public class SqoopDelegator extends Laun
       String[] ofs = TempletonUtils.hadoopFsListAsArray(otherFiles, appConf, runAs);
       allFiles.addAll(Arrays.asList(ofs));
     }
+    if(TempletonUtils.isset(libdir) && TempletonUtils.isset(appConf.sqoopArchive())) {
+      /**Sqoop accesses databases via JDBC.  This means it needs to have appropriate JDBC
+      drivers available.  Normally, the user would install Sqoop and place these jars
+      into SQOOP_HOME/lib.  When WebHCat is configured to auto-ship the Sqoop tar file, we
+      need to make sure that relevant JDBC jars are available on target node.
+      The user is expected to place any JDBC jars into an HDFS directory and specify this
+      dir in "libdir" parameter.  All the files in this dir will be copied to lib/ of the
+      exploded Sqoop tar ball on target node.
+      {@link org.apache.hive.hcatalog.templeton.tool.LaunchMapper#handleSqoop(org.apache.hadoop.conf.Configuration, java.util.Map)}
+      */
+      LOG.debug("libdir=" + libdir);
+      List<Path> jarList = TempletonUtils.hadoopFsListChildren(libdir, appConf, runAs);
+      if(TempletonUtils.isset(jarList)) {
+        StringBuilder sb = new StringBuilder();
+        for(Path jar : jarList) {
+          allFiles.add(jar.toString());
+          sb.append(jar.getName()).append(',');
+        }
+        sb.setLength(sb.length() - 1);
+        //we use the same mechanism to copy "files"/"otherFiles" and "libdir", but we only want to put
+        //contents of "libdir" in Sqoop/lib, thus we pass the list of names here
+        addDef(args, JobSubmissionConstants.Sqoop.LIB_JARS, sb.toString());
+        addDef(args, AppConfig.SQOOP_HOME_PATH, appConf.get(AppConfig.SQOOP_HOME_PATH));
+      }
+    }
     args.addAll(makeLauncherArgs(appConf, statusdir, completedUrl, allFiles,
                 enablelog, JobType.SQOOP));
-    if (appConf.sqoopArchive() != null && !appConf.sqoopArchive().equals("")) {
+    if(TempletonUtils.isset(appConf.sqoopArchive())) {
       args.add("-archives");
       args.add(appConf.sqoopArchive());
     }

Modified: hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/JobSubmissionConstants.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/JobSubmissionConstants.java?rev=1634242&r1=1634241&r2=1634242&view=diff
==============================================================================
--- hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/JobSubmissionConstants.java (original)
+++ hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/JobSubmissionConstants.java Sat Oct 25 19:19:37 2014
@@ -58,4 +58,10 @@ public interface JobSubmissionConstants 
     public static final String HCAT_HOME = "HCAT_HOME";
     public static final String PIG_OPTS = "PIG_OPTS";
   }
+  public static interface Sqoop {
+    /**
+     * comma-separated list of jar names (short name) which are needed for Sqoop JDBC access
+     */
+    public static final String LIB_JARS = "templeton.sqoop.lib.jar";
+  }
 }

Modified: hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/LaunchMapper.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/LaunchMapper.java?rev=1634242&r1=1634241&r2=1634242&view=diff
==============================================================================
--- hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/LaunchMapper.java (original)
+++ hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/LaunchMapper.java Sat Oct 25 19:19:37 2014
@@ -18,6 +18,7 @@
  */
 package org.apache.hive.hcatalog.templeton.tool;
 
+import com.google.common.io.Files;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -33,6 +34,7 @@ import org.apache.hadoop.mapreduce.Mappe
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hive.hcatalog.templeton.AppConfig;
 import org.apache.hive.hcatalog.templeton.BadParam;
 import org.apache.hive.hcatalog.templeton.LauncherDelegator;
 
@@ -89,6 +91,24 @@ public class LaunchMapper extends Mapper
       env.put(PigConstants.PIG_OPTS, pigOpts.toString());
     }
   }
+
+  /**
+   * {@link #copyLocal(String, org.apache.hadoop.conf.Configuration)} should be called before this
+   * See {@link org.apache.hive.hcatalog.templeton.SqoopDelegator#makeBasicArgs(String, String, String, String, boolean, String)}
+   * for more comments
+   */
+  private static void handleSqoop(Configuration conf, Map<String, String> env) throws IOException {
+    if(TempletonUtils.isset(conf.get(Sqoop.LIB_JARS))) {
+      //LIB_JARS should only be set if Sqoop is auto-shipped
+      LOG.debug(Sqoop.LIB_JARS + "=" + conf.get(Sqoop.LIB_JARS));
+      //copy these (which have now been localized) jars to sqoop/lib
+      String destDir = conf.get(AppConfig.SQOOP_HOME_PATH) + File.separator + "lib";
+      String[] files = conf.getStrings(Sqoop.LIB_JARS);
+      for(String f : files) {
+        Files.copy(new File(f), new File(destDir + File.separator + f));
+      }
+    }
+  }
   protected Process startJob(Context context, String user, String overrideClasspath)
     throws IOException, InterruptedException {
     Configuration conf = context.getConfiguration();
@@ -108,6 +128,7 @@ public class LaunchMapper extends Mapper
     removeEnv.add("mapredcommand");
     Map<String, String> env = TempletonUtils.hadoopUserEnv(user, overrideClasspath);
     handlePigEnvVars(conf, env);
+    handleSqoop(conf, env);
     List<String> jarArgsList = new LinkedList<String>(Arrays.asList(jarArgs));
     handleTokenFile(jarArgsList, JobSubmissionConstants.TOKEN_FILE_ARG_PLACEHOLDER, "mapreduce.job.credentials.binary");
     handleTokenFile(jarArgsList, JobSubmissionConstants.TOKEN_FILE_ARG_PLACEHOLDER_TEZ, "tez.credentials.path");

Modified: hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/TempletonUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/TempletonUtils.java?rev=1634242&r1=1634241&r2=1634242&view=diff
==============================================================================
--- hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/TempletonUtils.java (original)
+++ hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/TempletonUtils.java Sat Oct 25 19:19:37 2014
@@ -29,6 +29,7 @@ import java.net.URLDecoder;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Enumeration;
 import java.util.HashMap;
 import java.util.List;
@@ -39,8 +40,10 @@ import java.util.regex.Pattern;
 import javax.ws.rs.core.UriBuilder;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
@@ -211,6 +214,28 @@ public class TempletonUtils {
   }
 
   /**
+   * Returns all files (non-recursive) in {@code dirName}
+   */
+  public static List<Path> hadoopFsListChildren(String dirName, Configuration conf, String user)
+    throws URISyntaxException, IOException, InterruptedException {
+
+    Path p = hadoopFsPath(dirName, conf, user);
+    FileSystem fs =  p.getFileSystem(conf);
+    if(!fs.exists(p)) {
+      return Collections.emptyList();
+    }
+    List<FileStatus> children = ShimLoader.getHadoopShims().listLocatedStatus(fs, p, null);
+    if(!isset(children)) {
+      return Collections.emptyList();
+    }
+    List<Path> files = new ArrayList<Path>();
+    for(FileStatus stat : children) {
+      files.add(stat.getPath());
+    }
+    return files;
+  }
+
+  /**
    * @return true iff we are sure the file is not there.
    */
   public static boolean hadoopFsIsMissing(FileSystem fs, Path p) {
@@ -239,8 +264,7 @@ public class TempletonUtils {
   }
 
   public static Path hadoopFsPath(String fname, final Configuration conf, String user)
-    throws URISyntaxException, IOException,
-    InterruptedException {
+    throws URISyntaxException, IOException, InterruptedException {
     if (fname == null || conf == null) {
       return null;
     }