|
@@ -19,6 +19,8 @@
|
|
|
package org.apache.hadoop.mapred;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.util.ArrayList;
|
|
|
+import java.util.List;
|
|
|
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
@@ -42,35 +44,40 @@ public abstract class MultiFileInputFormat<K extends WritableComparable,
|
|
|
public InputSplit[] getSplits(JobConf job, int numSplits)
|
|
|
throws IOException {
|
|
|
|
|
|
- MultiFileSplit[] splits = new MultiFileSplit[numSplits];
|
|
|
Path[] paths = listPaths(job);
|
|
|
- long[] lengths = new long[paths.length];
|
|
|
- long totLength = 0;
|
|
|
- for(int i=0; i<paths.length; i++) {
|
|
|
- FileSystem fs = paths[i].getFileSystem(job);
|
|
|
- lengths[i] = fs.getContentLength(paths[i]);
|
|
|
- totLength += lengths[i];
|
|
|
- }
|
|
|
- float avgLengthPerSplit = ((float)totLength) / numSplits;
|
|
|
- long cumulativeLength = 0;
|
|
|
+ List<MultiFileSplit> splits = new ArrayList<MultiFileSplit>(Math.min(numSplits, paths.length));
|
|
|
+ if (paths.length != 0) {
|
|
|
+ // HADOOP-1818: Manage splits only if there are paths
|
|
|
+ long[] lengths = new long[paths.length];
|
|
|
+ long totLength = 0;
|
|
|
+ for(int i=0; i<paths.length; i++) {
|
|
|
+ FileSystem fs = paths[i].getFileSystem(job);
|
|
|
+ lengths[i] = fs.getContentLength(paths[i]);
|
|
|
+ totLength += lengths[i];
|
|
|
+ }
|
|
|
+ float avgLengthPerSplit = ((float)totLength) / numSplits;
|
|
|
+ long cumulativeLength = 0;
|
|
|
|
|
|
- int startIndex = 0;
|
|
|
+ int startIndex = 0;
|
|
|
|
|
|
- for(int i=0; i<numSplits; i++) {
|
|
|
- int splitSize = findSize(i, avgLengthPerSplit, cumulativeLength
|
|
|
- , startIndex, lengths);
|
|
|
- Path[] splitPaths = new Path[splitSize];
|
|
|
- long[] splitLengths = new long[splitSize];
|
|
|
- System.arraycopy(paths, startIndex, splitPaths , 0, splitSize);
|
|
|
- System.arraycopy(lengths, startIndex, splitLengths , 0, splitSize);
|
|
|
- splits[i] = new MultiFileSplit(job, splitPaths, splitLengths);
|
|
|
- startIndex += splitSize;
|
|
|
- for(long l: splitLengths) {
|
|
|
- cumulativeLength += l;
|
|
|
+ for(int i=0; i<numSplits; i++) {
|
|
|
+ int splitSize = findSize(i, avgLengthPerSplit, cumulativeLength
|
|
|
+ , startIndex, lengths);
|
|
|
+ if (splitSize != 0) {
|
|
|
+ // HADOOP-1818: Manage split only if split size is not equals to 0
|
|
|
+ Path[] splitPaths = new Path[splitSize];
|
|
|
+ long[] splitLengths = new long[splitSize];
|
|
|
+ System.arraycopy(paths, startIndex, splitPaths , 0, splitSize);
|
|
|
+ System.arraycopy(lengths, startIndex, splitLengths , 0, splitSize);
|
|
|
+ splits.add(new MultiFileSplit(job, splitPaths, splitLengths));
|
|
|
+ startIndex += splitSize;
|
|
|
+ for(long l: splitLengths) {
|
|
|
+ cumulativeLength += l;
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
- return splits;
|
|
|
-
|
|
|
+ return splits.toArray(new MultiFileSplit[splits.size()]);
|
|
|
}
|
|
|
|
|
|
private int findSize(int splitIndex, float avgLengthPerSplit
|