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 bo...@apache.org on 2012/02/28 18:43:09 UTC
svn commit: r1294743 - in
/hadoop/common/trunk/hadoop-tools/hadoop-streaming/src:
main/java/org/apache/hadoop/streaming/PipeMapRed.java
test/java/org/apache/hadoop/streaming/OutputOnlyApp.java
test/java/org/apache/hadoop/streaming/TestUnconsumedInput.java
Author: bobby
Date: Tue Feb 28 17:43:08 2012
New Revision: 1294743
URL: http://svn.apache.org/viewvc?rev=1294743&view=rev
Log:
MAPREDUCE-3790 Broken pipe on streaming job can lead to truncated output for a successful job (Jason Lowe via bobby)
Added:
hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/OutputOnlyApp.java
hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestUnconsumedInput.java
Modified:
hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/PipeMapRed.java
Modified: hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/PipeMapRed.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/PipeMapRed.java?rev=1294743&r1=1294742&r2=1294743&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/PipeMapRed.java (original)
+++ hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/PipeMapRed.java Tue Feb 28 17:43:08 2012
@@ -521,11 +521,15 @@ public abstract class PipeMapRed {
LOG.info("mapRedFinished");
return;
}
- try {
- if (clientOut_ != null) {
+ if (clientOut_ != null) {
+ try {
clientOut_.flush();
clientOut_.close();
+ } catch (IOException io) {
+ LOG.warn(io);
}
+ }
+ try {
waitOutputThreads();
} catch (IOException io) {
LOG.warn(io);
Added: hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/OutputOnlyApp.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/OutputOnlyApp.java?rev=1294743&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/OutputOnlyApp.java (added)
+++ hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/OutputOnlyApp.java Tue Feb 28 17:43:08 2012
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming;
+
+import java.io.IOException;
+
+/**
+ * An application that outputs a specified number of lines
+ * without consuming any input.
+ */
+public class OutputOnlyApp {
+ public static void main(String[] args) throws IOException {
+ if (args.length < 1) {
+ System.err.println("Usage: OutputOnlyApp NUMRECORDS");
+ return;
+ }
+ int numRecords = Integer.parseInt(args[0]);
+ while (numRecords-- > 0) {
+ System.out.println("key\tvalue");
+ }
+ }
+}
Added: hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestUnconsumedInput.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestUnconsumedInput.java?rev=1294743&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestUnconsumedInput.java (added)
+++ hadoop/common/trunk/hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestUnconsumedInput.java Tue Feb 28 17:43:08 2012
@@ -0,0 +1,107 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming;
+
+import static org.junit.Assert.*;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.Test;
+
+public class TestUnconsumedInput {
+ protected final int EXPECTED_OUTPUT_SIZE = 10000;
+ protected File INPUT_FILE = new File("stream_uncinput_input.txt");
+ protected File OUTPUT_DIR = new File("stream_uncinput_out");
+ // map parses input lines and generates count entries for each word.
+ protected String input = "roses.are.red\nviolets.are.blue\nbunnies.are.pink\n";
+ protected String map = UtilTest.makeJavaCommand(OutputOnlyApp.class,
+ new String[]{Integer.toString(EXPECTED_OUTPUT_SIZE)});
+
+ private StreamJob job;
+
+ public TestUnconsumedInput() throws IOException
+ {
+ UtilTest utilTest = new UtilTest(getClass().getName());
+ utilTest.checkUserDir();
+ utilTest.redirectIfAntJunit();
+ }
+
+ protected void createInput() throws IOException
+ {
+ DataOutputStream out = new DataOutputStream(
+ new FileOutputStream(INPUT_FILE.getAbsoluteFile()));
+ for (int i=0; i<10000; ++i) {
+ out.write(input.getBytes("UTF-8"));
+ }
+ out.close();
+ }
+
+ protected String[] genArgs() {
+ return new String[] {
+ "-input", INPUT_FILE.getAbsolutePath(),
+ "-output", OUTPUT_DIR.getAbsolutePath(),
+ "-mapper", map,
+ "-reducer", "org.apache.hadoop.mapred.lib.IdentityReducer",
+ "-numReduceTasks", "0",
+ "-jobconf", "mapreduce.task.files.preserve.failedtasks=true",
+ "-jobconf", "stream.tmpdir="+System.getProperty("test.build.data","/tmp")
+ };
+ }
+
+ @Test
+ public void testUnconsumedInput() throws Exception
+ {
+ String outFileName = "part-00000";
+ File outFile = null;
+ try {
+ try {
+ FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
+ } catch (Exception e) {
+ }
+
+ createInput();
+
+ // setup config to ignore unconsumed input
+ Configuration conf = new Configuration();
+ conf.set("stream.minRecWrittenToEnableSkip_", "0");
+
+ job = new StreamJob();
+ job.setConf(conf);
+ int exitCode = job.run(genArgs());
+ assertEquals("Job failed", 0, exitCode);
+ outFile = new File(OUTPUT_DIR, outFileName).getAbsoluteFile();
+ String output = StreamUtil.slurp(outFile);
+ assertEquals("Output was truncated", EXPECTED_OUTPUT_SIZE,
+ StringUtils.countMatches(output, "\t"));
+ } finally {
+ INPUT_FILE.delete();
+ FileUtil.fullyDelete(OUTPUT_DIR.getAbsoluteFile());
+ }
+ }
+}