MAPREDUCE-1981. Improve getSplits performance by using listLocatedStatus. Contributed by Hairong Kuang and Jason Lowe
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1507385 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
55dfadec80
commit
ec18984252
|
@ -156,6 +156,9 @@ Release 2.3.0 - UNRELEASED
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
|
MAPREDUCE-1981. Improve getSplits performance by using listLocatedStatus
|
||||||
|
(Hairong Kuang and Jason Lowe via jlowe)
|
||||||
|
|
||||||
BUG FIXES
|
BUG FIXES
|
||||||
|
|
||||||
MAPREDUCE-5316. job -list-attempt-ids command does not handle illegal
|
MAPREDUCE-5316. job -list-attempt-ids command does not handle illegal
|
||||||
|
@ -1232,6 +1235,9 @@ Release 0.23.10 - UNRELEASED
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
|
MAPREDUCE-1981. Improve getSplits performance by using listLocatedStatus
|
||||||
|
(Hairong Kuang and Jason Lowe via jlowe)
|
||||||
|
|
||||||
BUG FIXES
|
BUG FIXES
|
||||||
|
|
||||||
MAPREDUCE-3193. FileInputFormat doesn't read files recursively in the
|
MAPREDUCE-3193. FileInputFormat doesn't read files recursively in the
|
||||||
|
|
|
@ -36,8 +36,10 @@ import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.fs.BlockLocation;
|
import org.apache.hadoop.fs.BlockLocation;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||||
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.RemoteIterator;
|
||||||
import org.apache.hadoop.mapreduce.security.TokenCache;
|
import org.apache.hadoop.mapreduce.security.TokenCache;
|
||||||
import org.apache.hadoop.net.NetworkTopology;
|
import org.apache.hadoop.net.NetworkTopology;
|
||||||
import org.apache.hadoop.net.Node;
|
import org.apache.hadoop.net.Node;
|
||||||
|
@ -169,11 +171,15 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
|
||||||
protected void addInputPathRecursively(List<FileStatus> result,
|
protected void addInputPathRecursively(List<FileStatus> result,
|
||||||
FileSystem fs, Path path, PathFilter inputFilter)
|
FileSystem fs, Path path, PathFilter inputFilter)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
for(FileStatus stat: fs.listStatus(path, inputFilter)) {
|
RemoteIterator<LocatedFileStatus> iter = fs.listLocatedStatus(path);
|
||||||
if (stat.isDirectory()) {
|
while (iter.hasNext()) {
|
||||||
addInputPathRecursively(result, fs, stat.getPath(), inputFilter);
|
LocatedFileStatus stat = iter.next();
|
||||||
} else {
|
if (inputFilter.accept(stat.getPath())) {
|
||||||
result.add(stat);
|
if (stat.isDirectory()) {
|
||||||
|
addInputPathRecursively(result, fs, stat.getPath(), inputFilter);
|
||||||
|
} else {
|
||||||
|
result.add(stat);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -221,12 +227,17 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
|
||||||
} else {
|
} else {
|
||||||
for (FileStatus globStat: matches) {
|
for (FileStatus globStat: matches) {
|
||||||
if (globStat.isDirectory()) {
|
if (globStat.isDirectory()) {
|
||||||
for(FileStatus stat: fs.listStatus(globStat.getPath(),
|
RemoteIterator<LocatedFileStatus> iter =
|
||||||
inputFilter)) {
|
fs.listLocatedStatus(globStat.getPath());
|
||||||
if (recursive && stat.isDirectory()) {
|
while (iter.hasNext()) {
|
||||||
addInputPathRecursively(result, fs, stat.getPath(), inputFilter);
|
LocatedFileStatus stat = iter.next();
|
||||||
} else {
|
if (inputFilter.accept(stat.getPath())) {
|
||||||
result.add(stat);
|
if (recursive && stat.isDirectory()) {
|
||||||
|
addInputPathRecursively(result, fs, stat.getPath(),
|
||||||
|
inputFilter);
|
||||||
|
} else {
|
||||||
|
result.add(stat);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
|
@ -254,7 +265,6 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
|
||||||
|
|
||||||
/** Splits files returned by {@link #listStatus(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 {
|
||||||
FileStatus[] files = listStatus(job);
|
FileStatus[] files = listStatus(job);
|
||||||
|
@ -278,31 +288,38 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
|
||||||
NetworkTopology clusterMap = new NetworkTopology();
|
NetworkTopology clusterMap = new NetworkTopology();
|
||||||
for (FileStatus file: files) {
|
for (FileStatus file: files) {
|
||||||
Path path = file.getPath();
|
Path path = file.getPath();
|
||||||
FileSystem fs = path.getFileSystem(job);
|
|
||||||
long length = file.getLen();
|
long length = file.getLen();
|
||||||
BlockLocation[] blkLocations = fs.getFileBlockLocations(file, 0, length);
|
if (length != 0) {
|
||||||
if ((length != 0) && isSplitable(fs, path)) {
|
FileSystem fs = path.getFileSystem(job);
|
||||||
long blockSize = file.getBlockSize();
|
BlockLocation[] blkLocations;
|
||||||
long splitSize = computeSplitSize(goalSize, minSize, blockSize);
|
if (file instanceof LocatedFileStatus) {
|
||||||
|
blkLocations = ((LocatedFileStatus) file).getBlockLocations();
|
||||||
long bytesRemaining = length;
|
} else {
|
||||||
while (((double) bytesRemaining)/splitSize > SPLIT_SLOP) {
|
blkLocations = fs.getFileBlockLocations(file, 0, length);
|
||||||
String[] splitHosts = getSplitHosts(blkLocations,
|
|
||||||
length-bytesRemaining, splitSize, clusterMap);
|
|
||||||
splits.add(makeSplit(path, length-bytesRemaining, splitSize,
|
|
||||||
splitHosts));
|
|
||||||
bytesRemaining -= splitSize;
|
|
||||||
}
|
}
|
||||||
|
if (isSplitable(fs, path)) {
|
||||||
|
long blockSize = file.getBlockSize();
|
||||||
|
long splitSize = computeSplitSize(goalSize, minSize, blockSize);
|
||||||
|
|
||||||
if (bytesRemaining != 0) {
|
long bytesRemaining = length;
|
||||||
String[] splitHosts = getSplitHosts(blkLocations, length
|
while (((double) bytesRemaining)/splitSize > SPLIT_SLOP) {
|
||||||
- bytesRemaining, bytesRemaining, clusterMap);
|
String[] splitHosts = getSplitHosts(blkLocations,
|
||||||
splits.add(makeSplit(path, length - bytesRemaining, bytesRemaining,
|
length-bytesRemaining, splitSize, clusterMap);
|
||||||
splitHosts));
|
splits.add(makeSplit(path, length-bytesRemaining, splitSize,
|
||||||
|
splitHosts));
|
||||||
|
bytesRemaining -= splitSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (bytesRemaining != 0) {
|
||||||
|
String[] splitHosts = getSplitHosts(blkLocations, length
|
||||||
|
- bytesRemaining, bytesRemaining, clusterMap);
|
||||||
|
splits.add(makeSplit(path, length - bytesRemaining, bytesRemaining,
|
||||||
|
splitHosts));
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
String[] splitHosts = getSplitHosts(blkLocations,0,length,clusterMap);
|
||||||
|
splits.add(makeSplit(path, 0, length, splitHosts));
|
||||||
}
|
}
|
||||||
} else if (length != 0) {
|
|
||||||
String[] splitHosts = getSplitHosts(blkLocations,0,length,clusterMap);
|
|
||||||
splits.add(makeSplit(path, 0, length, splitHosts));
|
|
||||||
} else {
|
} else {
|
||||||
//Create empty hosts array for zero length files
|
//Create empty hosts array for zero length files
|
||||||
splits.add(makeSplit(path, 0, length, new String[0]));
|
splits.add(makeSplit(path, 0, length, new String[0]));
|
||||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.hadoop.mapreduce.lib.input;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.LinkedList;
|
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
@ -33,7 +32,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.FileUtil;
|
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.BlockLocation;
|
import org.apache.hadoop.fs.BlockLocation;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
@ -204,46 +203,33 @@ public abstract class CombineFileInputFormat<K, V>
|
||||||
}
|
}
|
||||||
|
|
||||||
// all the files in input set
|
// all the files in input set
|
||||||
Path[] paths = FileUtil.stat2Paths(
|
List<FileStatus> stats = listStatus(job);
|
||||||
listStatus(job).toArray(new FileStatus[0]));
|
|
||||||
List<InputSplit> splits = new ArrayList<InputSplit>();
|
List<InputSplit> splits = new ArrayList<InputSplit>();
|
||||||
if (paths.length == 0) {
|
if (stats.size() == 0) {
|
||||||
return splits;
|
return splits;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Convert them to Paths first. This is a costly operation and
|
|
||||||
// we should do it first, otherwise we will incur doing it multiple
|
|
||||||
// times, one time each for each pool in the next loop.
|
|
||||||
List<Path> newpaths = new LinkedList<Path>();
|
|
||||||
for (int i = 0; i < paths.length; i++) {
|
|
||||||
FileSystem fs = paths[i].getFileSystem(conf);
|
|
||||||
Path p = fs.makeQualified(paths[i]);
|
|
||||||
newpaths.add(p);
|
|
||||||
}
|
|
||||||
|
|
||||||
// In one single iteration, process all the paths in a single pool.
|
// In one single iteration, process all the paths in a single pool.
|
||||||
// Processing one pool at a time ensures that a split contains paths
|
// Processing one pool at a time ensures that a split contains paths
|
||||||
// from a single pool only.
|
// from a single pool only.
|
||||||
for (MultiPathFilter onepool : pools) {
|
for (MultiPathFilter onepool : pools) {
|
||||||
ArrayList<Path> myPaths = new ArrayList<Path>();
|
ArrayList<FileStatus> myPaths = new ArrayList<FileStatus>();
|
||||||
|
|
||||||
// pick one input path. If it matches all the filters in a pool,
|
// pick one input path. If it matches all the filters in a pool,
|
||||||
// add it to the output set
|
// add it to the output set
|
||||||
for (Iterator<Path> iter = newpaths.iterator(); iter.hasNext();) {
|
for (Iterator<FileStatus> iter = stats.iterator(); iter.hasNext();) {
|
||||||
Path p = iter.next();
|
FileStatus p = iter.next();
|
||||||
if (onepool.accept(p)) {
|
if (onepool.accept(p.getPath())) {
|
||||||
myPaths.add(p); // add it to my output set
|
myPaths.add(p); // add it to my output set
|
||||||
iter.remove();
|
iter.remove();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// create splits for all files in this pool.
|
// create splits for all files in this pool.
|
||||||
getMoreSplits(job, myPaths.toArray(new Path[myPaths.size()]),
|
getMoreSplits(job, myPaths, maxSize, minSizeNode, minSizeRack, splits);
|
||||||
maxSize, minSizeNode, minSizeRack, splits);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// create splits for all files that are not in any pool.
|
// create splits for all files that are not in any pool.
|
||||||
getMoreSplits(job, newpaths.toArray(new Path[newpaths.size()]),
|
getMoreSplits(job, stats, maxSize, minSizeNode, minSizeRack, splits);
|
||||||
maxSize, minSizeNode, minSizeRack, splits);
|
|
||||||
|
|
||||||
// free up rackToNodes map
|
// free up rackToNodes map
|
||||||
rackToNodes.clear();
|
rackToNodes.clear();
|
||||||
|
@ -253,7 +239,7 @@ public abstract class CombineFileInputFormat<K, V>
|
||||||
/**
|
/**
|
||||||
* Return all the splits in the specified set of paths
|
* Return all the splits in the specified set of paths
|
||||||
*/
|
*/
|
||||||
private void getMoreSplits(JobContext job, Path[] paths,
|
private void getMoreSplits(JobContext job, List<FileStatus> stats,
|
||||||
long maxSize, long minSizeNode, long minSizeRack,
|
long maxSize, long minSizeNode, long minSizeRack,
|
||||||
List<InputSplit> splits)
|
List<InputSplit> splits)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
@ -274,15 +260,16 @@ public abstract class CombineFileInputFormat<K, V>
|
||||||
HashMap<String, List<OneBlockInfo>> nodeToBlocks =
|
HashMap<String, List<OneBlockInfo>> nodeToBlocks =
|
||||||
new HashMap<String, List<OneBlockInfo>>();
|
new HashMap<String, List<OneBlockInfo>>();
|
||||||
|
|
||||||
files = new OneFileInfo[paths.length];
|
files = new OneFileInfo[stats.size()];
|
||||||
if (paths.length == 0) {
|
if (stats.size() == 0) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
// populate all the blocks for all files
|
// populate all the blocks for all files
|
||||||
long totLength = 0;
|
long totLength = 0;
|
||||||
for (int i = 0; i < paths.length; i++) {
|
int i = 0;
|
||||||
files[i] = new OneFileInfo(paths[i], conf, isSplitable(job, paths[i]),
|
for (FileStatus stat : stats) {
|
||||||
|
files[i] = new OneFileInfo(stat, conf, isSplitable(job, stat.getPath()),
|
||||||
rackToBlocks, blockToNodes, nodeToBlocks,
|
rackToBlocks, blockToNodes, nodeToBlocks,
|
||||||
rackToNodes, maxSize);
|
rackToNodes, maxSize);
|
||||||
totLength += files[i].getLength();
|
totLength += files[i].getLength();
|
||||||
|
@ -523,7 +510,7 @@ public abstract class CombineFileInputFormat<K, V>
|
||||||
private long fileSize; // size of the file
|
private long fileSize; // size of the file
|
||||||
private OneBlockInfo[] blocks; // all blocks in this file
|
private OneBlockInfo[] blocks; // all blocks in this file
|
||||||
|
|
||||||
OneFileInfo(Path path, Configuration conf,
|
OneFileInfo(FileStatus stat, Configuration conf,
|
||||||
boolean isSplitable,
|
boolean isSplitable,
|
||||||
HashMap<String, List<OneBlockInfo>> rackToBlocks,
|
HashMap<String, List<OneBlockInfo>> rackToBlocks,
|
||||||
HashMap<OneBlockInfo, String[]> blockToNodes,
|
HashMap<OneBlockInfo, String[]> blockToNodes,
|
||||||
|
@ -534,10 +521,13 @@ public abstract class CombineFileInputFormat<K, V>
|
||||||
this.fileSize = 0;
|
this.fileSize = 0;
|
||||||
|
|
||||||
// get block locations from file system
|
// get block locations from file system
|
||||||
FileSystem fs = path.getFileSystem(conf);
|
BlockLocation[] locations;
|
||||||
FileStatus stat = fs.getFileStatus(path);
|
if (stat instanceof LocatedFileStatus) {
|
||||||
BlockLocation[] locations = fs.getFileBlockLocations(stat, 0,
|
locations = ((LocatedFileStatus) stat).getBlockLocations();
|
||||||
stat.getLen());
|
} else {
|
||||||
|
FileSystem fs = stat.getPath().getFileSystem(conf);
|
||||||
|
locations = fs.getFileBlockLocations(stat, 0, stat.getLen());
|
||||||
|
}
|
||||||
// create a list of all block and their locations
|
// create a list of all block and their locations
|
||||||
if (locations == null) {
|
if (locations == null) {
|
||||||
blocks = new OneBlockInfo[0];
|
blocks = new OneBlockInfo[0];
|
||||||
|
@ -552,8 +542,8 @@ public abstract class CombineFileInputFormat<K, V>
|
||||||
// full file length
|
// full file length
|
||||||
blocks = new OneBlockInfo[1];
|
blocks = new OneBlockInfo[1];
|
||||||
fileSize = stat.getLen();
|
fileSize = stat.getLen();
|
||||||
blocks[0] = new OneBlockInfo(path, 0, fileSize, locations[0]
|
blocks[0] = new OneBlockInfo(stat.getPath(), 0, fileSize,
|
||||||
.getHosts(), locations[0].getTopologyPaths());
|
locations[0].getHosts(), locations[0].getTopologyPaths());
|
||||||
} else {
|
} else {
|
||||||
ArrayList<OneBlockInfo> blocksList = new ArrayList<OneBlockInfo>(
|
ArrayList<OneBlockInfo> blocksList = new ArrayList<OneBlockInfo>(
|
||||||
locations.length);
|
locations.length);
|
||||||
|
@ -579,9 +569,9 @@ public abstract class CombineFileInputFormat<K, V>
|
||||||
myLength = Math.min(maxSize, left);
|
myLength = Math.min(maxSize, left);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
OneBlockInfo oneblock = new OneBlockInfo(path, myOffset,
|
OneBlockInfo oneblock = new OneBlockInfo(stat.getPath(),
|
||||||
myLength, locations[i].getHosts(), locations[i]
|
myOffset, myLength, locations[i].getHosts(),
|
||||||
.getTopologyPaths());
|
locations[i].getTopologyPaths());
|
||||||
left -= myLength;
|
left -= myLength;
|
||||||
myOffset += myLength;
|
myOffset += myLength;
|
||||||
|
|
||||||
|
@ -693,6 +683,9 @@ public abstract class CombineFileInputFormat<K, V>
|
||||||
|
|
||||||
protected BlockLocation[] getFileBlockLocations(
|
protected BlockLocation[] getFileBlockLocations(
|
||||||
FileSystem fs, FileStatus stat) throws IOException {
|
FileSystem fs, FileStatus stat) throws IOException {
|
||||||
|
if (stat instanceof LocatedFileStatus) {
|
||||||
|
return ((LocatedFileStatus) stat).getBlockLocations();
|
||||||
|
}
|
||||||
return fs.getFileBlockLocations(stat, 0, stat.getLen());
|
return fs.getFileBlockLocations(stat, 0, stat.getLen());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -29,9 +29,11 @@ import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||||
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.BlockLocation;
|
import org.apache.hadoop.fs.BlockLocation;
|
||||||
|
import org.apache.hadoop.fs.RemoteIterator;
|
||||||
import org.apache.hadoop.mapreduce.InputFormat;
|
import org.apache.hadoop.mapreduce.InputFormat;
|
||||||
import org.apache.hadoop.mapreduce.InputSplit;
|
import org.apache.hadoop.mapreduce.InputSplit;
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
|
@ -259,12 +261,17 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
|
||||||
} else {
|
} else {
|
||||||
for (FileStatus globStat: matches) {
|
for (FileStatus globStat: matches) {
|
||||||
if (globStat.isDirectory()) {
|
if (globStat.isDirectory()) {
|
||||||
for(FileStatus stat: fs.listStatus(globStat.getPath(),
|
RemoteIterator<LocatedFileStatus> iter =
|
||||||
inputFilter)) {
|
fs.listLocatedStatus(globStat.getPath());
|
||||||
if (recursive && stat.isDirectory()) {
|
while (iter.hasNext()) {
|
||||||
addInputPathRecursively(result, fs, stat.getPath(), inputFilter);
|
LocatedFileStatus stat = iter.next();
|
||||||
} else {
|
if (inputFilter.accept(stat.getPath())) {
|
||||||
result.add(stat);
|
if (recursive && stat.isDirectory()) {
|
||||||
|
addInputPathRecursively(result, fs, stat.getPath(),
|
||||||
|
inputFilter);
|
||||||
|
} else {
|
||||||
|
result.add(stat);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
|
@ -296,11 +303,15 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
|
||||||
protected void addInputPathRecursively(List<FileStatus> result,
|
protected void addInputPathRecursively(List<FileStatus> result,
|
||||||
FileSystem fs, Path path, PathFilter inputFilter)
|
FileSystem fs, Path path, PathFilter inputFilter)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
for(FileStatus stat: fs.listStatus(path, inputFilter)) {
|
RemoteIterator<LocatedFileStatus> iter = fs.listLocatedStatus(path);
|
||||||
if (stat.isDirectory()) {
|
while (iter.hasNext()) {
|
||||||
addInputPathRecursively(result, fs, stat.getPath(), inputFilter);
|
LocatedFileStatus stat = iter.next();
|
||||||
} else {
|
if (inputFilter.accept(stat.getPath())) {
|
||||||
result.add(stat);
|
if (stat.isDirectory()) {
|
||||||
|
addInputPathRecursively(result, fs, stat.getPath(), inputFilter);
|
||||||
|
} else {
|
||||||
|
result.add(stat);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -331,8 +342,13 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
|
||||||
Path path = file.getPath();
|
Path path = file.getPath();
|
||||||
long length = file.getLen();
|
long length = file.getLen();
|
||||||
if (length != 0) {
|
if (length != 0) {
|
||||||
FileSystem fs = path.getFileSystem(job.getConfiguration());
|
BlockLocation[] blkLocations;
|
||||||
BlockLocation[] blkLocations = fs.getFileBlockLocations(file, 0, length);
|
if (file instanceof LocatedFileStatus) {
|
||||||
|
blkLocations = ((LocatedFileStatus) file).getBlockLocations();
|
||||||
|
} else {
|
||||||
|
FileSystem fs = path.getFileSystem(job.getConfiguration());
|
||||||
|
blkLocations = fs.getFileBlockLocations(file, 0, length);
|
||||||
|
}
|
||||||
if (isSplitable(job, path)) {
|
if (isSplitable(job, path)) {
|
||||||
long blockSize = file.getBlockSize();
|
long blockSize = file.getBlockSize();
|
||||||
long splitSize = computeSplitSize(blockSize, minSize, maxSize);
|
long splitSize = computeSplitSize(blockSize, minSize, maxSize);
|
||||||
|
|
|
@ -0,0 +1,113 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.mapred;
|
||||||
|
|
||||||
|
import java.io.FileNotFoundException;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
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.LocatedFileStatus;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.fs.PathFilter;
|
||||||
|
import org.apache.hadoop.fs.RawLocalFileSystem;
|
||||||
|
import org.apache.hadoop.fs.RemoteIterator;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
public class TestFileInputFormat {
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testListLocatedStatus() throws Exception {
|
||||||
|
Configuration conf = getConfiguration();
|
||||||
|
conf.setBoolean("fs.test.impl.disable.cache", false);
|
||||||
|
conf.set(org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR,
|
||||||
|
"test:///a1/a2");
|
||||||
|
MockFileSystem mockFs =
|
||||||
|
(MockFileSystem) new Path("test:///").getFileSystem(conf);
|
||||||
|
Assert.assertEquals("listLocatedStatus already called",
|
||||||
|
0, mockFs.numListLocatedStatusCalls);
|
||||||
|
JobConf job = new JobConf(conf);
|
||||||
|
TextInputFormat fileInputFormat = new TextInputFormat();
|
||||||
|
fileInputFormat.configure(job);
|
||||||
|
InputSplit[] splits = fileInputFormat.getSplits(job, 1);
|
||||||
|
Assert.assertEquals("Input splits are not correct", 2, splits.length);
|
||||||
|
Assert.assertEquals("listLocatedStatuss calls",
|
||||||
|
1, mockFs.numListLocatedStatusCalls);
|
||||||
|
}
|
||||||
|
|
||||||
|
private Configuration getConfiguration() {
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
conf.set("fs.test.impl.disable.cache", "true");
|
||||||
|
conf.setClass("fs.test.impl", MockFileSystem.class, FileSystem.class);
|
||||||
|
conf.set(org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR,
|
||||||
|
"test:///a1");
|
||||||
|
return conf;
|
||||||
|
}
|
||||||
|
|
||||||
|
static class MockFileSystem extends RawLocalFileSystem {
|
||||||
|
int numListLocatedStatusCalls = 0;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FileStatus[] listStatus(Path f) throws FileNotFoundException,
|
||||||
|
IOException {
|
||||||
|
if (f.toString().equals("test:/a1")) {
|
||||||
|
return new FileStatus[] {
|
||||||
|
new FileStatus(0, true, 1, 150, 150, new Path("test:/a1/a2")),
|
||||||
|
new FileStatus(10, false, 1, 150, 150, new Path("test:/a1/file1")) };
|
||||||
|
} else if (f.toString().equals("test:/a1/a2")) {
|
||||||
|
return new FileStatus[] {
|
||||||
|
new FileStatus(10, false, 1, 150, 150,
|
||||||
|
new Path("test:/a1/a2/file2")),
|
||||||
|
new FileStatus(10, false, 1, 151, 150,
|
||||||
|
new Path("test:/a1/a2/file3")) };
|
||||||
|
}
|
||||||
|
return new FileStatus[0];
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FileStatus[] globStatus(Path pathPattern, PathFilter filter)
|
||||||
|
throws IOException {
|
||||||
|
return new FileStatus[] { new FileStatus(10, true, 1, 150, 150,
|
||||||
|
pathPattern) };
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FileStatus[] listStatus(Path f, PathFilter filter)
|
||||||
|
throws FileNotFoundException, IOException {
|
||||||
|
return this.listStatus(f);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public BlockLocation[] getFileBlockLocations(Path p, long start, long len)
|
||||||
|
throws IOException {
|
||||||
|
return new BlockLocation[] {
|
||||||
|
new BlockLocation(new String[] { "localhost:50010" },
|
||||||
|
new String[] { "localhost" }, 0, len) };
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f,
|
||||||
|
PathFilter filter) throws FileNotFoundException, IOException {
|
||||||
|
++numListLocatedStatusCalls;
|
||||||
|
return super.listLocatedStatus(f, filter);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -24,11 +24,14 @@ import java.util.List;
|
||||||
import junit.framework.Assert;
|
import junit.framework.Assert;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.BlockLocation;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||||
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.RawLocalFileSystem;
|
import org.apache.hadoop.fs.RawLocalFileSystem;
|
||||||
|
import org.apache.hadoop.fs.RemoteIterator;
|
||||||
import org.apache.hadoop.mapreduce.InputSplit;
|
import org.apache.hadoop.mapreduce.InputSplit;
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -77,6 +80,23 @@ public class TestFileInputFormat {
|
||||||
.toString());
|
.toString());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testListLocatedStatus() throws Exception {
|
||||||
|
Configuration conf = getConfiguration();
|
||||||
|
conf.setBoolean("fs.test.impl.disable.cache", false);
|
||||||
|
conf.set(FileInputFormat.INPUT_DIR, "test:///a1/a2");
|
||||||
|
MockFileSystem mockFs =
|
||||||
|
(MockFileSystem) new Path("test:///").getFileSystem(conf);
|
||||||
|
Assert.assertEquals("listLocatedStatus already called",
|
||||||
|
0, mockFs.numListLocatedStatusCalls);
|
||||||
|
Job job = Job.getInstance(conf);
|
||||||
|
FileInputFormat<?, ?> fileInputFormat = new TextInputFormat();
|
||||||
|
List<InputSplit> splits = fileInputFormat.getSplits(job);
|
||||||
|
Assert.assertEquals("Input splits are not correct", 2, splits.size());
|
||||||
|
Assert.assertEquals("listLocatedStatuss calls",
|
||||||
|
1, mockFs.numListLocatedStatusCalls);
|
||||||
|
}
|
||||||
|
|
||||||
private Configuration getConfiguration() {
|
private Configuration getConfiguration() {
|
||||||
Configuration conf = new Configuration();
|
Configuration conf = new Configuration();
|
||||||
conf.set("fs.test.impl.disable.cache", "true");
|
conf.set("fs.test.impl.disable.cache", "true");
|
||||||
|
@ -86,13 +106,14 @@ public class TestFileInputFormat {
|
||||||
}
|
}
|
||||||
|
|
||||||
static class MockFileSystem extends RawLocalFileSystem {
|
static class MockFileSystem extends RawLocalFileSystem {
|
||||||
|
int numListLocatedStatusCalls = 0;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FileStatus[] listStatus(Path f) throws FileNotFoundException,
|
public FileStatus[] listStatus(Path f) throws FileNotFoundException,
|
||||||
IOException {
|
IOException {
|
||||||
if (f.toString().equals("test:/a1")) {
|
if (f.toString().equals("test:/a1")) {
|
||||||
return new FileStatus[] {
|
return new FileStatus[] {
|
||||||
new FileStatus(10, true, 1, 150, 150, new Path("test:/a1/a2")),
|
new FileStatus(0, true, 1, 150, 150, new Path("test:/a1/a2")),
|
||||||
new FileStatus(10, false, 1, 150, 150, new Path("test:/a1/file1")) };
|
new FileStatus(10, false, 1, 150, 150, new Path("test:/a1/file1")) };
|
||||||
} else if (f.toString().equals("test:/a1/a2")) {
|
} else if (f.toString().equals("test:/a1/a2")) {
|
||||||
return new FileStatus[] {
|
return new FileStatus[] {
|
||||||
|
@ -116,5 +137,20 @@ public class TestFileInputFormat {
|
||||||
throws FileNotFoundException, IOException {
|
throws FileNotFoundException, IOException {
|
||||||
return this.listStatus(f);
|
return this.listStatus(f);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public BlockLocation[] getFileBlockLocations(Path p, long start, long len)
|
||||||
|
throws IOException {
|
||||||
|
return new BlockLocation[] {
|
||||||
|
new BlockLocation(new String[] { "localhost:50010" },
|
||||||
|
new String[] { "localhost" }, 0, len) };
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f,
|
||||||
|
PathFilter filter) throws FileNotFoundException, IOException {
|
||||||
|
++numListLocatedStatusCalls;
|
||||||
|
return super.listLocatedStatus(f, filter);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue