You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by th...@apache.org on 2013/09/27 01:29:07 UTC
svn commit: r1526748 - in /hive/trunk/hcatalog:
src/docs/src/documentation/content/xdocs/
webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/
webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/
Author: thejas
Date: Thu Sep 26 23:29:06 2013
New Revision: 1526748
URL: http://svn.apache.org/r1526748
Log:
HIVE-5031: [WebHCat] GET job/:jobid to return userargs for a job in addtion to status information (Daniel Dai via Thejas Nair)
Modified:
hive/trunk/hcatalog/src/docs/src/documentation/content/xdocs/queue.xml
hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/HiveDelegator.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/PigDelegator.java
hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/QueueStatusBean.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/StreamingDelegator.java
hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/JobState.java
Modified: hive/trunk/hcatalog/src/docs/src/documentation/content/xdocs/queue.xml
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/src/docs/src/documentation/content/xdocs/queue.xml?rev=1526748&r1=1526747&r2=1526748&view=diff
==============================================================================
--- hive/trunk/hcatalog/src/docs/src/documentation/content/xdocs/queue.xml (original)
+++ hive/trunk/hcatalog/src/docs/src/documentation/content/xdocs/queue.xml Thu Sep 26 23:29:06 2013
@@ -143,7 +143,17 @@
"exitValue": 0,
"user": "ctdean",
"callback": null,
- "completed": "done"
+ "completed": "done",
+ "userargs" => {
+ "callback" => null,
+ "define" => [],
+ "enablelog" => "false",
+ "execute" => "select a,rand(b) from mynums",
+ "file" => null,
+ "files" => [],
+ "statusdir" => null,
+ "user.name" => "hadoopqa",
+ },
}
</source>
</section>
Modified: hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/HiveDelegator.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/HiveDelegator.java?rev=1526748&r1=1526747&r2=1526748&view=diff
==============================================================================
--- hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/HiveDelegator.java (original)
+++ hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/HiveDelegator.java Thu Sep 26 23:29:06 2013
@@ -24,6 +24,7 @@ import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
+import java.util.Map;
import org.apache.commons.exec.ExecuteException;
import org.apache.hive.hcatalog.templeton.tool.TempletonControllerJob;
@@ -40,7 +41,7 @@ public class HiveDelegator extends Launc
super(appConf);
}
- public EnqueueBean run(String user,
+ public EnqueueBean run(String user, Map<String, Object> userArgs,
String execute, String srcFile, List<String> defines,
List<String> hiveArgs, String otherFiles,
String statusdir, String callback, String completedUrl, boolean enablelog)
@@ -51,7 +52,7 @@ public class HiveDelegator extends Launc
List<String> args = makeArgs(execute, srcFile, defines, hiveArgs, otherFiles, statusdir,
completedUrl, enablelog);
- return enqueueController(user, callback, args);
+ return enqueueController(user, userArgs, callback, args);
}
private List<String> makeArgs(String execute, String srcFile,
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=1526748&r1=1526747&r2=1526748&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 Thu Sep 26 23:29:06 2013
@@ -23,6 +23,7 @@ import java.io.IOException;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.List;
+import java.util.Map;
import org.apache.commons.exec.ExecuteException;
import org.apache.hive.hcatalog.templeton.tool.TempletonControllerJob;
@@ -38,7 +39,7 @@ public class JarDelegator extends Launch
super(appConf);
}
- public EnqueueBean run(String user, String jar, String mainClass,
+ public EnqueueBean run(String user, Map<String, Object> userArgs, String jar, String mainClass,
String libjars, String files,
List<String> jarArgs, List<String> defines,
String statusdir, String callback, String completedUrl,
@@ -50,7 +51,7 @@ public class JarDelegator extends Launch
libjars, files, jarArgs, defines,
statusdir, completedUrl, enablelog, jobType);
- return enqueueController(user, callback, args);
+ return enqueueController(user, userArgs, callback, args);
}
private List<String> makeArgs(String jar, String mainClass,
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=1526748&r1=1526747&r2=1526748&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 Thu Sep 26 23:29:06 2013
@@ -22,6 +22,7 @@ import java.io.IOException;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.List;
+import java.util.Map;
import org.apache.commons.exec.ExecuteException;
import org.apache.commons.logging.Log;
@@ -49,13 +50,15 @@ public class LauncherDelegator extends T
super(appConf);
}
- public void registerJob(String id, String user, String callback)
+ public void registerJob(String id, String user, String callback,
+ Map<String, Object> userArgs)
throws IOException {
JobState state = null;
try {
state = new JobState(id, Main.getAppConfigInstance());
state.setUser(user);
state.setCallback(callback);
+ state.setUserArgs(userArgs);
} finally {
if (state != null)
state.close();
@@ -65,7 +68,7 @@ public class LauncherDelegator extends T
/**
* Enqueue the TempletonControllerJob directly calling doAs.
*/
- public EnqueueBean enqueueController(String user, String callback,
+ public EnqueueBean enqueueController(String user, Map<String, Object> userArgs, String callback,
List<String> args)
throws NotAuthorizedException, BusyException, ExecuteException,
IOException, QueueException {
@@ -82,7 +85,7 @@ public class LauncherDelegator extends T
if (id == null)
throw new QueueException("Unable to get job id");
- registerJob(id, user, callback);
+ registerJob(id, user, callback, userArgs);
return new EnqueueBean(id);
} catch (InterruptedException e) {
Modified: hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/PigDelegator.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/PigDelegator.java?rev=1526748&r1=1526747&r2=1526748&view=diff
==============================================================================
--- hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/PigDelegator.java (original)
+++ hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/PigDelegator.java Thu Sep 26 23:29:06 2013
@@ -24,6 +24,7 @@ import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
+import java.util.Map;
import org.apache.commons.exec.ExecuteException;
import org.apache.hive.hcatalog.templeton.tool.TempletonControllerJob;
@@ -39,7 +40,7 @@ public class PigDelegator extends Launch
super(appConf);
}
- public EnqueueBean run(String user,
+ public EnqueueBean run(String user, Map<String, Object> userArgs,
String execute, String srcFile,
List<String> pigArgs, String otherFiles,
String statusdir, String callback, String completedUrl, boolean enablelog)
@@ -50,7 +51,7 @@ public class PigDelegator extends Launch
srcFile, pigArgs,
otherFiles, statusdir, completedUrl, enablelog);
- return enqueueController(user, callback, args);
+ return enqueueController(user, userArgs, callback, args);
}
private List<String> makeArgs(String execute, String srcFile,
Modified: hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/QueueStatusBean.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/QueueStatusBean.java?rev=1526748&r1=1526747&r2=1526748&view=diff
==============================================================================
--- hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/QueueStatusBean.java (original)
+++ hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/QueueStatusBean.java Thu Sep 26 23:29:06 2013
@@ -19,6 +19,7 @@
package org.apache.hive.hcatalog.templeton;
import java.io.IOException;
+import java.util.Map;
import org.apache.hadoop.mapred.JobStatus;
import org.apache.hadoop.mapred.JobProfile;
@@ -38,6 +39,7 @@ public class QueueStatusBean {
public String user;
public String callback;
public String completed;
+ public Map<String, Object> userargs;
public QueueStatusBean() {
}
@@ -63,5 +65,6 @@ public class QueueStatusBean {
user = state.getUser();
callback = state.getCallback();
completed = state.getCompleteStatus();
+ userargs = state.getUserArgs();
}
}
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=1526748&r1=1526747&r2=1526748&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 Thu Sep 26 23:29:06 2013
@@ -601,10 +601,23 @@ public class Server {
verifyParam(mapper, "mapper");
verifyParam(reducer, "reducer");
+ Map<String, Object> userArgs = new HashMap<String, Object>();
+ userArgs.put("user.name", getDoAsUser());
+ userArgs.put("input", inputs);
+ userArgs.put("output", output);
+ userArgs.put("mapper", mapper);
+ userArgs.put("reducer", reducer);
+ userArgs.put("files", files);
+ userArgs.put("define", defines);
+ userArgs.put("cmdenv", cmdenvs);
+ userArgs.put("arg", args);
+ userArgs.put("statusdir", statusdir);
+ userArgs.put("callback", callback);
+ userArgs.put("enablelog", Boolean.toString(enablelog));
checkEnableLogPrerequisite(enablelog, statusdir);
StreamingDelegator d = new StreamingDelegator(appConf);
- return d.run(getDoAsUser(), inputs, output, mapper, reducer,
+ return d.run(getDoAsUser(), userArgs, inputs, output, mapper, reducer,
files, defines, cmdenvs, args,
statusdir, callback, getCompletedUrl(), enablelog, JobType.STREAMING);
}
@@ -630,10 +643,22 @@ public class Server {
verifyParam(jar, "jar");
verifyParam(mainClass, "class");
+ Map<String, Object> userArgs = new HashMap<String, Object>();
+ userArgs.put("user.name", getDoAsUser());
+ userArgs.put("jar", jar);
+ userArgs.put("class", mainClass);
+ userArgs.put("libjars", libjars);
+ userArgs.put("files", files);
+ userArgs.put("arg", args);
+ userArgs.put("define", defines);
+ userArgs.put("statusdir", statusdir);
+ userArgs.put("callback", callback);
+ userArgs.put("enablelog", Boolean.toString(enablelog));
+
checkEnableLogPrerequisite(enablelog, statusdir);
JarDelegator d = new JarDelegator(appConf);
- return d.run(getDoAsUser(),
+ return d.run(getDoAsUser(), userArgs,
jar, mainClass,
libjars, files, args, defines,
statusdir, callback, getCompletedUrl(), enablelog, JobType.JAR);
@@ -658,10 +683,21 @@ public class Server {
if (execute == null && srcFile == null)
throw new BadParam("Either execute or file parameter required");
+ //add all function arguments to a map
+ Map<String, Object> userArgs = new HashMap<String, Object>();
+ userArgs.put("user.name", getDoAsUser());
+ userArgs.put("execute", execute);
+ userArgs.put("file", srcFile);
+ userArgs.put("arg", pigArgs);
+ userArgs.put("files", otherFiles);
+ userArgs.put("statusdir", statusdir);
+ userArgs.put("callback", callback);
+ userArgs.put("enablelog", Boolean.toString(enablelog));
+
checkEnableLogPrerequisite(enablelog, statusdir);
PigDelegator d = new PigDelegator(appConf);
- return d.run(getDoAsUser(),
+ return d.run(getDoAsUser(), userArgs,
execute, srcFile,
pigArgs, otherFiles,
statusdir, callback, getCompletedUrl(), enablelog);
@@ -699,10 +735,21 @@ public class Server {
if (execute == null && srcFile == null)
throw new BadParam("Either execute or file parameter required");
+ //add all function arguments to a map
+ Map<String, Object> userArgs = new HashMap<String, Object>();
+ userArgs.put("user.name", getDoAsUser());
+ userArgs.put("execute", execute);
+ userArgs.put("file", srcFile);
+ userArgs.put("define", defines);
+ userArgs.put("files", otherFiles);
+ userArgs.put("statusdir", statusdir);
+ userArgs.put("callback", callback);
+ userArgs.put("enablelog", Boolean.toString(enablelog));
+
checkEnableLogPrerequisite(enablelog, statusdir);
HiveDelegator d = new HiveDelegator(appConf);
- return d.run(getDoAsUser(), execute, srcFile, defines, hiveArgs, otherFiles,
+ return d.run(getDoAsUser(), userArgs, execute, srcFile, defines, hiveArgs, otherFiles,
statusdir, callback, getCompletedUrl(), enablelog);
}
Modified: hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/StreamingDelegator.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/StreamingDelegator.java?rev=1526748&r1=1526747&r2=1526748&view=diff
==============================================================================
--- hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/StreamingDelegator.java (original)
+++ hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/StreamingDelegator.java Thu Sep 26 23:29:06 2013
@@ -21,6 +21,7 @@ package org.apache.hive.hcatalog.templet
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
+import java.util.Map;
import org.apache.commons.exec.ExecuteException;
@@ -35,7 +36,7 @@ public class StreamingDelegator extends
super(appConf);
}
- public EnqueueBean run(String user,
+ public EnqueueBean run(String user, Map<String, Object> userArgs,
List<String> inputs, String output,
String mapper, String reducer,
List<String> files, List<String> defines,
@@ -52,7 +53,7 @@ public class StreamingDelegator extends
files, defines, cmdenvs, jarArgs);
JarDelegator d = new JarDelegator(appConf);
- return d.run(user,
+ return d.run(user, userArgs,
appConf.streamingJar(), null,
null, null, args, defines,
statusdir, callback, completedUrl, enableLog, jobType);
Modified: hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/JobState.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/JobState.java?rev=1526748&r1=1526747&r2=1526748&view=diff
==============================================================================
--- hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/JobState.java (original)
+++ hive/trunk/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/tool/JobState.java Thu Sep 26 23:29:06 2013
@@ -21,10 +21,12 @@ package org.apache.hive.hcatalog.templet
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
+import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hive.hcatalog.templeton.JsonBuilder;
/**
* The persistent state of a job. The state is stored in one of the
@@ -232,6 +234,20 @@ public class JobState {
setField("user", user);
}
+ @SuppressWarnings("unchecked")
+ public Map<String, Object> getUserArgs()
+ throws IOException
+ {
+ String jsonString = getField("userArgs");
+ return (Map<String, Object>)JsonBuilder.jsonToMap(jsonString);
+ }
+ public void setUserArgs(Map<String, Object> userArgs)
+ throws IOException
+ {
+ String jsonString = JsonBuilder.mapToJson(userArgs);
+ setField("userArgs", jsonString);
+ }
+
/**
* The url callback
*/