You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by "ctubbsii (via GitHub)" <gi...@apache.org> on 2023/05/10 12:37:23 UTC

[GitHub] [accumulo] ctubbsii commented on a diff in pull request #3342: Add RowRange object

ctubbsii commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1189795051


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -1485,12 +1486,22 @@ public Map<String,String> tableIdMap() {
   }
 
   @Override
+  @Deprecated(since = "3.0.0")
   public Text getMaxRow(String tableName, Authorizations auths, Text startRow,
       boolean startInclusive, Text endRow, boolean endInclusive) throws TableNotFoundException {
-    EXISTING_TABLE_NAME.validate(tableName);
+    return getMaxRow(tableName, auths,
+        RowRange.range(startRow, startInclusive, endRow, endInclusive));
+  }

Review Comment:
   Not public API. Just delete. Don't deprecate.



##########
test/src/main/java/org/apache/accumulo/test/NamespacesIT.java:
##########
@@ -1106,7 +1107,7 @@ public void verifyTableOperationsExceptions() throws Exception {
     assertNoTableNoNamespace(() -> ops.getIteratorSetting(tableName, "a", IteratorScope.scan));
     assertNoTableNoNamespace(() -> ops.getLocalityGroups(tableName));
     assertNoTableNoNamespace(
-        () -> ops.getMaxRow(tableName, Authorizations.EMPTY, a, true, z, true));
+        () -> ops.getMaxRow(tableName, Authorizations.EMPTY, RowRange.open(a, z)));

Review Comment:
   Inclusive = true means this should be closed on `[a,z]`, not open.



##########
core/src/test/java/org/apache/accumulo/core/clientImpl/TableOperationsHelperTest.java:
##########
@@ -89,11 +90,17 @@ public Collection<Text> listSplits(String tableName, int maxSplits) {
     }
 
     @Override
+    @Deprecated(since = "3.0.0")
     public Text getMaxRow(String tableName, Authorizations auths, Text startRow,
         boolean startInclusive, Text endRow, boolean endInclusive) {
       return null;
     }

Review Comment:
   Can just delete this method. It's not public API. No need to deprecate.



##########
core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java:
##########
@@ -222,18 +223,35 @@ Locations locate(String tableName, Collection<Range> ranges)
    * Finds the max row within a given range. To find the max row in a table, pass null for start and
    * end row.
    *
+   * @param tableName the table to search
    * @param auths find the max row that can seen with these auths
    * @param startRow row to start looking at, null means -Infinity
    * @param startInclusive determines if the start row is included
    * @param endRow row to stop looking at, null means Infinity
    * @param endInclusive determines if the end row is included
    *
    * @return The max row in the range, or null if there is no visible data in the range.
