|
@@ -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 {
|
|
|
|
|
@@ -1111,6 +1112,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;
|