You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-commits@hadoop.apache.org by ma...@apache.org on 2011/11/14 23:44:44 UTC
svn commit: r1201951 - in /hadoop/common/trunk/hadoop-mapreduce-project:
CHANGES.txt
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java
Author: mahadev
Date: Mon Nov 14 22:44:44 2011
New Revision: 1201951
URL: http://svn.apache.org/viewvc?rev=1201951&view=rev
Log:
MAPREDUCE-3243. Invalid tracking URL for streaming jobs (Jonathan Eagles via mahadev)
Modified:
hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt
hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java
Modified: hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt?rev=1201951&r1=1201950&r2=1201951&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt Mon Nov 14 22:44:44 2011
@@ -82,6 +82,9 @@ Release 0.23.1 - Unreleased
MAPREDUCE-3341. Enhance logging of initalized queue limit values.
(Anupam Seth via mahadev)
+ MAPREDUCE-3243. Invalid tracking URL for streaming jobs (Jonathan Eagles
+ via mahadev)
+
OPTIMIZATIONS
BUG FIXES
Modified: hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java?rev=1201951&r1=1201950&r2=1201951&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java Mon Nov 14 22:44:44 2011
@@ -984,19 +984,6 @@ public class StreamJob implements Tool {
return jobConf_.get(JTConfig.JT_IPC_ADDRESS);
}
- protected void jobInfo() {
- if (isLocalHadoop()) {
- LOG.info("Job running in-process (local Hadoop)");
- } else {
- String hp = getJobTrackerHostPort();
- LOG.info("To kill this job, run:");
- LOG.info(getHadoopClientHome() + "/bin/hadoop job -D" + JTConfig.JT_IPC_ADDRESS + "=" + hp + " -kill "
- + jobId_);
- //LOG.info("Job file: " + running_.getJobFile());
- LOG.info("Tracking URL: " + StreamUtil.qualifyHost(running_.getTrackingURL()));
- }
- }
-
// Based on JobClient
public int submitAndMonitorJob() throws IOException {
@@ -1012,7 +999,6 @@ public class StreamJob implements Tool {
try {
running_ = jc_.submitJob(jobConf_);
jobId_ = running_.getID();
- jobInfo();
if (background_) {
LOG.info("Job is running in background.");
} else if (!jc_.monitorAndPrintJob(jobConf_, running_)) {