+   *
+   * @deprecated since 3.0.0, use {@link #getMaxRow(String, Authorizations, RowRange)} instead
    */
+  @Deprecated(since = "3.0.0")
   Text getMaxRow(String tableName, Authorizations auths, Text startRow, boolean startInclusive,
       Text endRow, boolean endInclusive)
       throws TableNotFoundException, AccumuloException, AccumuloSecurityException;

Review Comment:
   Can turn this into a default method:
   
   ```suggestion
     @Deprecated(since = "3.0.0")
     default Text getMaxRow(String tableName, Authorizations auths, Text startRow, boolean startInclusive,
         Text endRow, boolean endInclusive)
         throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
         return getMaxRow(tableName, auths, RowRange.range(startRow, startInclusive, endRow, endInclusive));
     }
   ```



##########
core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSummariesRetriever.java:
##########
@@ -94,7 +94,7 @@ public Collection<Summary> read() throws IOException {
         SummaryReader fileSummary = SummaryReader.load(in.getFileSystem().getConf(), sources[i],
             "source-" + i, summarySelector, factory, cservice);
         SummaryCollection sc = fileSummary
-            .getSummaries(Collections.singletonList(new Gatherer.RowRange(startRow, endRow)));
+            .getSummaries(Collections.singletonList(RowRange.range(startRow, false, endRow, true)));

Review Comment:
   ```suggestion
               .getSummaries(Collections.singletonList(RowRange.openClosed(startRow, endRow)));
   ```



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,635 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.data;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ *
+ * @since 3.0.0
+ */
+public class RowRange implements Comparable<RowRange> {
+
+  private static final Comparator<Text> START_ROW_COMPARATOR =
+      Comparator.nullsFirst(Text::compareTo);
+  private static final Comparator<Text> END_ROW_COMPARATOR = Comparator.nullsLast(Text::compareTo);
+  public static final Comparator<RowRange> ROW_RANGE_COMPARATOR = (r1, r2) -> {
+    if (r1.startRow == null && r2.startRow == null) {
+      return 0;
+    } else if (r1.startRow == null) {
+      return -1;
+    } else if (r2.startRow == null) {
+      return 1;
+    }
+    return r1.compareTo(r2);
+  };
+
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  final private boolean infiniteStartRow;
+  final private boolean infiniteEndRow;
+
+  /**
+   * Creates a range that includes all possible rows.
+   */
+  public static RowRange all() {
+    return range((Text) null, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(Text row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(CharSequence row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");
+    return range(startRow, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");
+    return range(startRow, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows strictly greater than startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange greaterThan(Text startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, false, null, true);
+  }
+
+  /**
+   * Creates a range of rows strictly greater than startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange greaterThan(CharSequence startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, false, null, true);
+  }
+
+  /**
+   * Creates a range of rows greater than or equal to startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange atLeast(Text startRow) {

Review Comment:
   I would rename the `startRow` and `endRow` variables throughout to `lowerBound` and `upperBound` or similar. `lower` and `upper` are better names than `start` and `end`, and the fact that everything is a row is already understood.



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,635 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.data;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ *
+ * @since 3.0.0
+ */
+public class RowRange implements Comparable<RowRange> {
+
+  private static final Comparator<Text> START_ROW_COMPARATOR =
+      Comparator.nullsFirst(Text::compareTo);
+  private static final Comparator<Text> END_ROW_COMPARATOR = Comparator.nullsLast(Text::compareTo);
+  public static final Comparator<RowRange> ROW_RANGE_COMPARATOR = (r1, r2) -> {
+    if (r1.startRow == null && r2.startRow == null) {
+      return 0;
+    } else if (r1.startRow == null) {
+      return -1;
+    } else if (r2.startRow == null) {
+      return 1;
+    }
+    return r1.compareTo(r2);
+  };
+
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  final private boolean infiniteStartRow;
+  final private boolean infiniteEndRow;
+
+  /**
+   * Creates a range that includes all possible rows.
+   */
+  public static RowRange all() {
+    return range((Text) null, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(Text row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(CharSequence row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");
+    return range(startRow, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");
+    return range(startRow, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows strictly greater than startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange greaterThan(Text startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, false, null, true);
+  }
+
+  /**
+   * Creates a range of rows strictly greater than startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange greaterThan(CharSequence startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, false, null, true);
+  }
+
+  /**
+   * Creates a range of rows greater than or equal to startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange atLeast(Text startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows greater than or equal to startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange atLeast(CharSequence startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows strictly less than endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange lessThan(Text endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows strictly less than endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange lessThan(CharSequence endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows less than or equal to endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange atMost(Text endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows less than or equal to endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange atMost(CharSequence endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange range(Text startRow, boolean startInclusive, Text endRow,
+      boolean endInclusive) {
+    return new RowRange(startRow, startInclusive, endRow, endInclusive);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange range(CharSequence startRow, boolean startInclusive, CharSequence endRow,
+      boolean endInclusive) {
+    return new RowRange(startRow == null ? null : new Text(startRow.toString()), startInclusive,
+        endRow == null ? null : new Text(endRow.toString()), endInclusive);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  private RowRange(Text startRow, boolean startInclusive, Text endRow, boolean endInclusive) {
+    this.startRow = startRow;
+    this.startRowInclusive = startInclusive;
+    this.infiniteStartRow = startRow == null;
+    this.endRow = endRow;
+    this.endRowInclusive = endInclusive;
+    this.infiniteEndRow = endRow == null;
+
+    if (!infiniteStartRow && !infiniteEndRow && beforeStartRowImpl(endRow)) {
+      throw new IllegalArgumentException(
+          "Start row must be less than end row in row range (" + startRow + ", " + endRow + ")");
+    }
+  }
+
+  public Text getStartRow() {
+    return startRow;
+  }
+
+  public boolean isStartRowInclusive() {

Review Comment:
   ```suggestion
     public boolean lowerBoundInclusive() {
   ```



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,635 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.data;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ *
+ * @since 3.0.0
+ */
+public class RowRange implements Comparable<RowRange> {
+
+  private static final Comparator<Text> START_ROW_COMPARATOR =
+      Comparator.nullsFirst(Text::compareTo);
+  private static final Comparator<Text> END_ROW_COMPARATOR = Comparator.nullsLast(Text::compareTo);
+  public static final Comparator<RowRange> ROW_RANGE_COMPARATOR = (r1, r2) -> {
+    if (r1.startRow == null && r2.startRow == null) {
+      return 0;
+    } else if (r1.startRow == null) {
+      return -1;
+    } else if (r2.startRow == null) {
+      return 1;
+    }
+    return r1.compareTo(r2);
+  };
+
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  final private boolean infiniteStartRow;
+  final private boolean infiniteEndRow;
+
+  /**
+   * Creates a range that includes all possible rows.
+   */
+  public static RowRange all() {
+    return range((Text) null, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(Text row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(CharSequence row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");
+    return range(startRow, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");
+    return range(startRow, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows strictly greater than startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange greaterThan(Text startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, false, null, true);
+  }
+
+  /**
+   * Creates a range of rows strictly greater than startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange greaterThan(CharSequence startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, false, null, true);
+  }
+
+  /**
+   * Creates a range of rows greater than or equal to startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange atLeast(Text startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows greater than or equal to startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange atLeast(CharSequence startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows strictly less than endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange lessThan(Text endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows strictly less than endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange lessThan(CharSequence endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows less than or equal to endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange atMost(Text endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows less than or equal to endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange atMost(CharSequence endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange range(Text startRow, boolean startInclusive, Text endRow,
+      boolean endInclusive) {
+    return new RowRange(startRow, startInclusive, endRow, endInclusive);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange range(CharSequence startRow, boolean startInclusive, CharSequence endRow,
+      boolean endInclusive) {
+    return new RowRange(startRow == null ? null : new Text(startRow.toString()), startInclusive,
+        endRow == null ? null : new Text(endRow.toString()), endInclusive);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  private RowRange(Text startRow, boolean startInclusive, Text endRow, boolean endInclusive) {
+    this.startRow = startRow;
+    this.startRowInclusive = startInclusive;
+    this.infiniteStartRow = startRow == null;
+    this.endRow = endRow;
+    this.endRowInclusive = endInclusive;
+    this.infiniteEndRow = endRow == null;
+
+    if (!infiniteStartRow && !infiniteEndRow && beforeStartRowImpl(endRow)) {
+      throw new IllegalArgumentException(
+          "Start row must be less than end row in row range (" + startRow + ", " + endRow + ")");
+    }
+  }
+
+  public Text getStartRow() {

Review Comment:
   ```suggestion
     public Text lowerBound() {
   ```



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,635 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.data;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ *
+ * @since 3.0.0
+ */
+public class RowRange implements Comparable<RowRange> {
+
+  private static final Comparator<Text> START_ROW_COMPARATOR =
+      Comparator.nullsFirst(Text::compareTo);
+  private static final Comparator<Text> END_ROW_COMPARATOR = Comparator.nullsLast(Text::compareTo);
+  public static final Comparator<RowRange> ROW_RANGE_COMPARATOR = (r1, r2) -> {
+    if (r1.startRow == null && r2.startRow == null) {
+      return 0;
+    } else if (r1.startRow == null) {
+      return -1;
+    } else if (r2.startRow == null) {
+      return 1;
+    }
+    return r1.compareTo(r2);
+  };
+
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  final private boolean infiniteStartRow;
+  final private boolean infiniteEndRow;
+
+  /**
+   * Creates a range that includes all possible rows.
+   */
+  public static RowRange all() {
+    return range((Text) null, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(Text row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(CharSequence row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");
+    return range(startRow, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");
+    return range(startRow, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows strictly greater than startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange greaterThan(Text startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, false, null, true);
+  }
+
+  /**
+   * Creates a range of rows strictly greater than startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange greaterThan(CharSequence startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, false, null, true);
+  }
+
+  /**
+   * Creates a range of rows greater than or equal to startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange atLeast(Text startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows greater than or equal to startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange atLeast(CharSequence startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows strictly less than endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange lessThan(Text endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows strictly less than endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange lessThan(CharSequence endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows less than or equal to endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange atMost(Text endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows less than or equal to endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange atMost(CharSequence endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange range(Text startRow, boolean startInclusive, Text endRow,
+      boolean endInclusive) {
+    return new RowRange(startRow, startInclusive, endRow, endInclusive);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange range(CharSequence startRow, boolean startInclusive, CharSequence endRow,
+      boolean endInclusive) {
+    return new RowRange(startRow == null ? null : new Text(startRow.toString()), startInclusive,
+        endRow == null ? null : new Text(endRow.toString()), endInclusive);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  private RowRange(Text startRow, boolean startInclusive, Text endRow, boolean endInclusive) {
+    this.startRow = startRow;
+    this.startRowInclusive = startInclusive;
+    this.infiniteStartRow = startRow == null;
+    this.endRow = endRow;
+    this.endRowInclusive = endInclusive;
+    this.infiniteEndRow = endRow == null;
+
+    if (!infiniteStartRow && !infiniteEndRow && beforeStartRowImpl(endRow)) {
+      throw new IllegalArgumentException(
+          "Start row must be less than end row in row range (" + startRow + ", " + endRow + ")");
+    }
+  }
+
+  public Text getStartRow() {
+    return startRow;
+  }
+
+  public boolean isStartRowInclusive() {
+    return startRowInclusive;
+  }
+
+  public Text getEndRow() {
+    return endRow;
+  }
+
+  public boolean isEndRowInclusive() {
+    return endRowInclusive;
+  }
+
+  /**
+   * Converts this row range to a {@link Range} object.
+   */
+  public Range toRange() {
+    return new Range(startRow, startRowInclusive, endRow, endRowInclusive);
+  }
+
+  /**
+   * Determines if the given row is before the start row of this row range.
+   *
+   * @param row row to check
+   * @return true if the given row is before the row range, otherwise false
+   */
+  public boolean beforeStartRow(Text row) {

Review Comment:
   This method reads like "Is `this` before the provided `row`?".
   It feels more natural to treat the current object as the subject, and method parameters as the object of a sentence in human speech.
   
   So, I would suggest something like:
   
   ```suggestion
     public boolean isAfter(Text that) {
   ```
   
   or
   
   ```suggestion
     public boolean startsAfter(Text that) {
   ```
   To ask if `this` range starts after `that` row.
   Alternatively, you could use `greaterThan` and `lessThan`, but I think that might get confusing with the static initializers with similar names.



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,635 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.data;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ *
+ * @since 3.0.0
+ */
+public class RowRange implements Comparable<RowRange> {
+
+  private static final Comparator<Text> START_ROW_COMPARATOR =
+      Comparator.nullsFirst(Text::compareTo);
+  private static final Comparator<Text> END_ROW_COMPARATOR = Comparator.nullsLast(Text::compareTo);
+  public static final Comparator<RowRange> ROW_RANGE_COMPARATOR = (r1, r2) -> {
+    if (r1.startRow == null && r2.startRow == null) {
+      return 0;
+    } else if (r1.startRow == null) {
+      return -1;
+    } else if (r2.startRow == null) {
+      return 1;
+    }
+    return r1.compareTo(r2);
+  };
+
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  final private boolean infiniteStartRow;
+  final private boolean infiniteEndRow;
+
+  /**
+   * Creates a range that includes all possible rows.
+   */
+  public static RowRange all() {
+    return range((Text) null, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(Text row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(CharSequence row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");
+    return range(startRow, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");
+    return range(startRow, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows strictly greater than startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange greaterThan(Text startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, false, null, true);
+  }
+
+  /**
+   * Creates a range of rows strictly greater than startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange greaterThan(CharSequence startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, false, null, true);
+  }
+
+  /**
+   * Creates a range of rows greater than or equal to startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange atLeast(Text startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows greater than or equal to startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange atLeast(CharSequence startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows strictly less than endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange lessThan(Text endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows strictly less than endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange lessThan(CharSequence endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows less than or equal to endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange atMost(Text endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows less than or equal to endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange atMost(CharSequence endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange range(Text startRow, boolean startInclusive, Text endRow,
+      boolean endInclusive) {
+    return new RowRange(startRow, startInclusive, endRow, endInclusive);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange range(CharSequence startRow, boolean startInclusive, CharSequence endRow,
+      boolean endInclusive) {
+    return new RowRange(startRow == null ? null : new Text(startRow.toString()), startInclusive,
+        endRow == null ? null : new Text(endRow.toString()), endInclusive);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  private RowRange(Text startRow, boolean startInclusive, Text endRow, boolean endInclusive) {
+    this.startRow = startRow;
+    this.startRowInclusive = startInclusive;
+    this.infiniteStartRow = startRow == null;
+    this.endRow = endRow;
+    this.endRowInclusive = endInclusive;
+    this.infiniteEndRow = endRow == null;
+
+    if (!infiniteStartRow && !infiniteEndRow && beforeStartRowImpl(endRow)) {
+      throw new IllegalArgumentException(
+          "Start row must be less than end row in row range (" + startRow + ", " + endRow + ")");
+    }
+  }
+
+  public Text getStartRow() {
+    return startRow;
+  }
+
+  public boolean isStartRowInclusive() {
+    return startRowInclusive;

Review Comment:
   Even if you specify false to this, if you have `null` as the start row, then this is irrelevant. It would be good to normalize these, so regardless of which one you put in, you always have this return `true` if `null` is the start  row. Similar comment about the inclusivity of the end row/upper bound.



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,635 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.data;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ *
+ * @since 3.0.0
+ */
+public class RowRange implements Comparable<RowRange> {
+
+  private static final Comparator<Text> START_ROW_COMPARATOR =
+      Comparator.nullsFirst(Text::compareTo);
+  private static final Comparator<Text> END_ROW_COMPARATOR = Comparator.nullsLast(Text::compareTo);
+  public static final Comparator<RowRange> ROW_RANGE_COMPARATOR = (r1, r2) -> {
+    if (r1.startRow == null && r2.startRow == null) {

Review Comment:
   What if r1 or r2 are null? Should this check those, or is it okay to assume non-null for both, and rely on users using `Comparator.nulls{First,Last}(ROW_RANGE_COMPARATOR)`? (And will that work?)



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,635 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.data;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ *
+ * @since 3.0.0
+ */
+public class RowRange implements Comparable<RowRange> {
+
+  private static final Comparator<Text> START_ROW_COMPARATOR =
+      Comparator.nullsFirst(Text::compareTo);
+  private static final Comparator<Text> END_ROW_COMPARATOR = Comparator.nullsLast(Text::compareTo);
+  public static final Comparator<RowRange> ROW_RANGE_COMPARATOR = (r1, r2) -> {
+    if (r1.startRow == null && r2.startRow == null) {
+      return 0;
+    } else if (r1.startRow == null) {
+      return -1;
+    } else if (r2.startRow == null) {
+      return 1;
+    }
+    return r1.compareTo(r2);
+  };
+
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  final private boolean infiniteStartRow;
+  final private boolean infiniteEndRow;
+
+  /**
+   * Creates a range that includes all possible rows.
+   */
+  public static RowRange all() {
+    return range((Text) null, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");

Review Comment:
   A static import on `requireNonNull` would make a lot of the boilerplate throughout this class slightly smaller.



##########
core/src/main/java/org/apache/accumulo/core/client/admin/FindMax.java:
##########
@@ -135,15 +138,17 @@ private static Text _findMax(Scanner scanner, Text start, boolean inclStart, Tex
         return next.getRow();
       }
 
-      Text ret = _findMax(scanner, next.followingKey(PartialKey.ROW).getRow(), true, end, inclEnd);
+      Text ret = _findMax(scanner,
+          RowRange.range(next.followingKey(PartialKey.ROW).getRow(), true, end, inclEnd));
       if (ret == null) {
         return next.getRow();
       } else {
         return ret;
       }
     } else {
 
-      return _findMax(scanner, start, inclStart, mid, mid.equals(start) ? inclStart : false);
+      return _findMax(scanner,
+          RowRange.range(start, inclStart, mid, mid.equals(start) && inclStart));

Review Comment:
   This is fine, but I think the ternary expression is a little more readable than the boolean conjunction.



##########
core/src/main/java/org/apache/accumulo/core/summary/Gatherer.java:
##########
@@ -433,11 +434,13 @@ public interface FileSystemResolver {
   public Future<SummaryCollection> processFiles(FileSystemResolver volMgr,
       Map<String,List<TRowRange>> files, BlockCache summaryCache, BlockCache indexCache,
       Cache<String,Long> fileLenCache, ExecutorService srp) {
+    Function<TRowRange,RowRange> fromThrift =
+        tRowRange -> RowRange.closedOpen(ByteBufferUtil.toText(tRowRange.startRow),
+            ByteBufferUtil.toText(tRowRange.endRow));

Review Comment:
   I think this is supposed to be openClosed, not closedOpen.
   
   ```suggestion
       Function<TRowRange,RowRange> fromThrift =
           tRowRange -> RowRange.openClosed(ByteBufferUtil.toText(tRowRange.startRow),
               ByteBufferUtil.toText(tRowRange.endRow));
   ```



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,635 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.data;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ *
+ * @since 3.0.0
+ */
+public class RowRange implements Comparable<RowRange> {
+
+  private static final Comparator<Text> START_ROW_COMPARATOR =
+      Comparator.nullsFirst(Text::compareTo);
+  private static final Comparator<Text> END_ROW_COMPARATOR = Comparator.nullsLast(Text::compareTo);
+  public static final Comparator<RowRange> ROW_RANGE_COMPARATOR = (r1, r2) -> {
+    if (r1.startRow == null && r2.startRow == null) {
+      return 0;
+    } else if (r1.startRow == null) {
+      return -1;
+    } else if (r2.startRow == null) {
+      return 1;
+    }
+    return r1.compareTo(r2);
+  };
+
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  final private boolean infiniteStartRow;
+  final private boolean infiniteEndRow;
+
+  /**
+   * Creates a range that includes all possible rows.
+   */
+  public static RowRange all() {
+    return range((Text) null, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(Text row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(CharSequence row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");

Review Comment:
   ```suggestion
       Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
   ```



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,635 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.data;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ *
+ * @since 3.0.0
+ */
+public class RowRange implements Comparable<RowRange> {
+
+  private static final Comparator<Text> START_ROW_COMPARATOR =
+      Comparator.nullsFirst(Text::compareTo);
+  private static final Comparator<Text> END_ROW_COMPARATOR = Comparator.nullsLast(Text::compareTo);
+  public static final Comparator<RowRange> ROW_RANGE_COMPARATOR = (r1, r2) -> {
+    if (r1.startRow == null && r2.startRow == null) {
+      return 0;
+    } else if (r1.startRow == null) {
+      return -1;
+    } else if (r2.startRow == null) {
+      return 1;
+    }
+    return r1.compareTo(r2);
+  };
+
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  final private boolean infiniteStartRow;
+  final private boolean infiniteEndRow;
+
+  /**
+   * Creates a range that includes all possible rows.
+   */
+  public static RowRange all() {
+    return range((Text) null, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(Text row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(CharSequence row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");

Review Comment:
   I think this would be:
   ```suggestion
       Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
   ```



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,635 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.data;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ *
+ * @since 3.0.0
+ */
+public class RowRange implements Comparable<RowRange> {
+
+  private static final Comparator<Text> START_ROW_COMPARATOR =
+      Comparator.nullsFirst(Text::compareTo);
+  private static final Comparator<Text> END_ROW_COMPARATOR = Comparator.nullsLast(Text::compareTo);
+  public static final Comparator<RowRange> ROW_RANGE_COMPARATOR = (r1, r2) -> {
+    if (r1.startRow == null && r2.startRow == null) {
+      return 0;
+    } else if (r1.startRow == null) {
+      return -1;
+    } else if (r2.startRow == null) {
+      return 1;
+    }
+    return r1.compareTo(r2);
+  };
+
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  final private boolean infiniteStartRow;
+  final private boolean infiniteEndRow;
+
+  /**
+   * Creates a range that includes all possible rows.
+   */
+  public static RowRange all() {
+    return range((Text) null, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(Text row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(CharSequence row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");

Review Comment:
   ```suggestion
       Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
   ```



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,635 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.data;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ *
+ * @since 3.0.0
+ */
+public class RowRange implements Comparable<RowRange> {
+
+  private static final Comparator<Text> START_ROW_COMPARATOR =
+      Comparator.nullsFirst(Text::compareTo);
+  private static final Comparator<Text> END_ROW_COMPARATOR = Comparator.nullsLast(Text::compareTo);
+  public static final Comparator<RowRange> ROW_RANGE_COMPARATOR = (r1, r2) -> {
+    if (r1.startRow == null && r2.startRow == null) {
+      return 0;
+    } else if (r1.startRow == null) {
+      return -1;
+    } else if (r2.startRow == null) {
+      return 1;
+    }
+    return r1.compareTo(r2);
+  };
+
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  final private boolean infiniteStartRow;
+  final private boolean infiniteEndRow;
+
+  /**
+   * Creates a range that includes all possible rows.
+   */
+  public static RowRange all() {
+    return range((Text) null, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)

Review Comment:
   The javadocs still recommend using `null` for these, even though we explicitly check that they are not null.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -1485,12 +1486,22 @@ public Map<String,String> tableIdMap() {
   }
 
   @Override
+  @Deprecated(since = "3.0.0")
   public Text getMaxRow(String tableName, Authorizations auths, Text startRow,
       boolean startInclusive, Text endRow, boolean endInclusive) throws TableNotFoundException {
-    EXISTING_TABLE_NAME.validate(tableName);
+    return getMaxRow(tableName, auths,
+        RowRange.range(startRow, startInclusive, endRow, endInclusive));
+  }
 
-    Scanner scanner = context.createScanner(tableName, auths);
-    return FindMax.findMax(scanner, startRow, startInclusive, endRow, endInclusive);
+  @Override
+  public Text getMaxRow(String tableName, Authorizations auths, RowRange rowRange)
+      throws TableNotFoundException {
+    EXISTING_TABLE_NAME.validate(tableName);
+    Text maxRow;
+    try (Scanner scanner = context.createScanner(tableName, auths)) {
+      maxRow = FindMax.findMax(scanner, rowRange);
+    }
+    return maxRow;

Review Comment:
   ```suggestion
       try (Scanner scanner = context.createScanner(tableName, auths)) {
         return FindMax.findMax(scanner, rowRange);
       }
   ```



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,635 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.data;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ *
+ * @since 3.0.0
+ */
+public class RowRange implements Comparable<RowRange> {
+
+  private static final Comparator<Text> START_ROW_COMPARATOR =
+      Comparator.nullsFirst(Text::compareTo);
+  private static final Comparator<Text> END_ROW_COMPARATOR = Comparator.nullsLast(Text::compareTo);
+  public static final Comparator<RowRange> ROW_RANGE_COMPARATOR = (r1, r2) -> {
+    if (r1.startRow == null && r2.startRow == null) {
+      return 0;
+    } else if (r1.startRow == null) {
+      return -1;
+    } else if (r2.startRow == null) {
+      return 1;
+    }
+    return r1.compareTo(r2);
+  };
+
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  final private boolean infiniteStartRow;
+  final private boolean infiniteEndRow;
+
+  /**
+   * Creates a range that includes all possible rows.
+   */
+  public static RowRange all() {
+    return range((Text) null, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(Text row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(CharSequence row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");
+    return range(startRow, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");

Review Comment:
   ```suggestion
       Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
   ```



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,635 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.data;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ *
+ * @since 3.0.0
+ */
+public class RowRange implements Comparable<RowRange> {
+
+  private static final Comparator<Text> START_ROW_COMPARATOR =
+      Comparator.nullsFirst(Text::compareTo);
+  private static final Comparator<Text> END_ROW_COMPARATOR = Comparator.nullsLast(Text::compareTo);
+  public static final Comparator<RowRange> ROW_RANGE_COMPARATOR = (r1, r2) -> {
+    if (r1.startRow == null && r2.startRow == null) {
+      return 0;
+    } else if (r1.startRow == null) {
+      return -1;
+    } else if (r2.startRow == null) {
+      return 1;
+    }
+    return r1.compareTo(r2);
+  };
+
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  final private boolean infiniteStartRow;
+  final private boolean infiniteEndRow;
+
+  /**
+   * Creates a range that includes all possible rows.
+   */
+  public static RowRange all() {
+    return range((Text) null, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(Text row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(CharSequence row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");
+    return range(startRow, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");
+    return range(startRow, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows strictly greater than startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange greaterThan(Text startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, false, null, true);
+  }
+
+  /**
+   * Creates a range of rows strictly greater than startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange greaterThan(CharSequence startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, false, null, true);
+  }
+
+  /**
+   * Creates a range of rows greater than or equal to startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange atLeast(Text startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows greater than or equal to startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange atLeast(CharSequence startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows strictly less than endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange lessThan(Text endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows strictly less than endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange lessThan(CharSequence endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows less than or equal to endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange atMost(Text endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows less than or equal to endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange atMost(CharSequence endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange range(Text startRow, boolean startInclusive, Text endRow,
+      boolean endInclusive) {
+    return new RowRange(startRow, startInclusive, endRow, endInclusive);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange range(CharSequence startRow, boolean startInclusive, CharSequence endRow,
+      boolean endInclusive) {
+    return new RowRange(startRow == null ? null : new Text(startRow.toString()), startInclusive,
+        endRow == null ? null : new Text(endRow.toString()), endInclusive);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  private RowRange(Text startRow, boolean startInclusive, Text endRow, boolean endInclusive) {
+    this.startRow = startRow;
+    this.startRowInclusive = startInclusive;
+    this.infiniteStartRow = startRow == null;
+    this.endRow = endRow;
+    this.endRowInclusive = endInclusive;
+    this.infiniteEndRow = endRow == null;
+
+    if (!infiniteStartRow && !infiniteEndRow && beforeStartRowImpl(endRow)) {
+      throw new IllegalArgumentException(
+          "Start row must be less than end row in row range (" + startRow + ", " + endRow + ")");
+    }
+  }
+
+  public Text getStartRow() {
+    return startRow;
+  }
+
+  public boolean isStartRowInclusive() {
+    return startRowInclusive;
+  }
+
+  public Text getEndRow() {

Review Comment:
   ```suggestion
     public Text upperBound() {
   ```



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,635 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.data;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ *
+ * @since 3.0.0
+ */
+public class RowRange implements Comparable<RowRange> {
+
+  private static final Comparator<Text> START_ROW_COMPARATOR =
+      Comparator.nullsFirst(Text::compareTo);
+  private static final Comparator<Text> END_ROW_COMPARATOR = Comparator.nullsLast(Text::compareTo);
+  public static final Comparator<RowRange> ROW_RANGE_COMPARATOR = (r1, r2) -> {
+    if (r1.startRow == null && r2.startRow == null) {
+      return 0;
+    } else if (r1.startRow == null) {
+      return -1;
+    } else if (r2.startRow == null) {
+      return 1;
+    }
+    return r1.compareTo(r2);
+  };
+
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  final private boolean infiniteStartRow;
+  final private boolean infiniteEndRow;
+
+  /**
+   * Creates a range that includes all possible rows.
+   */
+  public static RowRange all() {
+    return range((Text) null, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(Text row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(CharSequence row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");
+    return range(startRow, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");
+    return range(startRow, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows strictly greater than startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange greaterThan(Text startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, false, null, true);
+  }
+
+  /**
+   * Creates a range of rows strictly greater than startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange greaterThan(CharSequence startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, false, null, true);
+  }
+
+  /**
+   * Creates a range of rows greater than or equal to startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange atLeast(Text startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows greater than or equal to startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange atLeast(CharSequence startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows strictly less than endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange lessThan(Text endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows strictly less than endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange lessThan(CharSequence endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows less than or equal to endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange atMost(Text endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows less than or equal to endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange atMost(CharSequence endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange range(Text startRow, boolean startInclusive, Text endRow,
+      boolean endInclusive) {
+    return new RowRange(startRow, startInclusive, endRow, endInclusive);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange range(CharSequence startRow, boolean startInclusive, CharSequence endRow,
+      boolean endInclusive) {
+    return new RowRange(startRow == null ? null : new Text(startRow.toString()), startInclusive,
+        endRow == null ? null : new Text(endRow.toString()), endInclusive);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  private RowRange(Text startRow, boolean startInclusive, Text endRow, boolean endInclusive) {
+    this.startRow = startRow;
+    this.startRowInclusive = startInclusive;
+    this.infiniteStartRow = startRow == null;
+    this.endRow = endRow;
+    this.endRowInclusive = endInclusive;
+    this.infiniteEndRow = endRow == null;
+
+    if (!infiniteStartRow && !infiniteEndRow && beforeStartRowImpl(endRow)) {
+      throw new IllegalArgumentException(
+          "Start row must be less than end row in row range (" + startRow + ", " + endRow + ")");
+    }
+  }
+
+  public Text getStartRow() {
+    return startRow;
+  }
+
+  public boolean isStartRowInclusive() {
+    return startRowInclusive;
+  }
+
+  public Text getEndRow() {
+    return endRow;
+  }
+
+  public boolean isEndRowInclusive() {
+    return endRowInclusive;
+  }
+
+  /**
+   * Converts this row range to a {@link Range} object.
+   */
+  public Range toRange() {

Review Comment:
   I think `asRange` is a more natural API than `toRange`. `asRange` allows you to read it like a human: "show me this RowRange as a Range", whereas `toRange` feels more robotic: "convert this RowRange to a Range".
   
   Actually, I prefer `asKeyRange`, because I think our other `Range` should be called `KeyRange`, but I don't think we should use that yet, because return types are not part of the method signature, so we can't change the return type later, if we decide to deprecate Range in favor of a KeyRange class that looks like this RowRange class.
   
   ```suggestion
     public Range asRange() {
   ```



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,635 @@
+/*
+ * 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
+ *
+ *   https://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.accumulo.core.data;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hadoop.io.BinaryComparable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ *
+ * @since 3.0.0
+ */
+public class RowRange implements Comparable<RowRange> {
+
+  private static final Comparator<Text> START_ROW_COMPARATOR =
+      Comparator.nullsFirst(Text::compareTo);
+  private static final Comparator<Text> END_ROW_COMPARATOR = Comparator.nullsLast(Text::compareTo);
+  public static final Comparator<RowRange> ROW_RANGE_COMPARATOR = (r1, r2) -> {
+    if (r1.startRow == null && r2.startRow == null) {
+      return 0;
+    } else if (r1.startRow == null) {
+      return -1;
+    } else if (r2.startRow == null) {
+      return 1;
+    }
+    return r1.compareTo(r2);
+  };
+
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  final private boolean infiniteStartRow;
+  final private boolean infiniteEndRow;
+
+  /**
+   * Creates a range that includes all possible rows.
+   */
+  public static RowRange all() {
+    return range((Text) null, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange open(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.greaterThan(row)?");
+    return range(startRow, false, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, true, endRow, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(Text row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range that covers an entire row.
+   *
+   * @param row row to cover; set to null to cover all rows
+   */
+  public static RowRange closed(CharSequence row) {
+    Objects.requireNonNull(row, "Did you mean to use RowRange.all()?");
+    return range(row, true, row, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow exclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange openClosed(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.atMost(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.atLeast(row)?");
+    return range(startRow, false, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(Text startRow, Text endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");
+    return range(startRow, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow exclusive.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param endRow ending row; set to null for positive infinity
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange closedOpen(CharSequence startRow, CharSequence endRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.lessThan(row)?");
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.moreThan(row)?");
+    return range(startRow, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows strictly greater than startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange greaterThan(Text startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, false, null, true);
+  }
+
+  /**
+   * Creates a range of rows strictly greater than startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange greaterThan(CharSequence startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, false, null, true);
+  }
+
+  /**
+   * Creates a range of rows greater than or equal to startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange atLeast(Text startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows greater than or equal to startRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   */
+  public static RowRange atLeast(CharSequence startRow) {
+    Objects.requireNonNull(startRow, "Did you mean to use RowRange.all()?");
+    return range(startRow, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows strictly less than endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange lessThan(Text endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows strictly less than endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange lessThan(CharSequence endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, false);
+  }
+
+  /**
+   * Creates a range of rows less than or equal to endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange atMost(Text endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows less than or equal to endRow.
+   *
+   * @param endRow ending row; set to null for positive infinity
+   */
+  public static RowRange atMost(CharSequence endRow) {
+    Objects.requireNonNull(endRow, "Did you mean to use RowRange.all()?");
+    return range(null, true, endRow, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange range(Text startRow, boolean startInclusive, Text endRow,
+      boolean endInclusive) {
+    return new RowRange(startRow, startInclusive, endRow, endInclusive);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  public static RowRange range(CharSequence startRow, boolean startInclusive, CharSequence endRow,
+      boolean endInclusive) {
+    return new RowRange(startRow == null ? null : new Text(startRow.toString()), startInclusive,
+        endRow == null ? null : new Text(endRow.toString()), endInclusive);
+  }
+
+  /**
+   * Creates a range of rows from startRow to endRow.
+   *
+   * @param startRow starting row; set to null for the smallest possible row (an empty one)
+   * @param startInclusive true to include start row, false to skip
+   * @param endRow ending row; set to null for positive infinity
+   * @param endInclusive true to include end row, false to skip
+   * @throws IllegalArgumentException if end row is before start row
+   */
+  private RowRange(Text startRow, boolean startInclusive, Text endRow, boolean endInclusive) {
+    this.startRow = startRow;
+    this.startRowInclusive = startInclusive;
+    this.infiniteStartRow = startRow == null;
+    this.endRow = endRow;
+    this.endRowInclusive = endInclusive;
+    this.infiniteEndRow = endRow == null;
+
+    if (!infiniteStartRow && !infiniteEndRow && beforeStartRowImpl(endRow)) {
+      throw new IllegalArgumentException(
+          "Start row must be less than end row in row range (" + startRow + ", " + endRow + ")");
+    }
+  }
+
+  public Text getStartRow() {
+    return startRow;
+  }
+
+  public boolean isStartRowInclusive() {
+    return startRowInclusive;
+  }
+
+  public Text getEndRow() {
+    return endRow;
+  }
+
+  public boolean isEndRowInclusive() {

Review Comment:
   ```suggestion
     public boolean upperBoundInclusive() {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org