MAPREDUCE-3952. In MR2, when Total input paths to process == 1, CombinefileInputFormat.getSplits() returns 0 split. (zhenxiao via tucu)
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1297293 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
2b634e92a7
commit
739f688f5a
@ -128,6 +128,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
|
||||
|
@ -514,7 +514,7 @@ private static class OneFileInfo {
|
||||
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 @@ private static class OneFileInfo {
|
||||
myOffset += myLength;
|
||||
|
||||
blocksList.add(oneblock);
|
||||
}
|
||||
} while (left > 0);
|
||||
}
|
||||
blocks = blocksList.toArray(new OneBlockInfo[blocksList.size()]);
|
||||
}
|
||||
|
@ -42,6 +42,7 @@
|
||||
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 {
|
||||
|
||||
@ -1111,6 +1112,34 @@ public void testMissingBlocks() throws IOException {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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;
|
||||
|
Loading…
Reference in New Issue
Block a user