MAPREDUCE-6048. Fixed TestJavaSerialization failure. Contributed by Varun Vasudev
This commit is contained in:
parent
0762b4a3fb
commit
73068f677b
@ -456,6 +456,9 @@ Release 2.6.0 - UNRELEASED
|
||||
MAPREDUCE-6022. map_input_file is missing from streaming job environment.
|
||||
(jlowe via kihwal)
|
||||
|
||||
MAPREDUCE-6048. Fixed TestJavaSerialization failure. (Varun Vasudev via
|
||||
jianhe)
|
||||
|
||||
Release 2.5.2 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -17,11 +17,9 @@
|
||||
*/
|
||||
package org.apache.hadoop.mapred;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.InputStreamReader;
|
||||
import java.io.OutputStream;
|
||||
import java.io.OutputStreamWriter;
|
||||
import java.io.Writer;
|
||||
@ -30,6 +28,7 @@
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@ -56,7 +55,10 @@ public void map(LongWritable key, Text value,
|
||||
throws IOException {
|
||||
StringTokenizer st = new StringTokenizer(value.toString());
|
||||
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 void testMapReduceJob() throws Exception {
|
||||
|
||||
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);
|
||||
|
||||
Path[] outputFiles = FileUtil.stat2Paths(
|
||||
fs.listStatus(OUTPUT_DIR,
|
||||
new Utils.OutputFileUtils.OutputFilesFilter()));
|
||||
Path[] outputFiles =
|
||||
FileUtil.stat2Paths(fs.listStatus(OUTPUT_DIR,
|
||||
new Utils.OutputFileUtils.OutputFilesFilter()));
|
||||
assertEquals(1, outputFiles.length);
|
||||
InputStream is = fs.open(outputFiles[0]);
|
||||
BufferedReader reader = new BufferedReader(new InputStreamReader(is));
|
||||
assertEquals("a\t1", reader.readLine());
|
||||
assertEquals("b\t1", reader.readLine());
|
||||
assertNull(reader.readLine());
|
||||
reader.close();
|
||||
String reduceOutput = org.apache.commons.io.IOUtils.toString(is);
|
||||
String[] lines = reduceOutput.split(System.getProperty("line.separator"));
|
||||
assertEquals("Unexpected output; received output '" + reduceOutput + "'",
|
||||
"a\t1", lines[0]);
|
||||
assertEquals("Unexpected output; received output '" + reduceOutput + "'",
|
||||
"b\t1", lines[1]);
|
||||
assertEquals("Reduce output has extra lines; output is '" + reduceOutput
|
||||
+ "'", 2, lines.length);
|
||||
is.close();
|
||||
}
|
||||
|
||||
/**
|
||||
|
Loading…
Reference in New Issue
Block a user