|
@@ -33,7 +33,9 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.BlockLocation;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
+import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.mapreduce.InputSplit;
|
|
|
import org.apache.hadoop.mapreduce.Job;
|
|
|
import org.apache.hadoop.mapreduce.JobContext;
|
|
@@ -136,6 +138,43 @@ public class TestFileInputFormat {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * 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) 1024);
|
|
|
+ out.write(new byte[0]);
|
|
|
+ out.close();
|
|
|
+
|
|
|
+ // split it using a File input format
|
|
|
+ DummyInputFormat inFormat = new DummyInputFormat();
|
|
|
+ Job job = Job.getInstance(conf);
|
|
|
+ FileInputFormat.setInputPaths(job, "test");
|
|
|
+ List<InputSplit> splits = inFormat.getSplits(job);
|
|
|
+ assertEquals(1, splits.size());
|
|
|
+ FileSplit fileSplit = (FileSplit) splits.get(0);
|
|
|
+ assertEquals(0, fileSplit.getLocations().length);
|
|
|
+ assertEquals(file.getName(), fileSplit.getPath().getName());
|
|
|
+ assertEquals(0, fileSplit.getStart());
|
|
|
+ assertEquals(0, fileSplit.getLength());
|
|
|
+
|
|
|
+ fileSys.delete(file.getParent(), true);
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Dummy class to extend FileInputFormat*/
|
|
|
+ private class DummyInputFormat extends FileInputFormat<Text, Text> {
|
|
|
+ @Override
|
|
|
+ public RecordReader<Text,Text> createRecordReader(InputSplit split,
|
|
|
+ TaskAttemptContext context) throws IOException {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private class FileInputFormatForTest<K, V> extends FileInputFormat<K, V> {
|
|
|
|
|
|
long splitSize;
|