|
@@ -18,18 +18,18 @@
|
|
|
|
|
|
package org.apache.hadoop.mapred;
|
|
package org.apache.hadoop.mapred;
|
|
|
|
|
|
-import java.io.FileNotFoundException;
|
|
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
|
|
+import java.util.Arrays;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
|
+import org.apache.hadoop.fs.FileStatus;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.PathFilter;
|
|
import org.apache.hadoop.fs.PathFilter;
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
-import org.apache.hadoop.fs.FileStatus;
|
|
|
|
import org.apache.hadoop.fs.BlockLocation;
|
|
import org.apache.hadoop.fs.BlockLocation;
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
import org.apache.hadoop.util.StringUtils;
|
|
import org.apache.hadoop.util.StringUtils;
|
|
@@ -38,9 +38,9 @@ import org.apache.hadoop.util.StringUtils;
|
|
* A base class for file-based {@link InputFormat}.
|
|
* A base class for file-based {@link InputFormat}.
|
|
*
|
|
*
|
|
* <p><code>FileInputFormat</code> is the base class for all file-based
|
|
* <p><code>FileInputFormat</code> is the base class for all file-based
|
|
- * <code>InputFormat</code>s. This provides generic implementations of
|
|
|
|
- * {@link #validateInput(JobConf)} and {@link #getSplits(JobConf, int)}.
|
|
|
|
- * Implementations fo <code>FileInputFormat</code> can also override the
|
|
|
|
|
|
+ * <code>InputFormat</code>s. This provides a generic implementation of
|
|
|
|
+ * {@link #getSplits(JobConf, int)}.
|
|
|
|
+ * Subclasses of <code>FileInputFormat</code> can also override the
|
|
* {@link #isSplitable(FileSystem, Path)} method to ensure input-files are
|
|
* {@link #isSplitable(FileSystem, Path)} method to ensure input-files are
|
|
* not split-up and are processed as a whole by {@link Mapper}s.
|
|
* not split-up and are processed as a whole by {@link Mapper}s.
|
|
*/
|
|
*/
|
|
@@ -121,7 +121,7 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
|
|
* @return the PathFilter instance set for the job, NULL if none has been set.
|
|
* @return the PathFilter instance set for the job, NULL if none has been set.
|
|
*/
|
|
*/
|
|
public static PathFilter getInputPathFilter(JobConf conf) {
|
|
public static PathFilter getInputPathFilter(JobConf conf) {
|
|
- Class filterClass = conf.getClass("mapred.input.pathFilter.class", null,
|
|
|
|
|
|
+ Class<?> filterClass = conf.getClass("mapred.input.pathFilter.class", null,
|
|
PathFilter.class);
|
|
PathFilter.class);
|
|
return (filterClass != null) ?
|
|
return (filterClass != null) ?
|
|
(PathFilter) ReflectionUtils.newInstance(filterClass, conf) : null;
|
|
(PathFilter) ReflectionUtils.newInstance(filterClass, conf) : null;
|
|
@@ -132,18 +132,18 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
|
|
* expression.
|
|
* expression.
|
|
*
|
|
*
|
|
* @param job the job to list input paths for
|
|
* @param job the job to list input paths for
|
|
- * @return array of Path objects
|
|
|
|
|
|
+ * @return array of FileStatus objects
|
|
* @throws IOException if zero items.
|
|
* @throws IOException if zero items.
|
|
*/
|
|
*/
|
|
- protected Path[] listPaths(JobConf job)
|
|
|
|
- throws IOException {
|
|
|
|
|
|
+ protected FileStatus[] listStatus(JobConf job) throws IOException {
|
|
Path[] dirs = getInputPaths(job);
|
|
Path[] dirs = getInputPaths(job);
|
|
if (dirs.length == 0) {
|
|
if (dirs.length == 0) {
|
|
throw new IOException("No input paths specified in job");
|
|
throw new IOException("No input paths specified in job");
|
|
}
|
|
}
|
|
|
|
|
|
- List<Path> result = new ArrayList<Path>();
|
|
|
|
-
|
|
|
|
|
|
+ List<FileStatus> result = new ArrayList<FileStatus>();
|
|
|
|
+ List<IOException> errors = new ArrayList<IOException>();
|
|
|
|
+
|
|
// creates a MultiPathFilter with the hiddenFileFilter and the
|
|
// creates a MultiPathFilter with the hiddenFileFilter and the
|
|
// user provided one (if any).
|
|
// user provided one (if any).
|
|
List<PathFilter> filters = new ArrayList<PathFilter>();
|
|
List<PathFilter> filters = new ArrayList<PathFilter>();
|
|
@@ -156,73 +156,119 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
|
|
|
|
|
|
for (Path p: dirs) {
|
|
for (Path p: dirs) {
|
|
FileSystem fs = p.getFileSystem(job);
|
|
FileSystem fs = p.getFileSystem(job);
|
|
- FileStatus[] matches = fs.listStatus(FileUtil.stat2Paths(fs.globStatus(p,
|
|
|
|
- inputFilter)), inputFilter);
|
|
|
|
-
|
|
|
|
- for (FileStatus match: matches) {
|
|
|
|
- result.add(fs.makeQualified(match.getPath()));
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- return result.toArray(new Path[result.size()]);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- public void validateInput(JobConf job) throws IOException {
|
|
|
|
- Path[] inputDirs = getInputPaths(job);
|
|
|
|
- if (inputDirs.length == 0) {
|
|
|
|
- throw new IOException("No input paths specified in input");
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- List<IOException> result = new ArrayList<IOException>();
|
|
|
|
- int totalFiles = 0;
|
|
|
|
- for (Path p: inputDirs) {
|
|
|
|
- FileSystem fs = p.getFileSystem(job);
|
|
|
|
- if (fs.exists(p)) {
|
|
|
|
- // make sure all paths are files to avoid exception
|
|
|
|
- // while generating splits
|
|
|
|
- Path[] subPaths = FileUtil.stat2Paths(fs.listStatus(p,
|
|
|
|
- hiddenFileFilter));
|
|
|
|
- for (Path subPath : subPaths) {
|
|
|
|
- FileSystem subFS = subPath.getFileSystem(job);
|
|
|
|
- if (!subFS.exists(subPath)) {
|
|
|
|
- result.add(new IOException(
|
|
|
|
- "Input path does not exist: " + subPath));
|
|
|
|
- } else {
|
|
|
|
- totalFiles++;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
+ FileStatus[] matches = fs.globStatus(p, inputFilter);
|
|
|
|
+ if (matches == null) {
|
|
|
|
+ errors.add(new IOException("Input path does not exist: " + p));
|
|
|
|
+ } else if (matches.length == 0) {
|
|
|
|
+ errors.add(new IOException("Input Pattern " + p + " matches 0 files"));
|
|
} else {
|
|
} else {
|
|
- Path [] paths = FileUtil.stat2Paths(fs.globStatus(p,
|
|
|
|
- hiddenFileFilter), p);
|
|
|
|
- if (paths.length == 0) {
|
|
|
|
- result.add(
|
|
|
|
- new IOException("Input Pattern " + p + " matches 0 files"));
|
|
|
|
- } else {
|
|
|
|
- // validate globbed paths
|
|
|
|
- for (Path gPath : paths) {
|
|
|
|
- FileSystem gPathFS = gPath.getFileSystem(job);
|
|
|
|
- if (!gPathFS.exists(gPath)) {
|
|
|
|
- result.add(
|
|
|
|
- new FileNotFoundException(
|
|
|
|
- "Input path doesnt exist : " + gPath));
|
|
|
|
- }
|
|
|
|
|
|
+ for (FileStatus globStat: matches) {
|
|
|
|
+ if (globStat.isDir()) {
|
|
|
|
+ for(FileStatus stat: fs.listStatus(globStat.getPath(),
|
|
|
|
+ inputFilter)) {
|
|
|
|
+ result.add(stat);
|
|
|
|
+ }
|
|
|
|
+ } else {
|
|
|
|
+ result.add(globStat);
|
|
}
|
|
}
|
|
- totalFiles += paths.length;
|
|
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- if (!result.isEmpty()) {
|
|
|
|
- throw new InvalidInputException(result);
|
|
|
|
|
|
+
|
|
|
|
+ if (!errors.isEmpty()) {
|
|
|
|
+ throw new InvalidInputException(errors);
|
|
}
|
|
}
|
|
- // send output to client.
|
|
|
|
- LOG.info("Total input paths to process : " + totalFiles);
|
|
|
|
|
|
+ LOG.info("Total input paths to process : " + result.size());
|
|
|
|
+ return result.toArray(new FileStatus[result.size()]);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /** List input directories.
|
|
|
|
+ * Subclasses may override to, e.g., select only files matching a regular
|
|
|
|
+ * expression.
|
|
|
|
+ *
|
|
|
|
+ * @param job the job to list input paths for
|
|
|
|
+ * @return array of Path objects
|
|
|
|
+ * @throws IOException if zero items.
|
|
|
|
+ * @deprecated Use {@link #listStatus(JobConf)} instead.
|
|
|
|
+ */
|
|
|
|
+ @Deprecated
|
|
|
|
+ protected Path[] listPaths(JobConf job)
|
|
|
|
+ throws IOException {
|
|
|
|
+ return FileUtil.stat2Paths(listStatus(job));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Deprecated
|
|
|
|
+ public void validateInput(JobConf job) throws IOException {
|
|
|
|
+ // handled by getSplits
|
|
}
|
|
}
|
|
|
|
|
|
- /** Splits files returned by {@link #listPaths(JobConf)} when
|
|
|
|
|
|
+ /** Splits files returned by {@link #listStatus(JobConf)} when
|
|
* they're too big.*/
|
|
* they're too big.*/
|
|
|
|
+ @SuppressWarnings("deprecation")
|
|
public InputSplit[] getSplits(JobConf job, int numSplits)
|
|
public InputSplit[] getSplits(JobConf job, int numSplits)
|
|
throws IOException {
|
|
throws IOException {
|
|
- Path[] files = listPaths(job);
|
|
|
|
|
|
+ FileStatus[] files = listStatus(job);
|
|
|
|
+
|
|
|
|
+ // Applications may have overridden listPaths so we need to check if
|
|
|
|
+ // it returns a different set of paths to listStatus.
|
|
|
|
+ // If it does we revert to the old behavior using Paths not FileStatus
|
|
|
|
+ // objects.
|
|
|
|
+ // When listPaths is removed, this check can be removed too.
|
|
|
|
+ Path[] paths = listPaths(job);
|
|
|
|
+ if (!Arrays.equals(paths, FileUtil.stat2Paths(files))) {
|
|
|
|
+ LOG.warn("FileInputFormat#listPaths is deprecated, override listStatus " +
|
|
|
|
+ "instead.");
|
|
|
|
+ return getSplitsForPaths(job, numSplits, paths);
|
|
|
|
+ }
|
|
|
|
+ long totalSize = 0; // compute total size
|
|
|
|
+ for (FileStatus file: files) { // check we have valid files
|
|
|
|
+ if (file.isDir()) {
|
|
|
|
+ throw new IOException("Not a file: "+ file.getPath());
|
|
|
|
+ }
|
|
|
|
+ totalSize += file.getLen();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ long goalSize = totalSize / (numSplits == 0 ? 1 : numSplits);
|
|
|
|
+ long minSize = Math.max(job.getLong("mapred.min.split.size", 1),
|
|
|
|
+ minSplitSize);
|
|
|
|
+
|
|
|
|
+ // generate splits
|
|
|
|
+ ArrayList<FileSplit> splits = new ArrayList<FileSplit>(numSplits);
|
|
|
|
+ for (FileStatus file: files) {
|
|
|
|
+ Path path = file.getPath();
|
|
|
|
+ FileSystem fs = path.getFileSystem(job);
|
|
|
|
+ long length = file.getLen();
|
|
|
|
+ BlockLocation[] blkLocations = fs.getFileBlockLocations(file, 0, length);
|
|
|
|
+ if ((length != 0) && isSplitable(fs, path)) {
|
|
|
|
+ long blockSize = file.getBlockSize();
|
|
|
|
+ long splitSize = computeSplitSize(goalSize, minSize, blockSize);
|
|
|
|
+
|
|
|
|
+ long bytesRemaining = length;
|
|
|
|
+ while (((double) bytesRemaining)/splitSize > SPLIT_SLOP) {
|
|
|
|
+ int blkIndex = getBlockIndex(blkLocations, length-bytesRemaining);
|
|
|
|
+ splits.add(new FileSplit(path, length-bytesRemaining, splitSize,
|
|
|
|
+ blkLocations[blkIndex].getHosts()));
|
|
|
|
+ bytesRemaining -= splitSize;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if (bytesRemaining != 0) {
|
|
|
|
+ splits.add(new FileSplit(path, length-bytesRemaining, bytesRemaining,
|
|
|
|
+ blkLocations[blkLocations.length-1].getHosts()));
|
|
|
|
+ }
|
|
|
|
+ } else if (length != 0) {
|
|
|
|
+ splits.add(new FileSplit(path, 0, length, blkLocations[0].getHosts()));
|
|
|
|
+ } else {
|
|
|
|
+ //Create empty hosts array for zero length files
|
|
|
|
+ splits.add(new FileSplit(path, 0, length, new String[0]));
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ LOG.debug("Total # of splits: " + splits.size());
|
|
|
|
+ return splits.toArray(new FileSplit[splits.size()]);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Deprecated
|
|
|
|
+ private InputSplit[] getSplitsForPaths(JobConf job, int numSplits,
|
|
|
|
+ Path[] files) throws IOException {
|
|
long totalSize = 0; // compute total size
|
|
long totalSize = 0; // compute total size
|
|
for (int i = 0; i < files.length; i++) { // check we have valid files
|
|
for (int i = 0; i < files.length; i++) { // check we have valid files
|
|
Path file = files[i];
|
|
Path file = files[i];
|