You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ac...@apache.org on 2011/12/29 09:24:10 UTC
svn commit: r1225471 - in
/hadoop/common/trunk/hadoop-tools/hadoop-streaming/src:
main/java/org/apache/hadoop/streaming/StreamJob.java
test/java/org/apache/hadoop/streaming/TestStreamJob.java
Author: acmurthy
Date: Thu Dec 29 08:24:09 2011
New Revision: 1225471
URL: http://svn.apache.org/viewvc?rev=1225471&view=rev
Log:
MAPREDUCE-3521. Fixed streaming to ensure it doesn't silently ignore unknown arguments. Contributed by Robert Evans.
Modified:
hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java
hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamJob.java
Modified: hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java?rev=1225471&r1=1225470&r2=1225471&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java (original)
+++ hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java Thu Dec 29 08:24:09 2011
@@ -255,6 +255,13 @@ public class StreamJob implements Tool {
}
if (cmdLine != null) {
+ @SuppressWarnings("unchecked")
+ List<String> args = cmdLine.getArgList();
+ if(args != null && args.size() > 0) {
+ fail("Found " + args.size() + " unexpected arguments on the " +
+ "command line " + args);
+ }
+
detailedUsage_ = cmdLine.hasOption("info");
if (cmdLine.hasOption("help") || detailedUsage_) {
printUsage = true;
Modified: hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamJob.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamJob.java?rev=1225471&r1=1225470&r2=1225471&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamJob.java (original)
+++ hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamJob.java Thu Dec 29 08:24:09 2011
@@ -32,7 +32,18 @@ import static org.junit.Assert.*;
* This class tests hadoop Streaming's StreamJob class.
*/
public class TestStreamJob {
-
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testCreateJobWithExtraArgs() throws IOException {
+ ArrayList<String> dummyArgs = new ArrayList<String>();
+ dummyArgs.add("-input"); dummyArgs.add("dummy");
+ dummyArgs.add("-output"); dummyArgs.add("dummy");
+ dummyArgs.add("-mapper"); dummyArgs.add("dummy");
+ dummyArgs.add("dummy");
+ dummyArgs.add("-reducer"); dummyArgs.add("dummy");
+ StreamJob.createJob(dummyArgs.toArray(new String[] {}));
+ }
+
@Test
public void testCreateJob() throws IOException {
JobConf job;