// should also return false.
assertFalse(rr.nextKeyValue());
}
public void testSplitPlacement() throws IOException {
MiniDFSCluster dfs = null;
FileSystem fileSys = null;
try {
/* Start 3 datanodes, one each in rack r1, r2, r3. Create three files
* 1) file1, just after starting the datanode on r1, with
* a repl factor of 1, and,
* 2) file2, just after starting the datanode on r2, with
* a repl factor of 2, and,
* 3) file3 after starting the all three datanodes, with a repl
* factor of 3.
* At the end, file1 will be present on only datanode1, file2 will be
* present on datanode 1 and datanode2 and
* file3 will be present on all datanodes.
*/
Configuration conf = new Configuration();
conf.setBoolean("dfs.replication.considerLoad", false);
dfs = new MiniDFSCluster(conf, 1, true, rack1, hosts1);
dfs.waitActive();
fileSys = dfs.getFileSystem();
if (!fileSys.mkdirs(inDir)) {
throw new IOException("Mkdirs failed to create " + inDir.toString());
}
Path file1 = new Path(dir1 + "/file1");
writeFile(conf, file1, (short)1, 1);
dfs.startDataNodes(conf, 1, true, null, rack2, hosts2, null);
dfs.waitActive();
// create file on two datanodes.
Path file2 = new Path(dir2 + "/file2");
writeFile(conf, file2, (short)2, 2);
// split it using a CombinedFile input format
DummyInputFormat inFormat = new DummyInputFormat();
Job job = new Job(conf);
FileInputFormat.setInputPaths(job, dir1 + "," + dir2);
inFormat.setMinSplitSizeRack(BLOCKSIZE);
List<InputSplit> splits = inFormat.getSplits(job);
System.out.println("Made splits(Test1): " + splits.size());
// make sure that each split has different locations
for (InputSplit split : splits) {
System.out.println("File split(Test1): " + split);
}
assertEquals(splits.size(), 2);
CombineFileSplit fileSplit = (CombineFileSplit) splits.get(0);
assertEquals(fileSplit.getNumPaths(), 2);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getPath(0).getName(), file2.getName());
assertEquals(fileSplit.getOffset(0), 0);
assertEquals(fileSplit.getLength(0), BLOCKSIZE);
assertEquals(fileSplit.getPath(1).getName(), file2.getName());
assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
assertEquals(fileSplit.getLength(1), BLOCKSIZE);
assertEquals(fileSplit.getLocations()[0], hosts2[0]); // should be on r2
fileSplit = (CombineFileSplit) splits.get(1);
assertEquals(fileSplit.getNumPaths(), 1);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getPath(0).getName(), file1.getName());
assertEquals(fileSplit.getOffset(0), 0);
assertEquals(fileSplit.getLength(0), BLOCKSIZE);
assertEquals(fileSplit.getLocations()[0], hosts1[0]); // should be on r1
// create another file on 3 datanodes and 3 racks.
dfs.startDataNodes(conf, 1, true, null, rack3, hosts3, null);
dfs.waitActive();
Path file3 = new Path(dir3 + "/file3");
writeFile(conf, new Path(dir3 + "/file3"), (short)3, 3);
inFormat = new DummyInputFormat();
FileInputFormat.setInputPaths(job, dir1 + "," + dir2 + "," + dir3);
inFormat.setMinSplitSizeRack(BLOCKSIZE);
splits = inFormat.getSplits(job);
for (InputSplit split : splits) {
System.out.println("File split(Test2): " + split);
}
assertEquals(splits.size(), 3);
fileSplit = (CombineFileSplit) splits.get(0);
assertEquals(fileSplit.getNumPaths(), 3);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getPath(0).getName(), file3.getName());
assertEquals(fileSplit.getOffset(0), 0);
assertEquals(fileSplit.getLength(0), BLOCKSIZE);
assertEquals(fileSplit.getPath(1).getName(), file3.getName());
assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
assertEquals(fileSplit.getLength(1), BLOCKSIZE);
assertEquals(fileSplit.getPath(2).getName(), file3.getName());
assertEquals(fileSplit.getOffset(2), 2 * BLOCKSIZE);
assertEquals(fileSplit.getLength(2), BLOCKSIZE);
assertEquals(fileSplit.getLocations()[0], hosts3[0]); // should be on r3
fileSplit = (CombineFileSplit) splits.get(1);
assertEquals(fileSplit.getNumPaths(), 2);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getPath(0).getName(), file2.getName());
assertEquals(fileSplit.getOffset(0), 0);
assertEquals(fileSplit.getLength(0), BLOCKSIZE);
assertEquals(fileSplit.getPath(1).getName(), file2.getName());
assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
assertEquals(fileSplit.getLength(1), BLOCKSIZE);
assertEquals(fileSplit.getLocations()[0], hosts2[0]); // should be on r2
fileSplit = (CombineFileSplit) splits.get(2);
assertEquals(fileSplit.getNumPaths(), 1);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getPath(0).getName(), file1.getName());
assertEquals(fileSplit.getOffset(0), 0);
assertEquals(fileSplit.getLength(0), BLOCKSIZE);
assertEquals(fileSplit.getLocations()[0], hosts1[0]); // should be on r1
// create file4 on all three racks
Path file4 = new Path(dir4 + "/file4");
writeFile(conf, file4, (short)3, 3);
inFormat = new DummyInputFormat();
FileInputFormat.setInputPaths(job, dir1 + "," + dir2 + "," + dir3 + "," + dir4);
inFormat.setMinSplitSizeRack(BLOCKSIZE);
splits = inFormat.getSplits(job);
for (InputSplit split : splits) {
System.out.println("File split(Test3): " + split);
}
assertEquals(splits.size(), 3);
fileSplit = (CombineFileSplit) splits.get(0);
assertEquals(fileSplit.getNumPaths(), 6);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getPath(0).getName(), file3.getName());
assertEquals(fileSplit.getOffset(0), 0);
assertEquals(fileSplit.getLength(0), BLOCKSIZE);
assertEquals(fileSplit.getPath(1).getName(), file3.getName());
assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
assertEquals(fileSplit.getLength(1), BLOCKSIZE);
assertEquals(fileSplit.getPath(2).getName(), file3.getName());
assertEquals(fileSplit.getOffset(2), 2 * BLOCKSIZE);
assertEquals(fileSplit.getLength(2), BLOCKSIZE);
assertEquals(fileSplit.getLocations()[0], hosts3[0]); // should be on r3
fileSplit = (CombineFileSplit) splits.get(1);
assertEquals(fileSplit.getNumPaths(), 2);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getPath(0).getName(), file2.getName());
assertEquals(fileSplit.getOffset(0), 0);
assertEquals(fileSplit.getLength(0), BLOCKSIZE);
assertEquals(fileSplit.getPath(1).getName(), file2.getName());
assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
assertEquals(fileSplit.getLength(1), BLOCKSIZE);
assertEquals(fileSplit.getLocations()[0], hosts2[0]); // should be on r2
fileSplit = (CombineFileSplit) splits.get(2);
assertEquals(fileSplit.getNumPaths(), 1);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getPath(0).getName(), file1.getName());
assertEquals(fileSplit.getOffset(0), 0);
assertEquals(fileSplit.getLength(0), BLOCKSIZE);
assertEquals(fileSplit.getLocations()[0], hosts1[0]); // should be on r1
// maximum split size is 2 blocks
inFormat = new DummyInputFormat();
inFormat.setMinSplitSizeNode(BLOCKSIZE);
inFormat.setMaxSplitSize(2*BLOCKSIZE);
FileInputFormat.setInputPaths(job,
dir1 + "," + dir2 + "," + dir3 + "," + dir4);
splits = inFormat.getSplits(job);
for (InputSplit split : splits) {
System.out.println("File split(Test4): " + split);
}
assertEquals(splits.size(), 5);
fileSplit = (CombineFileSplit) splits.get(0);
assertEquals(fileSplit.getNumPaths(), 2);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getPath(0).getName(), file3.getName());
assertEquals(fileSplit.getOffset(0), 0);
assertEquals(fileSplit.getLength(0), BLOCKSIZE);
assertEquals(fileSplit.getPath(1).getName(), file3.getName());
assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
assertEquals(fileSplit.getLength(1), BLOCKSIZE);
assertEquals(fileSplit.getLocations()[0], "host3.rack3.com");
fileSplit = (CombineFileSplit) splits.get(1);
assertEquals(fileSplit.getPath(0).getName(), file3.getName());
assertEquals(fileSplit.getOffset(0), 2 * BLOCKSIZE);
assertEquals(fileSplit.getLength(0), BLOCKSIZE);
assertEquals(fileSplit.getPath(1).getName(), file4.getName());
assertEquals(fileSplit.getOffset(1), 0);
assertEquals(fileSplit.getLength(1), BLOCKSIZE);
assertEquals(fileSplit.getLocations()[0], "host3.rack3.com");
fileSplit = (CombineFileSplit) splits.get(2);
assertEquals(fileSplit.getNumPaths(), 2);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getPath(0).getName(), file4.getName());
assertEquals(fileSplit.getOffset(0), BLOCKSIZE);
assertEquals(fileSplit.getLength(0), BLOCKSIZE);
assertEquals(fileSplit.getPath(1).getName(), file4.getName());
assertEquals(fileSplit.getOffset(1), 2 * BLOCKSIZE);
assertEquals(fileSplit.getLength(1), BLOCKSIZE);
assertEquals(fileSplit.getLocations()[0], "host3.rack3.com");
// maximum split size is 3 blocks
inFormat = new DummyInputFormat();
inFormat.setMinSplitSizeNode(BLOCKSIZE);
inFormat.setMaxSplitSize(3*BLOCKSIZE);
FileInputFormat.setInputPaths(job,
dir1 + "," + dir2 + "," + dir3 + "," + dir4);
splits = inFormat.getSplits(job);
for (InputSplit split : splits) {
System.out.println("File split(Test1): " + split);
}
assertEquals(splits.size(), 4);
fileSplit = (CombineFileSplit) splits.get(0);
assertEquals(fileSplit.getNumPaths(), 3);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getPath(0).getName(), file3.getName());
assertEquals(fileSplit.getOffset(0), 0);
assertEquals(fileSplit.getLength(0), BLOCKSIZE);
assertEquals(fileSplit.getPath(1).getName(), file3.getName());
assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
assertEquals(fileSplit.getLength(1), BLOCKSIZE);
assertEquals(fileSplit.getPath(2).getName(), file3.getName());
assertEquals(fileSplit.getOffset(2), 2 * BLOCKSIZE);
assertEquals(fileSplit.getLength(2), BLOCKSIZE);
assertEquals(fileSplit.getLocations()[0], "host3.rack3.com");
fileSplit = (CombineFileSplit) splits.get(1);
assertEquals(fileSplit.getPath(0).getName(), file4.getName());
assertEquals(fileSplit.getOffset(0), 0);
assertEquals(fileSplit.getLength(0), BLOCKSIZE);
assertEquals(fileSplit.getPath(1).getName(), file4.getName());
assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
assertEquals(fileSplit.getLength(1), BLOCKSIZE);
assertEquals(fileSplit.getPath(2).getName(), file4.getName());
assertEquals(fileSplit.getOffset(2), 2 * BLOCKSIZE);
assertEquals(fileSplit.getLength(2), BLOCKSIZE);
assertEquals(fileSplit.getLocations()[0], "host3.rack3.com");
fileSplit = (CombineFileSplit) splits.get(2);
assertEquals(fileSplit.getNumPaths(), 2);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getPath(0).getName(), file2.getName());
assertEquals(fileSplit.getOffset(0), 0);
assertEquals(fileSplit.getLength(0), BLOCKSIZE);
assertEquals(fileSplit.getPath(1).getName(), file2.getName());
assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
assertEquals(fileSplit.getLength(1), BLOCKSIZE);
assertEquals(fileSplit.getLocations()[0], "host2.rack2.com");
fileSplit = (CombineFileSplit) splits.get(3);
assertEquals(fileSplit.getNumPaths(), 1);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getPath(0).getName(), file1.getName());
assertEquals(fileSplit.getOffset(0), 0);
assertEquals(fileSplit.getLength(0), BLOCKSIZE);
assertEquals(fileSplit.getLocations()[0], "host1.rack1.com");
// maximum split size is 4 blocks
inFormat = new DummyInputFormat();
inFormat.setMaxSplitSize(4*BLOCKSIZE);
FileInputFormat.setInputPaths(job, dir1 + "," + dir2 + "," + dir3 + "," + dir4);
splits = inFormat.getSplits(job);
for (InputSplit split : splits) {
System.out.println("File split(Test1): " + split);
}
assertEquals(splits.size(), 3);
fileSplit = (CombineFileSplit) splits.get(0);
assertEquals(fileSplit.getNumPaths(), 4);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getPath(0).getName(), file3.getName());
assertEquals(fileSplit.getOffset(0), 0);
assertEquals(fileSplit.getLength(0), BLOCKSIZE);
assertEquals(fileSplit.getPath(1).getName(), file3.getName());
assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
assertEquals(fileSplit.getLength(1), BLOCKSIZE);
assertEquals(fileSplit.getPath(2).getName(), file3.getName());
assertEquals(fileSplit.getOffset(2), 2 * BLOCKSIZE);
assertEquals(fileSplit.getLength(2), BLOCKSIZE);
assertEquals(fileSplit.getLocations()[0], "host3.rack3.com");
fileSplit = (CombineFileSplit) splits.get(1);
assertEquals(fileSplit.getNumPaths(), 4);
assertEquals(fileSplit.getPath(0).getName(), file2.getName());
assertEquals(fileSplit.getOffset(0), 0);
assertEquals(fileSplit.getLength(0), BLOCKSIZE);
assertEquals(fileSplit.getPath(1).getName(), file2.getName());
assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
assertEquals(fileSplit.getLength(1), BLOCKSIZE);
assertEquals(fileSplit.getPath(2).getName(), file4.getName());
assertEquals(fileSplit.getOffset(2), BLOCKSIZE);
assertEquals(fileSplit.getLength(2), BLOCKSIZE);
assertEquals(fileSplit.getPath(3).getName(), file4.getName());
assertEquals(fileSplit.getOffset(3), 2 * BLOCKSIZE);
assertEquals(fileSplit.getLength(3), BLOCKSIZE);
assertEquals(fileSplit.getLocations()[0], "host2.rack2.com");
fileSplit = (CombineFileSplit) splits.get(2);
assertEquals(fileSplit.getNumPaths(), 1);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getPath(0).getName(), file1.getName());
assertEquals(fileSplit.getOffset(0), 0);
assertEquals(fileSplit.getLength(0), BLOCKSIZE);
assertEquals(fileSplit.getLocations()[0], hosts1[0]); // should be on r1
// maximum split size is 7 blocks and min is 3 blocks
inFormat = new DummyInputFormat();
inFormat.setMaxSplitSize(7*BLOCKSIZE);
inFormat.setMinSplitSizeNode(3*BLOCKSIZE);
inFormat.setMinSplitSizeRack(3*BLOCKSIZE);
FileInputFormat.setInputPaths(job,
dir1 + "," + dir2 + "," + dir3 + "," + dir4);
splits = inFormat.getSplits(job);
for (InputSplit split : splits) {
System.out.println("File split(Test1): " + split);
}
assertEquals(splits.size(), 2);
fileSplit = (CombineFileSplit) splits.get(0);
assertEquals(fileSplit.getNumPaths(), 6);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getLocations()[0], "host3.rack3.com");
fileSplit = (CombineFileSplit) splits.get(1);
assertEquals(fileSplit.getNumPaths(), 3);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getLocations()[0], "host1.rack1.com");
// Rack 1 has file1, file2 and file3 and file4
// Rack 2 has file2 and file3 and file4
// Rack 3 has file3 and file4
// setup a filter so that only file1 and file2 can be combined
inFormat = new DummyInputFormat();
FileInputFormat.addInputPath(job, inDir);
inFormat.setMinSplitSizeRack(1); // everything is at least rack local
inFormat.createPool(new TestFilter(dir1),
new TestFilter(dir2));
splits = inFormat.getSplits(job);
for (InputSplit split : splits) {
System.out.println("File split(Test1): " + split);
}
assertEquals(splits.size(), 3);
fileSplit = (CombineFileSplit) splits.get(0);
assertEquals(fileSplit.getNumPaths(), 2);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getLocations()[0], hosts2[0]); // should be on r2
fileSplit = (CombineFileSplit) splits.get(1);
assertEquals(fileSplit.getNumPaths(), 1);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getLocations()[0], hosts1[0]); // should be on r1
fileSplit = (CombineFileSplit) splits.get(2);
assertEquals(fileSplit.getNumPaths(), 6);
assertEquals(fileSplit.getLocations().length, 1);
assertEquals(fileSplit.getLocations()[0], hosts3[0]); // should be on r3
// measure performance when there are multiple pools and
// many files in each pool.
int numPools = 100;
int numFiles = 1000;
DummyInputFormat1 inFormat1 = new DummyInputFormat1();
for (int i = 0; i < numFiles; i++) {
FileInputFormat.setInputPaths(job, file1);
}
inFormat1.setMinSplitSizeRack(1); // everything is at least rack local
final Path dirNoMatch1 = new Path(inDir, "/dirxx");
final Path dirNoMatch2 = new Path(inDir, "/diryy");
for (int i = 0; i < numPools; i++) {
inFormat1.createPool(new TestFilter(dirNoMatch1),
new TestFilter(dirNoMatch2));
}
long start = System.currentTimeMillis();
splits = inFormat1.getSplits(job);
long end = System.currentTimeMillis();
System.out.println("Elapsed time for " + numPools + " pools " +
" and " + numFiles + " files is " +
((end - start)/1000) + " seconds.");
} finally {
if (dfs != null) {
dfs.shutdown();
}
}
}