瀏覽代碼

HADOOP-19253. Google GCS compilation fails due to VectorIO changes (#7002)

Fixes a compilation failure ncaused by HADOOP-19098

Restore original sortRanges() method signature,
  FileRange[] sortRanges(List<? extends FileRange>)

This ensures that google GCS connector will compile again.
It has also been marked as Stable so it is left alone

The version returning List<? extends FileRange>
has been renamed sortRangeList()

Contributed by Steve Loughran
Steve Loughran 8 月之前
父節點
當前提交
75845e6685

+ 15 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/VectoredReadUtils.java

@@ -308,7 +308,7 @@ public final class VectoredReadUtils {
       validateRangeRequest(input.get(0));
       sortedRanges = input;
     } else {
-      sortedRanges = sortRanges(input);
+      sortedRanges = sortRangeList(input);
       FileRange prev = null;
       for (final FileRange current : sortedRanges) {
         validateRangeRequest(current);
@@ -341,12 +341,25 @@ public final class VectoredReadUtils {
    * @param input input ranges.
    * @return a new list of the ranges, sorted by offset.
    */
-  public static List<? extends FileRange> sortRanges(List<? extends FileRange> input) {
+  public static List<? extends FileRange> sortRangeList(List<? extends FileRange> input) {
     final List<? extends FileRange> l = new ArrayList<>(input);
     l.sort(Comparator.comparingLong(FileRange::getOffset));
     return l;
   }
 
+  /**
+   * Sort the input ranges by offset; no validation is done.
+   * <p>
+   * This method is used externally and must be retained with
+   * the signature unchanged.
+   * @param input input ranges.
+   * @return a new list of the ranges, sorted by offset.
+   */
+  @InterfaceStability.Stable
+  public static FileRange[] sortRanges(List<? extends FileRange> input) {
+    return sortRangeList(input).toArray(new FileRange[0]);
+  }
+
   /**
    * Merge sorted ranges to optimize the access from the underlying file
    * system.

+ 29 - 6
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/TestVectoredReadUtils.java

@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.IntBuffer;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.List;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
@@ -47,6 +48,7 @@ import static org.apache.hadoop.fs.VectoredReadUtils.isOrderedDisjoint;
 import static org.apache.hadoop.fs.VectoredReadUtils.mergeSortedRanges;
 import static org.apache.hadoop.fs.VectoredReadUtils.readRangeFrom;
 import static org.apache.hadoop.fs.VectoredReadUtils.readVectored;
+import static org.apache.hadoop.fs.VectoredReadUtils.sortRangeList;
 import static org.apache.hadoop.fs.VectoredReadUtils.sortRanges;
 import static org.apache.hadoop.fs.VectoredReadUtils.validateAndSortRanges;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -196,7 +198,7 @@ public class TestVectoredReadUtils extends HadoopTestBase {
         );
     assertIsNotOrderedDisjoint(input, 100, 800);
     final List<CombinedFileRange> outputList = mergeSortedRanges(
-            sortRanges(input), 100, 1001, 2500);
+            sortRangeList(input), 100, 1001, 2500);
 
     assertRangeListSize(outputList, 1);
     CombinedFileRange output = outputList.get(0);
@@ -208,7 +210,7 @@ public class TestVectoredReadUtils extends HadoopTestBase {
     // the minSeek doesn't allow the first two to merge
     assertIsNotOrderedDisjoint(input, 100, 100);
     final List<CombinedFileRange> list2 = mergeSortedRanges(
-        sortRanges(input),
+        sortRangeList(input),
             100, 1000, 2100);
     assertRangeListSize(list2, 2);
     assertRangeElement(list2, 0, 1000, 100);
@@ -219,7 +221,7 @@ public class TestVectoredReadUtils extends HadoopTestBase {
     // the maxSize doesn't allow the third range to merge
     assertIsNotOrderedDisjoint(input, 100, 800);
     final List<CombinedFileRange> list3 = mergeSortedRanges(
-        sortRanges(input),
+        sortRangeList(input),
             100, 1001, 2099);
     assertRangeListSize(list3, 2);
     CombinedFileRange range0 = list3.get(0);
@@ -240,7 +242,7 @@ public class TestVectoredReadUtils extends HadoopTestBase {
     // test the round up and round down (the maxSize doesn't allow any merges)
     assertIsNotOrderedDisjoint(input, 16, 700);
     final List<CombinedFileRange> list4 = mergeSortedRanges(
-        sortRanges(input),
+        sortRangeList(input),
         16, 1001, 100);
     assertRangeListSize(list4, 3);
     // range[992,1104)
@@ -273,6 +275,27 @@ public class TestVectoredReadUtils extends HadoopTestBase {
         .isEqualTo(length);
   }
 
+  /**
+   * Verify that {@link VectoredReadUtils#sortRanges(List)}
+   * returns an array matching the list sort ranges.
+   */
+  @Test
+  public void testArraySortRange() throws Throwable {
+    List<FileRange> input = asList(
+        createFileRange(3000, 100, "1"),
+        createFileRange(2100, 100, null),
+        createFileRange(1000, 100, "3")
+        );
+    final FileRange[] rangeArray = sortRanges(input);
+    final List<? extends FileRange> rangeList = sortRangeList(input);
+    Assertions.assertThat(rangeArray)
+        .describedAs("range array from sortRanges()")
+        .isSortedAccordingTo(Comparator.comparingLong(FileRange::getOffset));
+    Assertions.assertThat(rangeList.toArray(new FileRange[0]))
+        .describedAs("range from sortRangeList()")
+        .isEqualTo(rangeArray);
+  }
+
   /**
    * Assert that a file range satisfies the conditions.
    * @param range range to validate
@@ -399,7 +422,7 @@ public class TestVectoredReadUtils extends HadoopTestBase {
     );
     assertIsNotOrderedDisjoint(input, 100, 800);
     List<CombinedFileRange> outputList = mergeSortedRanges(
-            sortRanges(input), 1, 1001, 2500);
+            sortRangeList(input), 1, 1001, 2500);
     Assertions.assertThat(outputList)
             .describedAs("merged range size")
             .hasSize(1);
@@ -411,7 +434,7 @@ public class TestVectoredReadUtils extends HadoopTestBase {
     assertOrderedDisjoint(outputList, 1, 800);
 
     outputList = mergeSortedRanges(
-            sortRanges(input), 100, 1001, 2500);
+            sortRangeList(input), 100, 1001, 2500);
     assertRangeListSize(outputList, 1);
 
     output = outputList.get(0);