MAPREDUCE-4969. TestKeyValueTextInputFormat test fails with Open JDK 7. Contributed by Arpit Agarwal.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1441069 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Suresh Srinivas 2013-01-31 17:11:36 +00:00
parent e7380b4f9c
commit ac21e15a39
2 changed files with 56 additions and 31 deletions

View File

@ -281,6 +281,9 @@ Release 2.0.3-alpha - Unreleased
MAPREDUCE-2264. Job status exceeds 100% in some cases.
(devaraj.k and sandyr via tucu)
MAPREDUCE-4969. TestKeyValueTextInputFormat test fails with Open JDK 7.
(Arpit Agarwal via suresh)
Release 2.0.2-alpha - 2012-09-07
INCOMPATIBLE CHANGES

View File

@ -136,32 +136,47 @@ public class TestKeyValueTextInputFormat extends TestCase {
}
public void testUTF8() throws Exception {
LineReader in = makeStream("abcd\u20acbdcd\u20ac");
Text line = new Text();
in.readLine(line);
assertEquals("readLine changed utf8 characters",
"abcd\u20acbdcd\u20ac", line.toString());
in = makeStream("abc\u200axyz");
in.readLine(line);
assertEquals("split on fake newline", "abc\u200axyz", line.toString());
LineReader in = null;
try {
in = makeStream("abcd\u20acbdcd\u20ac");
Text line = new Text();
in.readLine(line);
assertEquals("readLine changed utf8 characters",
"abcd\u20acbdcd\u20ac", line.toString());
in = makeStream("abc\u200axyz");
in.readLine(line);
assertEquals("split on fake newline", "abc\u200axyz", line.toString());
} finally {
if (in != null) {
in.close();
}
}
}
public void testNewLines() throws Exception {
LineReader in = makeStream("a\nbb\n\nccc\rdddd\r\neeeee");
Text out = new Text();
in.readLine(out);
assertEquals("line1 length", 1, out.getLength());
in.readLine(out);
assertEquals("line2 length", 2, out.getLength());
in.readLine(out);
assertEquals("line3 length", 0, out.getLength());
in.readLine(out);
assertEquals("line4 length", 3, out.getLength());
in.readLine(out);
assertEquals("line5 length", 4, out.getLength());
in.readLine(out);
assertEquals("line5 length", 5, out.getLength());
assertEquals("end of file", 0, in.readLine(out));
LineReader in = null;
try {
in = makeStream("a\nbb\n\nccc\rdddd\r\neeeee");
Text out = new Text();
in.readLine(out);
assertEquals("line1 length", 1, out.getLength());
in.readLine(out);
assertEquals("line2 length", 2, out.getLength());
in.readLine(out);
assertEquals("line3 length", 0, out.getLength());
in.readLine(out);
assertEquals("line4 length", 3, out.getLength());
in.readLine(out);
assertEquals("line5 length", 4, out.getLength());
in.readLine(out);
assertEquals("line5 length", 5, out.getLength());
assertEquals("end of file", 0, in.readLine(out));
} finally {
if (in != null) {
in.close();
}
}
}
private static void writeFile(FileSystem fs, Path name,
@ -183,13 +198,20 @@ public class TestKeyValueTextInputFormat extends TestCase {
InputSplit split,
JobConf job) throws IOException {
List<Text> result = new ArrayList<Text>();
RecordReader<Text, Text> reader = format.getRecordReader(split, job,
voidReporter);
Text key = reader.createKey();
Text value = reader.createValue();
while (reader.next(key, value)) {
result.add(value);
value = reader.createValue();
RecordReader<Text, Text> reader = null;
try {
reader = format.getRecordReader(split, job, voidReporter);
Text key = reader.createKey();
Text value = reader.createValue();
while (reader.next(key, value)) {
result.add(value);
value = (Text) reader.createValue();
}
} finally {
if (reader != null) {
reader.close();
}
}
return result;
}