From 7bdf8f9337a4b25187b0d115d4470f597a71dd03 Mon Sep 17 00:00:00 2001 From: 18chanp1 <89321535+18chanp1@users.noreply.github.com> Date: Sun, 14 Dec 2025 22:22:19 -0600 Subject: [PATCH] MAPREDUCE-7524 Addressed indeterminism in TestCombineFileInputFormat by removing assertions based on the internal implementation of HDFS. --- .../lib/input/TestCombineFileInputFormat.java | 391 ++++++++---------- 1 file changed, 171 insertions(+), 220 deletions(-) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java index 0f4a8b74195c1..13556d8f0797d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java @@ -20,13 +20,7 @@ import java.io.IOException; import java.io.OutputStream; import java.net.URI; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.TreeMap; +import java.util.*; import java.util.concurrent.TimeoutException; import java.util.zip.GZIPOutputStream; @@ -63,10 +57,7 @@ import org.apache.hadoop.thirdparty.com.google.common.collect.HashMultiset; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.Assertions.fail; +import static org.junit.jupiter.api.Assertions.*; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.times; import static org.mockito.Mockito.reset; @@ -230,6 +221,34 @@ public void close() { } } + /** + * Helper methods to locate splits without using index + */ + private CombineFileSplit findSplitByHost( + List splits, String host) throws IOException { + for (InputSplit s : splits) { + CombineFileSplit cfs = (CombineFileSplit) s; + if (cfs.getLocations().length == 1 && + cfs.getLocations()[0].equals(host)) { + return cfs; + } + } + return null; + } + + private CombineFileSplit findSplitContaining( + List splits, String fileName) throws IOException { + for (InputSplit s : splits) { + CombineFileSplit cfs = (CombineFileSplit) s; + for (int i = 0; i < cfs.getNumPaths(); i++) { + if (cfs.getPath(i).getName().equals(fileName)) { + return cfs; + } + } + } + return null; + } + /** Extend CFIF to use CFRR with DummyRecordReader */ private class ChildRRInputFormat extends CombineFileInputFormat { @SuppressWarnings("unchecked") @@ -353,6 +372,12 @@ public boolean equals(Object obj) { } return false; } + + @Override + public int hashCode() { + return Objects.hash(name, length, offset); + } + } /** @@ -1137,6 +1162,8 @@ public void testSplitPlacementForCompressedFiles() throws Exception { // create another file on the same datanode Path file5 = new Path(dir5 + "/file5.gz"); FileStatus f5 = writeGzipFile(conf, file5, (short)1, 1); + DFSTestUtil.waitReplication(fileSys, file1, (short)1); + DFSTestUtil.waitReplication(fileSys, file5, (short)1); // split it using a CombinedFile input format DummyInputFormat inFormat = new DummyInputFormat(); Job job = Job.getInstance(conf); @@ -1147,16 +1174,31 @@ public void testSplitPlacementForCompressedFiles() throws Exception { System.out.println("File split(Test0): " + split); } assertEquals(1, splits.size()); - CombineFileSplit fileSplit = (CombineFileSplit) splits.get(0); - assertEquals(2, fileSplit.getNumPaths()); - assertEquals(1, fileSplit.getLocations().length); - assertEquals(file1.getName(), fileSplit.getPath(0).getName()); - assertEquals(0, fileSplit.getOffset(0)); - assertEquals(f1.getLen(), fileSplit.getLength(0)); - assertEquals(file5.getName(), fileSplit.getPath(1).getName()); - assertEquals(0, fileSplit.getOffset(1)); - assertEquals(f5.getLen(), fileSplit.getLength(1)); - assertEquals(hosts1[0], fileSplit.getLocations()[0]); + Set expectedFiles = Set.of("file1.gz", "file5.gz"); + Set actualFiles = new HashSet<>(); + Set actualHosts = new HashSet<>(); + for (InputSplit split : splits) { + CombineFileSplit fileSplit = (CombineFileSplit) split; + assertEquals(2, fileSplit.getNumPaths()); + assertEquals(1, fileSplit.getLocations().length); + for (int i = 0; i < fileSplit.getNumPaths(); i++) { + Path path = fileSplit.getPath(i); + long offset = fileSplit.getOffset(i); + long length = fileSplit.getLength(i); + actualFiles.add(path.getName()); + + if (path.getName().equals("file1.gz")) { + assertEquals(0, offset); + assertEquals(f1.getLen(), length); + } else if (path.getName().equals("file5.gz")) { + assertEquals(0, offset); + assertEquals(f5.getLen(), length); + } + } + actualHosts.addAll(Arrays.asList(fileSplit.getLocations())); + } + assertEquals(expectedFiles, actualFiles); + assertTrue(actualHosts.contains(hosts1[0])); dfs.startDataNodes(conf, 1, true, null, rack2, hosts2, null); dfs.waitActive(); @@ -1164,6 +1206,7 @@ public void testSplitPlacementForCompressedFiles() throws Exception { // create file on two datanodes. Path file2 = new Path(dir2 + "/file2.gz"); FileStatus f2 = writeGzipFile(conf, file2, (short)2, 2); + DFSTestUtil.waitReplication(fileSys, file2, (short)2); // split it using a CombinedFile input format inFormat = new DummyInputFormat(); @@ -1171,16 +1214,26 @@ public void testSplitPlacementForCompressedFiles() throws Exception { inFormat.setMinSplitSizeRack(f1.getLen()); splits = inFormat.getSplits(job); System.out.println("Made splits(Test1): " + splits.size()); + expectedFiles = Set.of("file1.gz", "file2.gz"); + actualFiles.clear(); + actualHosts.clear(); // make sure that each split has different locations for (InputSplit split : splits) { System.out.println("File split(Test1): " + split); + CombineFileSplit fileSplit = (CombineFileSplit) split; + for(Path path : fileSplit.getPaths()) { + actualFiles.add(path.getName()); + } + actualHosts.addAll(Arrays.asList(fileSplit.getLocations())); } - + assertEquals(expectedFiles, actualFiles); + assertTrue(actualHosts.contains(hosts1[0]) || + actualHosts.contains(hosts2[0])); Set expected = new HashSet<>(); expected.add(new Split(file1.getName(), f1.getLen(), 0)); expected.add(new Split(file2.getName(), f2.getLen(), 0)); - List actual = new ArrayList<>(); + Set actual = new HashSet<>(); /** * If rack1 is processed first by @@ -1188,49 +1241,31 @@ public void testSplitPlacementForCompressedFiles() throws Exception { * create only one split on rack1. Otherwise create two splits. */ for (InputSplit split : splits) { - fileSplit = (CombineFileSplit) split; - if (splits.size() == 2) { - if (split.equals(splits.get(0))) { - // first split is on rack2, contains file2. - assertEquals(1, fileSplit.getNumPaths()); - assertEquals(1, fileSplit.getLocations().length); - assertEquals(file2.getName(), fileSplit.getPath(0).getName()); - assertEquals(0, fileSplit.getOffset(0)); - assertEquals(f2.getLen(), fileSplit.getLength(0)); - assertEquals(hosts2[0], fileSplit.getLocations()[0]); - } - if (split.equals(splits.get(1))) { - // second split is on rack1, contains file1. - assertEquals(1, fileSplit.getNumPaths()); - assertEquals(1, fileSplit.getLocations().length); - assertEquals(file1.getName(), fileSplit.getPath(0).getName()); - assertEquals(0, fileSplit.getOffset(0)); - assertEquals(f1.getLen(), fileSplit.getLength(0)); - assertEquals(hosts1[0], fileSplit.getLocations()[0]); - } - } else if (splits.size() == 1) { - // first split is on rack1, contains file1 and file2. - assertEquals(2, fileSplit.getNumPaths()); - assertEquals(1, fileSplit.getLocations().length); - assertEquals(hosts1[0], fileSplit.getLocations()[0]); - } else { - fail("Split size should be 1 or 2."); - } - for (int i = 0; i < fileSplit.getNumPaths(); i++) { + CombineFileSplit fileSplit = (CombineFileSplit) split; + assertEquals(1, fileSplit.getLocations().length); + List locations = Arrays.asList(fileSplit.getLocations()); + assertTrue( + locations.contains(hosts1[0]) || locations.contains(hosts2[0]) + ); + assertTrue(fileSplit.getNumPaths() == 1 || fileSplit.getNumPaths() == 2); + for(int i = 0; i < fileSplit.getNumPaths(); i++) { String name = fileSplit.getPath(i).getName(); long length = fileSplit.getLength(i); long offset = fileSplit.getOffset(i); actual.add(new Split(name, length, offset)); + + assertEquals(0, fileSplit.getOffset(i)); } } + assertEquals(expected, actual); assertEquals(2, actual.size()); - assertTrue(actual.containsAll(expected)); // 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.gz"); FileStatus f3 = writeGzipFile(conf, file3, (short)3, 3); + DFSTestUtil.waitReplication(fileSys, file3, (short)3); inFormat = new DummyInputFormat(); FileInputFormat.setInputPaths(job, dir1 + "," + dir2 + "," + dir3); inFormat.setMinSplitSizeRack(f1.getLen()); @@ -1238,12 +1273,17 @@ public void testSplitPlacementForCompressedFiles() throws Exception { for (InputSplit split : splits) { System.out.println("File split(Test2): " + split); } - + expectedFiles = Set.of("file1.gz", "file2.gz", "file3.gz"); + actualFiles.clear(); + actualHosts.clear(); + expected = new HashSet<>(); + expected.add(new Split(file1.getName(), f1.getLen(), 0)); + expected.add(new Split(file2.getName(), f2.getLen(), 0)); expected.add(new Split(file3.getName(), f3.getLen(), 0)); actual.clear(); for (InputSplit split : splits) { - fileSplit = (CombineFileSplit) split; + CombineFileSplit fileSplit = (CombineFileSplit) split; /** * If rack1 is processed first by * {@link CombineFileInputFormat#createSplits}, @@ -1252,73 +1292,45 @@ public void testSplitPlacementForCompressedFiles() throws Exception { * create one split on rack2 or rack3 and the other split is on rack1. * Otherwise create 3 splits for each rack. */ - if (splits.size() == 3) { - // first split is on rack3, contains file3 - if (split.equals(splits.get(0))) { - assertEquals(1, fileSplit.getNumPaths()); - assertEquals(file3.getName(), fileSplit.getPath(0).getName()); - assertEquals(f3.getLen(), fileSplit.getLength(0)); - assertEquals(0, fileSplit.getOffset(0)); - assertEquals(1, fileSplit.getLocations().length); - assertEquals(hosts3[0], fileSplit.getLocations()[0]); - } - // second split is on rack2, contains file2 - if (split.equals(splits.get(1))) { - assertEquals(1, fileSplit.getNumPaths()); - assertEquals(file2.getName(), fileSplit.getPath(0).getName()); - assertEquals(f2.getLen(), fileSplit.getLength(0)); - assertEquals(0, fileSplit.getOffset(0)); - assertEquals(1, fileSplit.getLocations().length); - assertEquals(hosts2[0], fileSplit.getLocations()[0]); - } - // third split is on rack1, contains file1 - if (split.equals(splits.get(2))) { - assertEquals(1, fileSplit.getNumPaths()); - assertEquals(file1.getName(), fileSplit.getPath(0).getName()); - assertEquals(f1.getLen(), fileSplit.getLength(0)); - assertEquals(0, fileSplit.getOffset(0)); - assertEquals(1, fileSplit.getLocations().length); - assertEquals(hosts1[0], fileSplit.getLocations()[0]); - } - } else if (splits.size() == 2) { - // first split is on rack2 or rack3, contains one or two files. - if (split.equals(splits.get(0))) { - assertEquals(1, fileSplit.getLocations().length); - if (fileSplit.getLocations()[0].equals(hosts2[0])) { - assertEquals(2, fileSplit.getNumPaths()); - } else if (fileSplit.getLocations()[0].equals(hosts3[0])) { - assertEquals(1, fileSplit.getNumPaths()); - } else { - fail("First split should be on rack2 or rack3."); - } - } - // second split is on rack1, contains the rest files. - if (split.equals(splits.get(1))) { - assertEquals(1, fileSplit.getLocations().length); - assertEquals(hosts1[0], fileSplit.getLocations()[0]); - } - } else if (splits.size() == 1) { - // first split is rack1, contains all three files. - assertEquals(1, fileSplit.getLocations().length); - assertEquals(3, fileSplit.getNumPaths()); - assertEquals(hosts1[0], fileSplit.getLocations()[0]); - } else { - fail("Split size should be 1, 2, or 3."); - } - for (int i = 0; i < fileSplit.getNumPaths(); i++) { - String name = fileSplit.getPath(i).getName(); - long length = fileSplit.getLength(i); + + assertTrue(fileSplit.getNumPaths() >= 1); + String location = fileSplit.getLocations()[0]; + assertTrue( + location.equals(hosts1[0]) || + location.equals(hosts2[0]) || + location.equals(hosts3[0]), + "Unexpected split location: " + location + ); + + int numPaths = fileSplit.getNumPaths(); + assertTrue(numPaths >= 1 && numPaths <= 3, + "Unexpected number of paths: " + numPaths); + + for (int i = 0; i < numPaths; i++) { + Path path = fileSplit.getPath(i); long offset = fileSplit.getOffset(i); - actual.add(new Split(name, length, offset)); + long length = fileSplit.getLength(i); + + assertEquals(0, offset); + + if (path.getName().equals(file1.getName())) { + assertEquals(f1.getLen(), length); + } else if (path.getName().equals(file2.getName())) { + assertEquals(f2.getLen(), length); + } else if (path.getName().equals(file3.getName())) { + assertEquals(f3.getLen(), length); + } else { + fail("Unexpected file in split: " + path); + } + actual.add(new Split(path.getName(), length, offset)); } } - - assertEquals(3, actual.size()); - assertTrue(actual.containsAll(expected)); + assertEquals(expected, actual); // create file4 on all three racks Path file4 = new Path(dir4 + "/file4.gz"); FileStatus f4 = writeGzipFile(conf, file4, (short)3, 3); + DFSTestUtil.waitReplication(fileSys, file4, (short)3); inFormat = new DummyInputFormat(); FileInputFormat.setInputPaths(job, dir1 + "," + dir2 + "," + dir3 + "," + dir4); @@ -1327,12 +1339,17 @@ public void testSplitPlacementForCompressedFiles() throws Exception { for (InputSplit split : splits) { System.out.println("File split(Test3): " + split); } - + expected = new HashSet<>(); + expected.add(new Split(file1.getName(), f1.getLen(), 0)); + expected.add(new Split(file2.getName(), f2.getLen(), 0)); expected.add(new Split(file3.getName(), f3.getLen(), 0)); + expected.add(new Split(file4.getName(), f4.getLen(), 0)); + actual.clear(); + Set hostsSeen = new HashSet<>(); for (InputSplit split : splits) { - fileSplit = (CombineFileSplit) split; + CombineFileSplit fileSplit = (CombineFileSplit) split; /** * If rack1 is processed first by * {@link CombineFileInputFormat#createSplits}, @@ -1341,65 +1358,25 @@ public void testSplitPlacementForCompressedFiles() throws Exception { * create one split on rack2 or rack3 and the other split is on rack1. * Otherwise create 3 splits for each rack. */ - if (splits.size() == 3) { - // first split is on rack3, contains file3 and file4 - if (split.equals(splits.get(0))) { - assertEquals(2, fileSplit.getNumPaths()); - assertEquals(hosts3[0], fileSplit.getLocations()[0]); - } - // second split is on rack2, contains file2 - if (split.equals(splits.get(1))) { - assertEquals(1, fileSplit.getNumPaths()); - assertEquals(file2.getName(), fileSplit.getPath(0).getName()); - assertEquals(f2.getLen(), fileSplit.getLength(0)); - assertEquals(0, fileSplit.getOffset(0)); - assertEquals(1, fileSplit.getLocations().length); - assertEquals(hosts2[0], fileSplit.getLocations()[0]); - } - // third split is on rack1, contains file1 - if (split.equals(splits.get(2))) { - assertEquals(1, fileSplit.getNumPaths()); - assertEquals(file1.getName(), fileSplit.getPath(0).getName()); - assertEquals(f1.getLen(), fileSplit.getLength(0)); - assertEquals(0, fileSplit.getOffset(0)); - assertEquals(1, fileSplit.getLocations().length); - assertEquals(hosts1[0], fileSplit.getLocations()[0]); - } - } else if (splits.size() == 2) { - // first split is on rack2 or rack3, contains two or three files. - if (split.equals(splits.get(0))) { - assertEquals(1, fileSplit.getLocations().length); - if (fileSplit.getLocations()[0].equals(hosts2[0])) { - assertEquals(3, fileSplit.getNumPaths()); - } else if (fileSplit.getLocations()[0].equals(hosts3[0])) { - assertEquals(2, fileSplit.getNumPaths()); - } else { - fail("First split should be on rack2 or rack3."); - } - } - // second split is on rack1, contains the rest files. - if (split.equals(splits.get(1))) { - assertEquals(1, fileSplit.getLocations().length); - assertEquals(hosts1[0], fileSplit.getLocations()[0]); - } - } else if (splits.size() == 1) { - // first split is rack1, contains all four files. - assertEquals(1, fileSplit.getLocations().length); - assertEquals(4, fileSplit.getNumPaths()); - assertEquals(hosts1[0], fileSplit.getLocations()[0]); - } else { - fail("Split size should be 1, 2, or 3."); - } + + assertTrue(fileSplit.getNumPaths() >= 1); + assertTrue(fileSplit.getLocations().length >= 1); + hostsSeen.addAll(Arrays.asList(fileSplit.getLocations())); + for (int i = 0; i < fileSplit.getNumPaths(); i++) { - String name = fileSplit.getPath(i).getName(); - long length = fileSplit.getLength(i); + Path path = fileSplit.getPath(i); long offset = fileSplit.getOffset(i); - actual.add(new Split(name, length, offset)); + long length = fileSplit.getLength(i); + + // Add each path's name, offset, and length to actual + actual.add(new Split(path.getName(), length, offset)); } } - + assertEquals(expected, actual); + assertTrue(hostsSeen.contains(hosts1[0]) || + hostsSeen.contains(hosts2[0]) || + hostsSeen.contains(hosts3[0])); assertEquals(4, actual.size()); - assertTrue(actual.containsAll(expected)); // maximum split size is file1's length inFormat = new DummyInputFormat(); @@ -1415,7 +1392,7 @@ public void testSplitPlacementForCompressedFiles() throws Exception { actual.clear(); for (InputSplit split : splits) { - fileSplit = (CombineFileSplit) split; + CombineFileSplit fileSplit = (CombineFileSplit) split; for (int i = 0; i < fileSplit.getNumPaths(); i++) { String name = fileSplit.getPath(i).getName(); long length = fileSplit.getLength(i); @@ -1444,8 +1421,10 @@ public void testSplitPlacementForCompressedFiles() throws Exception { actual.clear(); reset(mockList); + hostsSeen = new HashSet<>(); for (InputSplit split : splits) { - fileSplit = (CombineFileSplit) split; + CombineFileSplit fileSplit = (CombineFileSplit) split; + assertTrue(fileSplit.getNumPaths() >= 1); for (int i = 0; i < fileSplit.getNumPaths(); i++) { String name = fileSplit.getPath(i).getName(); long length = fileSplit.getLength(i); @@ -1453,21 +1432,12 @@ public void testSplitPlacementForCompressedFiles() throws Exception { actual.add(new Split(name, length, offset)); } mockList.add(fileSplit.getLocations()[0]); + hostsSeen.addAll(Arrays.asList(fileSplit.getLocations())); } + assertEquals(expected, actual); assertEquals(4, actual.size()); assertTrue(actual.containsAll(expected)); - if (splits.size() == 3) { - // splits are on all the racks - verify(mockList, times(1)).add(hosts1[0]); - verify(mockList, times(1)).add(hosts2[0]); - verify(mockList, times(1)).add(hosts3[0]); - } else if (splits.size() == 2) { - // one split is on rack1, another split is on rack2 or rack3 - verify(mockList, times(1)).add(hosts1[0]); - } else { - fail("Split size should be 2 or 3."); - } // maximum split size is 4 times file1's length inFormat = new DummyInputFormat(); @@ -1488,20 +1458,21 @@ public void testSplitPlacementForCompressedFiles() throws Exception { assertTrue(splits.size() == 1 || splits.size() == 2, "Split size should be 1 or 2."); actual.clear(); + hostsSeen = new HashSet<>(); reset(mockList); for (InputSplit split : splits) { - fileSplit = (CombineFileSplit) split; + CombineFileSplit fileSplit = (CombineFileSplit) split; + assertTrue(fileSplit.getNumPaths() >= 1); for (int i = 0; i < fileSplit.getNumPaths(); i++) { String name = fileSplit.getPath(i).getName(); long length = fileSplit.getLength(i); long offset = fileSplit.getOffset(i); actual.add(new Split(name, length, offset)); } - mockList.add(fileSplit.getLocations()[0]); + hostsSeen.addAll(Arrays.asList(fileSplit.getLocations())); } assertEquals(4, actual.size()); assertTrue(actual.containsAll(expected)); - verify(mockList, times(1)).add(hosts1[0]); // maximum split size and min-split-size per rack is 4 times file1's length inFormat = new DummyInputFormat(); @@ -1510,14 +1481,19 @@ public void testSplitPlacementForCompressedFiles() throws Exception { FileInputFormat.setInputPaths(job, dir1 + "," + dir2 + "," + dir3 + "," + dir4); splits = inFormat.getSplits(job); + actual.clear(); + hostsSeen = new HashSet<>(); for (InputSplit split : splits) { System.out.println("File split(Test7): " + split); } assertEquals(1, splits.size()); - fileSplit = (CombineFileSplit) splits.get(0); + CombineFileSplit fileSplit = (CombineFileSplit) splits.get(0); assertEquals(4, fileSplit.getNumPaths()); assertEquals(1, fileSplit.getLocations().length); assertEquals(hosts1[0], fileSplit.getLocations()[0]); + hostsSeen.clear(); + hostsSeen.addAll(Arrays.asList(fileSplit.getLocations())); + assertTrue(hostsSeen.contains(hosts1[0])); // minimum split size per node is 4 times file1's length inFormat = new DummyInputFormat(); @@ -1533,6 +1509,8 @@ public void testSplitPlacementForCompressedFiles() throws Exception { assertEquals(4, fileSplit.getNumPaths()); assertEquals(1, fileSplit.getLocations().length); assertEquals(hosts1[0], fileSplit.getLocations()[0]); + hostsSeen = new HashSet<>(Arrays.asList(fileSplit.getLocations())); + assertTrue(hostsSeen.contains(hosts1[0])); // Rack 1 has file1, file2 and file3 and file4 // Rack 2 has file2 and file3 and file4 @@ -1549,40 +1527,13 @@ public void testSplitPlacementForCompressedFiles() throws Exception { } actual.clear(); + hostsSeen = new HashSet<>(); for (InputSplit split : splits) { fileSplit = (CombineFileSplit) split; - if (splits.size() == 3) { - // If rack2 is processed first - if (split.equals(splits.get(0))) { - assertEquals(1, fileSplit.getNumPaths()); - assertEquals(1, fileSplit.getLocations().length); - assertEquals(hosts2[0], fileSplit.getLocations()[0]); - } - if (split.equals(splits.get(1))) { - assertEquals(1, fileSplit.getNumPaths()); - assertEquals(1, fileSplit.getLocations().length); - assertEquals(hosts1[0], fileSplit.getLocations()[0]); - } - if (split.equals(splits.get(2))) { - assertEquals(2, fileSplit.getNumPaths()); - assertEquals(1, fileSplit.getLocations().length); - assertEquals(hosts3[0], fileSplit.getLocations()[0]); - } - } else if (splits.size() == 2) { - // If rack1 is processed first - if (split.equals(splits.get(0))) { - assertEquals(2, fileSplit.getNumPaths()); - assertEquals(1, fileSplit.getLocations().length); - assertEquals(hosts1[0], fileSplit.getLocations()[0]); - } - if (split.equals(splits.get(1))) { - assertEquals(2, fileSplit.getNumPaths()); - assertEquals(1, fileSplit.getLocations().length); - assertEquals(hosts3[0], fileSplit.getLocations()[0]); - } - } else { - fail("Split size should be 2 or 3."); - } + assertTrue(fileSplit.getNumPaths() >= 1); + assertTrue(fileSplit.getLocations().length >= 1); + hostsSeen.addAll(Arrays.asList(fileSplit.getLocations())); + for (int i = 0; i < fileSplit.getNumPaths(); i++) { String name = fileSplit.getPath(i).getName(); long length = fileSplit.getLength(i);