|
@@ -32,11 +32,17 @@ import org.apache.hadoop.conf.Configuration;
|
|
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.HdfsBlockLocation;
|
|
import org.apache.hadoop.fs.LocatedFileStatus;
|
|
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.fs.RemoteIterator;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
import org.apache.hadoop.mapred.SplitLocationInfo;
|
|
import org.apache.hadoop.mapred.SplitLocationInfo;
|
|
import org.apache.hadoop.mapreduce.InputSplit;
|
|
import org.apache.hadoop.mapreduce.InputSplit;
|
|
import org.apache.hadoop.mapreduce.Job;
|
|
import org.apache.hadoop.mapreduce.Job;
|
|
@@ -238,6 +244,50 @@ public class TestFileInputFormat {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Test
|
|
|
|
+ public void testShrinkStatus() throws IOException {
|
|
|
|
+ Configuration conf = getConfiguration();
|
|
|
|
+ MockFileSystem mockFs =
|
|
|
|
+ (MockFileSystem) new Path("test:///").getFileSystem(conf);
|
|
|
|
+ Path dir1 = new Path("test:/a1");
|
|
|
|
+ RemoteIterator<LocatedFileStatus> statuses = mockFs.listLocatedStatus(dir1);
|
|
|
|
+ boolean verified = false;
|
|
|
|
+ while (statuses.hasNext()) {
|
|
|
|
+ LocatedFileStatus orig = statuses.next();
|
|
|
|
+ LocatedFileStatus shrink =
|
|
|
|
+ (LocatedFileStatus)FileInputFormat.shrinkStatus(orig);
|
|
|
|
+ Assert.assertTrue(orig.equals(shrink));
|
|
|
|
+ if (shrink.getBlockLocations() != null) {
|
|
|
|
+ Assert.assertEquals(orig.getBlockLocations().length,
|
|
|
|
+ shrink.getBlockLocations().length);
|
|
|
|
+ for (int i = 0; i < shrink.getBlockLocations().length; i++) {
|
|
|
|
+ verified = true;
|
|
|
|
+ BlockLocation location = shrink.getBlockLocations()[i];
|
|
|
|
+ BlockLocation actual = orig.getBlockLocations()[i];
|
|
|
|
+ Assert.assertNotNull(((HdfsBlockLocation)actual).getLocatedBlock());
|
|
|
|
+ Assert.assertEquals(BlockLocation.class.getName(),
|
|
|
|
+ location.getClass().getName());
|
|
|
|
+ Assert.assertArrayEquals(actual.getHosts(), location.getHosts());
|
|
|
|
+ Assert.assertArrayEquals(actual.getCachedHosts(),
|
|
|
|
+ location.getCachedHosts());
|
|
|
|
+ Assert.assertArrayEquals(actual.getStorageIds(),
|
|
|
|
+ location.getStorageIds());
|
|
|
|
+ Assert.assertArrayEquals(actual.getStorageTypes(),
|
|
|
|
+ location.getStorageTypes());
|
|
|
|
+ Assert.assertArrayEquals(actual.getTopologyPaths(),
|
|
|
|
+ location.getTopologyPaths());
|
|
|
|
+ Assert.assertArrayEquals(actual.getNames(), location.getNames());
|
|
|
|
+ Assert.assertEquals(actual.getLength(), location.getLength());
|
|
|
|
+ Assert.assertEquals(actual.getOffset(), location.getOffset());
|
|
|
|
+ Assert.assertEquals(actual.isCorrupt(), location.isCorrupt());
|
|
|
|
+ }
|
|
|
|
+ } else {
|
|
|
|
+ Assert.assertTrue(orig.getBlockLocations() == null);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ Assert.assertTrue(verified);
|
|
|
|
+ }
|
|
|
|
+
|
|
public static List<Path> configureTestSimple(Configuration conf, FileSystem localFs)
|
|
public static List<Path> configureTestSimple(Configuration conf, FileSystem localFs)
|
|
throws IOException {
|
|
throws IOException {
|
|
Path base1 = new Path(TEST_ROOT_DIR, "input1");
|
|
Path base1 = new Path(TEST_ROOT_DIR, "input1");
|
|
@@ -437,10 +487,31 @@ public class TestFileInputFormat {
|
|
@Override
|
|
@Override
|
|
public BlockLocation[] getFileBlockLocations(FileStatus file, long start, long len)
|
|
public BlockLocation[] getFileBlockLocations(FileStatus file, long start, long len)
|
|
throws IOException {
|
|
throws IOException {
|
|
- return new BlockLocation[] {
|
|
|
|
- new BlockLocation(new String[] { "localhost:9866", "otherhost:9866" },
|
|
|
|
- new String[] { "localhost", "otherhost" }, new String[] { "localhost" },
|
|
|
|
- new String[0], 0, len, false) }; }
|
|
|
|
|
|
+ DatanodeInfo[] ds = new DatanodeInfo[2];
|
|
|
|
+ ds[0] = new DatanodeDescriptor(
|
|
|
|
+ new DatanodeID("127.0.0.1", "localhost", "abcd",
|
|
|
|
+ 9866, 9867, 9868, 9869));
|
|
|
|
+ ds[1] = new DatanodeDescriptor(
|
|
|
|
+ new DatanodeID("1.0.0.1", "otherhost", "efgh",
|
|
|
|
+ 9866, 9867, 9868, 9869));
|
|
|
|
+ long blockLen = len / 3;
|
|
|
|
+ ExtendedBlock b1 = new ExtendedBlock("bpid", 0, blockLen, 0);
|
|
|
|
+ ExtendedBlock b2 = new ExtendedBlock("bpid", 1, blockLen, 1);
|
|
|
|
+ ExtendedBlock b3 = new ExtendedBlock("bpid", 2, len - 2 * blockLen, 2);
|
|
|
|
+ String[] names = new String[]{ "localhost:9866", "otherhost:9866" };
|
|
|
|
+ String[] hosts = new String[]{ "localhost", "otherhost" };
|
|
|
|
+ String[] cachedHosts = {"localhost"};
|
|
|
|
+ BlockLocation loc1 = new BlockLocation(names, hosts, cachedHosts,
|
|
|
|
+ new String[0], 0, blockLen, false);
|
|
|
|
+ BlockLocation loc2 = new BlockLocation(names, hosts, cachedHosts,
|
|
|
|
+ new String[0], blockLen, blockLen, false);
|
|
|
|
+ BlockLocation loc3 = new BlockLocation(names, hosts, cachedHosts,
|
|
|
|
+ new String[0], 2 * blockLen, len - 2 * blockLen, false);
|
|
|
|
+ return new BlockLocation[]{
|
|
|
|
+ new HdfsBlockLocation(loc1, new LocatedBlock(b1, ds)),
|
|
|
|
+ new HdfsBlockLocation(loc2, new LocatedBlock(b2, ds)),
|
|
|
|
+ new HdfsBlockLocation(loc3, new LocatedBlock(b3, ds)) };
|
|
|
|
+ }
|
|
|
|
|
|
@Override
|
|
@Override
|
|
protected RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f,
|
|
protected RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f,
|