Merge -r 1297292:1297293 from trunk to branch. FIXES: MAPREDUCE-3952

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1297299 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Alejandro Abdelnur 2012-03-06 00:00:36 +00:00
parent afc35c31ea
commit a7899dae53
3 changed files with 34 additions and 2 deletions

View File

@ -46,6 +46,9 @@ Release 0.23.3 - UNRELEASED
MAPREDUCE-3728. ShuffleHandler can't access results when configured in a
secure mode (ahmed via tucu)
MAPREDUCE-3952. In MR2, when Total input paths to process == 1,
CombinefileInputFormat.getSplits() returns 0 split. (zhenxiao via tucu)
Release 0.23.2 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -514,7 +514,7 @@ public abstract class CombineFileInputFormat<K, V>
long left = locations[i].getLength();
long myOffset = locations[i].getOffset();
long myLength = 0;
while (left > 0) {
do {
if (maxSize == 0) {
myLength = left;
} else {
@ -536,7 +536,7 @@ public abstract class CombineFileInputFormat<K, V>
myOffset += myLength;
blocksList.add(oneblock);
}
} while (left > 0);
}
blocks = blocksList.toArray(new OneBlockInfo[blocksList.size()]);
}

View File

@ -42,6 +42,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
import org.junit.Test;
public class TestCombineFileInputFormat extends TestCase {
@ -1112,6 +1113,34 @@ public class TestCombineFileInputFormat extends TestCase {
}
}
/**
* Test when the input file's length is 0.
*/
@Test
public void testForEmptyFile() throws Exception {
Configuration conf = new Configuration();
FileSystem fileSys = FileSystem.get(conf);
Path file = new Path("test" + "/file");
FSDataOutputStream out = fileSys.create(file, true,
conf.getInt("io.file.buffer.size", 4096), (short) 1, (long) BLOCKSIZE);
out.write(new byte[0]);
out.close();
// split it using a CombinedFile input format
DummyInputFormat inFormat = new DummyInputFormat();
Job job = Job.getInstance(conf);
FileInputFormat.setInputPaths(job, "test");
List<InputSplit> splits = inFormat.getSplits(job);
assertEquals(splits.size(), 1);
CombineFileSplit fileSplit = (CombineFileSplit) splits.get(0);
assertEquals(1, fileSplit.getNumPaths());
assertEquals(file.getName(), fileSplit.getPath(0).getName());
assertEquals(0, fileSplit.getOffset(0));
assertEquals(0, fileSplit.getLength(0));
fileSys.delete(file.getParent(), true);
}
static class TestFilter implements PathFilter {
private Path p;