Author: acmurthy Date: Wed Nov 7 11:12:40 2007 New Revision: 592857 URL: http://svn.apache.org/viewvc?rev=592857&view=rev Log: HADOOP-1952. Improve handling of invalid, user-specified classes while configuring streaming jobs such as combiner, input/output formats etc. Now invalid options are caught, logged and jobs are failed early. Contributed by Lohit Vijayarenu.
Modified: lucene/hadoop/trunk/CHANGES.txt lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java lucene/hadoop/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestGzipInput.java lucene/hadoop/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreaming.java lucene/hadoop/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingFailure.java Modified: lucene/hadoop/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?rev=592857&r1=592856&r2=592857&view=diff ============================================================================== --- lucene/hadoop/trunk/CHANGES.txt (original) +++ lucene/hadoop/trunk/CHANGES.txt Wed Nov 7 11:12:40 2007 @@ -79,6 +79,11 @@ HADOOP-1348. Allow XML comments inside configuration files. (Rajagopal Natarajan and Enis Soztutar via enis) + HADOOP-1952. Improve handling of invalid, user-specified classes while + configuring streaming jobs such as combiner, input/output formats etc. + Now invalid options are caught, logged and jobs are failed early. (Lohit + Vijayarenu via acmurthy) + Release 0.15.1 - BUG FIXES Modified: lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java?rev=592857&r1=592856&r2=592857&view=diff ============================================================================== --- lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java (original) +++ lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java Wed Nov 7 11:12:40 2007 @@ -693,28 +693,33 @@ fmt = TextInputFormat.class; } else if (inputFormatSpec_ != null) { if (inputFormatSpec_.equals(TextInputFormat.class.getName()) - || inputFormatSpec_.equals(TextInputFormat.class.getCanonicalName())) { + || inputFormatSpec_.equals(TextInputFormat.class.getCanonicalName()) + || inputFormatSpec_.equals(TextInputFormat.class.getSimpleName())) { fmt = TextInputFormat.class; } else if (inputFormatSpec_.equals(KeyValueTextInputFormat.class .getName()) || inputFormatSpec_.equals(KeyValueTextInputFormat.class - .getCanonicalName())) { + .getCanonicalName()) + || inputFormatSpec_.equals(KeyValueTextInputFormat.class.getSimpleName())) { } else if (inputFormatSpec_.equals(SequenceFileInputFormat.class .getName()) || inputFormatSpec_ .equals(org.apache.hadoop.mapred.SequenceFileInputFormat.class - .getCanonicalName())) { + .getCanonicalName()) + || inputFormatSpec_ + .equals(org.apache.hadoop.mapred.SequenceFileInputFormat.class.getSimpleName())) { } else if (inputFormatSpec_.equals(SequenceFileAsTextInputFormat.class .getName()) || inputFormatSpec_.equals(SequenceFileAsTextInputFormat.class - .getCanonicalName())) { + .getCanonicalName()) + || inputFormatSpec_.equals(SequenceFileAsTextInputFormat.class.getSimpleName())) { fmt = SequenceFileAsTextInputFormat.class; } else { c = StreamUtil.goodClassOrNull(inputFormatSpec_, defaultPackage); if (c != null) { fmt = c; } else { - + fail("-inputformat : class not found : " + inputFormatSpec_); } } } @@ -744,7 +749,9 @@ c = StreamUtil.goodClassOrNull(comCmd_, defaultPackage); if (c != null) { jobConf_.setCombinerClass(c); - } + } else { + fail("-combiner : class not found : " + comCmd_); + } } boolean reducerNone_ = false; @@ -791,7 +798,9 @@ c = StreamUtil.goodClassOrNull(outputFormatSpec_, defaultPackage); if (c != null) { fmt = c; - } + } else { + fail("-outputformat : class not found : " + outputFormatSpec_); + } } if (fmt == null) { fmt = TextOutputFormat.class; @@ -802,7 +811,9 @@ c = StreamUtil.goodClassOrNull(partitionerSpec_, defaultPackage); if (c != null) { jobConf_.setPartitionerClass(c); - } + } else { + fail("-partitioner : class not found : " + partitionerSpec_); + } } if (numReduceTasksSpec_!= null) { Modified: lucene/hadoop/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestGzipInput.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestGzipInput.java?rev=592857&r1=592856&r2=592857&view=diff ============================================================================== --- lucene/hadoop/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestGzipInput.java (original) +++ lucene/hadoop/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestGzipInput.java Wed Nov 7 11:12:40 2007 @@ -47,7 +47,6 @@ "-input", INPUT_FILE.getAbsolutePath(), "-output", OUTPUT_DIR.getAbsolutePath(), "-mapper", map, - "-combiner", combine, "-reducer", reduce, "-jobconf", "stream.recordreader.compression=gzip" }; Modified: lucene/hadoop/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreaming.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreaming.java?rev=592857&r1=592856&r2=592857&view=diff ============================================================================== --- lucene/hadoop/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreaming.java (original) +++ lucene/hadoop/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreaming.java Wed Nov 7 11:12:40 2007 @@ -40,7 +40,6 @@ protected String map = StreamUtil.makeJavaCommand(TrApp.class, new String[]{".", "\\n"}); // reduce behave like /usr/bin/uniq. But also prepend lines with R. // command-line combiner does not have any effect any more. - protected String combine = StreamUtil.makeJavaCommand(UniqApp.class, new String[]{"C"}); protected String reduce = StreamUtil.makeJavaCommand(UniqApp.class, new String[]{"R"}); protected String outputExpect = "Rare\t\nRblue\t\nRbunnies\t\nRpink\t\nRred\t\nRroses\t\nRviolets\t\n"; @@ -66,7 +65,6 @@ "-input", INPUT_FILE.getAbsolutePath(), "-output", OUTPUT_DIR.getAbsolutePath(), "-mapper", map, - "-combiner", combine, "-reducer", reduce, //"-verbose", //"-jobconf", "stream.debug=set" Modified: lucene/hadoop/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingFailure.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingFailure.java?rev=592857&r1=592856&r2=592857&view=diff ============================================================================== --- lucene/hadoop/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingFailure.java (original) +++ lucene/hadoop/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingFailure.java Wed Nov 7 11:12:40 2007 @@ -47,7 +47,6 @@ "-input", INVALID_INPUT_FILE.getAbsolutePath(), "-output", OUTPUT_DIR.getAbsolutePath(), "-mapper", map, - "-combiner", combine, "-reducer", reduce, //"-verbose", //"-jobconf", "stream.debug=set"