|
@@ -18,9 +18,11 @@
|
|
|
|
|
|
package org.apache.hadoop.mapred;
|
|
|
|
|
|
+import java.io.FileNotFoundException;
|
|
|
import java.io.IOException;
|
|
|
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.List;
|
|
|
|
|
|
import org.apache.commons.logging.*;
|
|
|
|
|
@@ -52,8 +54,7 @@ public abstract class InputFormatBase implements InputFormat {
|
|
|
return true;
|
|
|
}
|
|
|
|
|
|
- public abstract RecordReader getRecordReader(FileSystem fs,
|
|
|
- FileSplit split,
|
|
|
+ public abstract RecordReader getRecordReader(InputSplit split,
|
|
|
JobConf job,
|
|
|
Reporter reporter)
|
|
|
throws IOException;
|
|
@@ -66,15 +67,17 @@ public abstract class InputFormatBase implements InputFormat {
|
|
|
* that is appended to all input dirs specified by job, and if the given fs
|
|
|
* lists those too, each is added to the returned array of Path.
|
|
|
*
|
|
|
- * @param fs
|
|
|
- * @param job
|
|
|
- * @return array of Path objects, never zero length.
|
|
|
+ * @param job the job to list input paths for
|
|
|
+ * @return array of Path objects
|
|
|
* @throws IOException if zero items.
|
|
|
*/
|
|
|
- protected Path[] listPaths(FileSystem ignored, JobConf job)
|
|
|
+ protected Path[] listPaths(JobConf job)
|
|
|
throws IOException {
|
|
|
- Path[] dirs = job.getInputPaths();
|
|
|
String subdir = job.get("mapred.input.subdir");
|
|
|
+ Path[] dirs = job.getInputPaths();
|
|
|
+ if (dirs.length == 0) {
|
|
|
+ throw new IOException("No input directories specified in job");
|
|
|
+ }
|
|
|
ArrayList result = new ArrayList();
|
|
|
for (int i = 0; i < dirs.length; i++) {
|
|
|
FileSystem fs = dirs[i].getFileSystem(job);
|
|
@@ -96,30 +99,34 @@ public abstract class InputFormatBase implements InputFormat {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- if (result.size() == 0) {
|
|
|
- throw new IOException("No input directories specified in: "+job);
|
|
|
- }
|
|
|
return (Path[])result.toArray(new Path[result.size()]);
|
|
|
}
|
|
|
|
|
|
- // NOTE: should really pass a Configuration here, not a FileSystem
|
|
|
- public boolean[] areValidInputDirectories(FileSystem fs, Path[] inputDirs)
|
|
|
- throws IOException {
|
|
|
- boolean[] result = new boolean[inputDirs.length];
|
|
|
+ public void validateInput(JobConf job) throws IOException {
|
|
|
+ Path[] inputDirs = job.getInputPaths();
|
|
|
+ List<IOException> result = new ArrayList();
|
|
|
for(int i=0; i < inputDirs.length; ++i) {
|
|
|
- result[i] =
|
|
|
- inputDirs[i].getFileSystem(fs.getConf()).isDirectory(inputDirs[i]);
|
|
|
+ FileSystem fs = inputDirs[i].getFileSystem(job);
|
|
|
+ if (!fs.exists(inputDirs[i])) {
|
|
|
+ result.add(new FileNotFoundException("Input directory " +
|
|
|
+ inputDirs[i] +
|
|
|
+ " doesn't exist."));
|
|
|
+ } else if (!fs.isDirectory(inputDirs[i])) {
|
|
|
+ result.add(new InvalidFileTypeException
|
|
|
+ ("Invalid input path, expecting directory : " +
|
|
|
+ inputDirs[i]));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (!result.isEmpty()) {
|
|
|
+ throw new InvalidInputException(result);
|
|
|
}
|
|
|
- return result;
|
|
|
}
|
|
|
|
|
|
- /** Splits files returned by {@link #listPaths(FileSystem,JobConf)} when
|
|
|
+ /** Splits files returned by {@link #listPaths(JobConf)} when
|
|
|
* they're too big.*/
|
|
|
- public FileSplit[] getSplits(FileSystem ignored, JobConf job, int numSplits)
|
|
|
+ public InputSplit[] getSplits(JobConf job, int numSplits)
|
|
|
throws IOException {
|
|
|
-
|
|
|
- Path[] files = listPaths(ignored, job);
|
|
|
-
|
|
|
+ Path[] files = listPaths(job);
|
|
|
long totalSize = 0; // compute total size
|
|
|
for (int i = 0; i < files.length; i++) { // check we have valid files
|
|
|
Path file = files[i];
|
|
@@ -145,16 +152,18 @@ public abstract class InputFormatBase implements InputFormat {
|
|
|
|
|
|
long bytesRemaining = length;
|
|
|
while (((double) bytesRemaining)/splitSize > SPLIT_SLOP) {
|
|
|
- splits.add(new FileSplit(file, length-bytesRemaining, splitSize));
|
|
|
+ splits.add(new FileSplit(file, length-bytesRemaining, splitSize,
|
|
|
+ job));
|
|
|
bytesRemaining -= splitSize;
|
|
|
}
|
|
|
|
|
|
if (bytesRemaining != 0) {
|
|
|
- splits.add(new FileSplit(file, length-bytesRemaining, bytesRemaining));
|
|
|
+ splits.add(new FileSplit(file, length-bytesRemaining,
|
|
|
+ bytesRemaining, job));
|
|
|
}
|
|
|
} else {
|
|
|
if (length != 0) {
|
|
|
- splits.add(new FileSplit(file, 0, length));
|
|
|
+ splits.add(new FileSplit(file, 0, length, job));
|
|
|
}
|
|
|
}
|
|
|
}
|