MAPREDUCE-6048. Fixed TestJavaSerialization failure. Contributed by Varun Vasudev

(cherry picked from commit 73068f677b)
This commit is contained in:
Jian He 2014-11-04 20:19:54 -08:00
parent b96fefc032
commit 944fb1c3a9
2 changed files with 26 additions and 11 deletions

View File

@ -237,6 +237,9 @@ Release 2.6.0 - UNRELEASED
MAPREDUCE-6022. map_input_file is missing from streaming job environment. MAPREDUCE-6022. map_input_file is missing from streaming job environment.
(jlowe via kihwal) (jlowe via kihwal)
MAPREDUCE-6048. Fixed TestJavaSerialization failure. (Varun Vasudev via
jianhe)
Release 2.5.2 - UNRELEASED Release 2.5.2 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -17,11 +17,9 @@
*/ */
package org.apache.hadoop.mapred; package org.apache.hadoop.mapred;
import java.io.BufferedReader;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.InputStreamReader;
import java.io.OutputStream; import java.io.OutputStream;
import java.io.OutputStreamWriter; import java.io.OutputStreamWriter;
import java.io.Writer; import java.io.Writer;
@ -30,6 +28,7 @@ import java.util.StringTokenizer;
import junit.framework.TestCase; import junit.framework.TestCase;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@ -56,7 +55,10 @@ public class TestJavaSerialization extends TestCase {
throws IOException { throws IOException {
StringTokenizer st = new StringTokenizer(value.toString()); StringTokenizer st = new StringTokenizer(value.toString());
while (st.hasMoreTokens()) { while (st.hasMoreTokens()) {
output.collect(st.nextToken(), 1L); String token = st.nextToken();
assertTrue("Invalid token; expected 'a' or 'b', got " + token,
token.equals("a") || token.equals("b"));
output.collect(token, 1L);
} }
} }
@ -116,18 +118,28 @@ public class TestJavaSerialization extends TestCase {
FileOutputFormat.setOutputPath(conf, OUTPUT_DIR); FileOutputFormat.setOutputPath(conf, OUTPUT_DIR);
String inputFileContents =
FileUtils.readFileToString(new File(INPUT_FILE.toUri().getPath()));
assertTrue("Input file contents not as expected; contents are '"
+ inputFileContents + "', expected \"b a\n\" ",
inputFileContents.equals("b a\n"));
JobClient.runJob(conf); JobClient.runJob(conf);
Path[] outputFiles = FileUtil.stat2Paths( Path[] outputFiles =
fs.listStatus(OUTPUT_DIR, FileUtil.stat2Paths(fs.listStatus(OUTPUT_DIR,
new Utils.OutputFileUtils.OutputFilesFilter())); new Utils.OutputFileUtils.OutputFilesFilter()));
assertEquals(1, outputFiles.length); assertEquals(1, outputFiles.length);
InputStream is = fs.open(outputFiles[0]); InputStream is = fs.open(outputFiles[0]);
BufferedReader reader = new BufferedReader(new InputStreamReader(is)); String reduceOutput = org.apache.commons.io.IOUtils.toString(is);
assertEquals("a\t1", reader.readLine()); String[] lines = reduceOutput.split(System.getProperty("line.separator"));
assertEquals("b\t1", reader.readLine()); assertEquals("Unexpected output; received output '" + reduceOutput + "'",
assertNull(reader.readLine()); "a\t1", lines[0]);
reader.close(); assertEquals("Unexpected output; received output '" + reduceOutput + "'",
"b\t1", lines[1]);
assertEquals("Reduce output has extra lines; output is '" + reduceOutput
+ "'", 2, lines.length);
is.close();
} }
/** /**