You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by "DomGarguilo (via GitHub)" <gi...@apache.org> on 2023/04/25 20:24:54 UTC

[GitHub] [accumulo] DomGarguilo opened a new pull request, #3342: WIP - Add RowRange object

DomGarguilo opened a new pull request, #3342:
URL: https://github.com/apache/accumulo/pull/3342

   Closes #3308
   
   This PR adds a RowRange object to the api.


-- 
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


[GitHub] [accumulo] keith-turner commented on a diff in pull request #3342: WIP - Add RowRange object

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1185467157


##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.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> {
+  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) {
+    return beforeStartRowImpl(row);
+  }
+
+  /**
+   * Determines if the given row is after the end row of this row range.
+   *
+   * @param row row to check
+   * @return true if the given row is after the row range, otherwise false
+   */
+  public boolean afterEndRow(Text row) {
+    if (infiniteEndRow) {
+      return false;
+    }
+
+    if (endRowInclusive) {
+      return row.compareTo(endRow) > 0;
+    }
+    return row.compareTo(endRow) >= 0;
+  }
+
+  /**
+   * Implements logic of {@link #beforeStartRow(Text)}, but in a private method, so that it can be
+   * safely used by constructors if a subclass overrides that {@link #beforeStartRow(Text)}
+   */
+  private boolean beforeStartRowImpl(Text row) {
+    if (this.infiniteStartRow) {
+      return false;
+    }
+
+    if (startRowInclusive) {
+      return row.compareTo(startRow) < 0;
+    }
+    return row.compareTo(startRow) <= 0;
+  }
+
+  @Override
+  public String toString() {
+    final String startRangeSymbol = (startRowInclusive && startRow != null) ? "[" : "(";
+    final String startRow = this.startRow == null ? "-inf" : this.startRow.toString();
+    final String endRow = this.endRow == null ? "+inf" : this.endRow.toString();
+    final String endRangeSymbol = (endRowInclusive && this.endRow != null) ? "]" : ")";
+    return startRangeSymbol + startRow + "," + endRow + endRangeSymbol;
+  }
+
+  @Override
+  public int hashCode() {
+    int startHash = infiniteStartRow ? 0 : startRow.hashCode() + (startRowInclusive ? 1 : 0);
+    int stopHash = infiniteEndRow ? 0 : endRow.hashCode() + (endRowInclusive ? 1 : 0);
+
+    return startHash + stopHash;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof RowRange)) {
+      return false;
+    }
+    return equals((RowRange) other);
+  }
+
+  /**
+   * Determines if this row range equals another.
+   *
+   * @param other range to compare
+   * @return true if row ranges are equals, false otherwise
+   */
+  public boolean equals(RowRange other) {
+    return compareTo(other) == 0;
+  }
+
+  /**
+   * Compares this row range to another row range. Compares in order: start row, inclusiveness of
+   * start row, end row, inclusiveness of end row. Infinite rows sort first, and non-infinite rows
+   * are compared with {@link Text#compareTo(BinaryComparable)}. Inclusive sorts before
+   * non-inclusive.
+   *
+   * @param other range row to compare
+   * @return comparison result
+   */
+  @Override
+  public int compareTo(RowRange other) {
+    // Compare infinite start rows
+    int comp = Boolean.compare(this.infiniteStartRow, other.infiniteStartRow);
+
+    if (comp == 0) {
+      // Compare non-infinite start rows and start row inclusiveness
+      if (!this.infiniteStartRow) {
+        if (this.startRow == null && other.startRow == null) {
+          comp = 0;
+        } else if (this.startRow == null) {
+          comp = -1;
+        } else if (other.startRow == null) {
+          comp = 1;
+        } else {
+          comp = this.startRow.compareTo(other.startRow);
+        }
+        if (comp == 0) {
+          comp = Boolean.compare(other.startRowInclusive, this.startRowInclusive);
+        }
+      }
+    }
+
+    if (comp == 0) {
+      // Compare infinite end rows
+      comp = Boolean.compare(this.infiniteEndRow, other.infiniteEndRow);
+
+      // Compare non-infinite end rows and end row inclusiveness
+      if (comp == 0 && !this.infiniteEndRow) {
+        if (this.endRow == null && other.endRow == null) {
+          comp = 0;
+        } else if (this.endRow == null) {
+          comp = 1;
+        } else if (other.endRow == null) {
+          comp = -1;
+        } else {
+          comp = this.endRow.compareTo(other.endRow);
+        }
+        if (comp == 0) {
+          comp = Boolean.compare(this.endRowInclusive, other.endRowInclusive);
+        }
+      }
+    }
+
+    return comp;
+  }
+
+  /**
+   * Determines if this row range contains the given row.
+   *
+   * @param row row to check
+   * @return true if the row is contained in the row range, false otherwise
+   */
+  public boolean contains(Text row) {
+    if (infiniteStartRow) {
+      return !afterEndRow(row);
+    } else if (infiniteEndRow) {
+      return !beforeStartRow(row);
+    } else {
+      return !beforeStartRow(row) && !afterEndRow(row);
+    }
+  }
+
+  /**
+   * Merges overlapping and adjacent row ranges. For example, given the following input:
+   *
+   * <pre>
+   * [a,c], (c, d], (g,m), (j,t]
+   * </pre>
+   *
+   * the following row ranges would be returned:
+   *
+   * <pre>
+   * [a,d], (g,t]
+   * </pre>
+   *
+   * @param rowRanges the collection of row ranges to merge
+   * @return a list of merged row ranges
+   */
+  public static List<RowRange> mergeOverlapping(Collection<RowRange> rowRanges) {
+    if (rowRanges.isEmpty()) {
+      return Collections.emptyList();
+    }
+    if (rowRanges.size() == 1) {
+      return Collections.singletonList(rowRanges.iterator().next());
+    }
+
+    List<RowRange> sortedRowRanges = new ArrayList<>(rowRanges);
+    // Sort row ranges by their startRow values
+    sortedRowRanges.sort((r1, r2) -> {
+      if (r1.startRow == null && r2.startRow == null) {

Review Comment:
   Must be doing something slightly different.  I thought it would do the same things as the code it was replacing, but I guess not.



-- 
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


[GitHub] [accumulo] keith-turner commented on a diff in pull request #3342: WIP - Add RowRange object

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1180637048


##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import org.apache.accumulo.core.dataImpl.thrift.TRowRange;
+import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.core.util.TextUtil;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ */
+public class RowRange {
+  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 create(null, true, null, 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 open(Text startRow, Text endRow) {
+    return create(startRow, true, endRow, 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 closed(Text startRow, Text endRow) {
+    return create(startRow, false, 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 openClosed(Text startRow, Text endRow) {
+    return create(startRow, true, endRow, false);
+  }
+
+  /**
+   * 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 closedOpen(Text startRow, Text endRow) {
+    return create(startRow, false, endRow, 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(Text startRow) {
+    return create(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) {
+    return create(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) {
+    return create(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) {
+    return create(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 create(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
+   */
+  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 && beforeStartKeyImpl(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 Thrift.
+   *
+   * @return Thrift row range
+   */
+  public TRowRange toThrift() {
+    final ByteBuffer startRow = TextUtil.getByteBuffer(this.startRow);
+    final ByteBuffer endRow = TextUtil.getByteBuffer(this.endRow);
+    return new TRowRange(startRow, endRow);
+  }

Review Comment:
   These method with thrift types should be placed somewhere outside of the public API.  Some of the Accumulo data classes have thrift types because its unavoidable because of the need to access private data, but that is not the case here.  Seems like conversion to and from thrift type can be accomplished using the public methods of this class.



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import org.apache.accumulo.core.dataImpl.thrift.TRowRange;
+import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.core.util.TextUtil;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.

Review Comment:
   Needs a since tag



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import org.apache.accumulo.core.dataImpl.thrift.TRowRange;
+import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.core.util.TextUtil;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ */
+public class RowRange {

Review Comment:
   The following methods would be useful
   
    Range toRange()
    boolean contains(Text row)
   boolean after(Text row)



-- 
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


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

Posted by "DomGarguilo (via GitHub)" <gi...@apache.org>.
DomGarguilo commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1191314724


##########
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:
   I think this should be private actually. I just moved it to a constant so that it doesn't have to be re-created every time `mergeOverlapping` is called.



-- 
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


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

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#issuecomment-1545191860

   It occurs to me that this entire class is basically just Guava's Range with Hadoop's Text as the generic parameter, `Range<Text>`. I don't think we should force users to use Guava's API, but I was thinking if we only cared about using this internally, then we might be able to just use that. I still think there's value in making this available for public API, though, without relying on Guava types in the public API, but this might take the pressure off, if there was any, to get it into the public API to address the internal uses.
   
   This is also another variant of KeyExtent, Tablet, and all those other internal types that maintain a span of row keys, of which there are already several, with slightly different purposes. Then again, making this stable public API might help simplify those others.
   
   Just a thought. I'll probably get a chance to re-review this next week.


-- 
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


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

Posted by "DomGarguilo (via GitHub)" <gi...@apache.org>.
DomGarguilo commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1178237175


##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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 org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ */
+public class RowRange {
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  private final boolean infiniteStartRow;
+  private final boolean infiniteEndRow;
+
+  /**
+   * Creates a range of rows that goes from negative to positive infinity
+   */

Review Comment:
   @ctubbsii do you think the `@param` descriptions should be changed as well? They are currently:
   
   `@param startRow starting row; set to null for negative infinity`



-- 
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


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

Posted by "DomGarguilo (via GitHub)" <gi...@apache.org>.
DomGarguilo commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1180668051


##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -45,8 +45,91 @@ public RowRange() {
    * @param endRow ending row; set to null for positive infinity
    * @throws IllegalArgumentException if end row is before start row
    */
-  public RowRange(Text startRow, Text endRow) {
-    this(startRow, true, endRow, true);
+  public static RowRange open(Text startRow, Text endRow) {
+    return create(startRow, true, endRow, 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 closed(Text startRow, Text endRow) {
+    return create(startRow, false, endRow, false);
+  }

Review Comment:
   Addressed in a527552 but please double check I got things correct



-- 
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


[GitHub] [accumulo] ctubbsii commented on pull request #3342: WIP - Add RowRange object

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#issuecomment-1532428242

   Haven't re-reviewed yet, but just thought of a suggestion (I don't know if it's workable). Internally, this could use Guava Range classes. As long as they're not exposed to this class' API, it might make some implementation easier (perhaps not, though... I'm not sure).


-- 
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


[GitHub] [accumulo] keith-turner commented on pull request #3342: WIP - Add RowRange object

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#issuecomment-1533617136

   @DomGarguilo  feel free to ignore the coverage comments, was just commenting on what I saw.  I think the most important to get coverage on are the clip and merge functions as those are the most complex.


-- 
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


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

Posted by "DomGarguilo (via GitHub)" <gi...@apache.org>.
DomGarguilo commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1178389127


##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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 org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ */
+public class RowRange {
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  private final boolean infiniteStartRow;
+  private final boolean infiniteEndRow;
+
+  /**
+   * Creates a range of rows that goes from negative to positive infinity
+   */
+  public RowRange() {
+    this(null, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for negative infinity

Review Comment:
   Added the static entry points in 2d63f6f



-- 
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


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

Posted by "DomGarguilo (via GitHub)" <gi...@apache.org>.
DomGarguilo commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1178254609


##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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 org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ */
+public class RowRange {
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  private final boolean infiniteStartRow;
+  private final boolean infiniteEndRow;
+
+  /**
+   * Creates a range of rows that goes from negative to positive infinity
+   */

Review Comment:
   Addressed in 46db626



-- 
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


[GitHub] [accumulo] DomGarguilo commented on pull request #3342: Add RowRange object

Posted by "DomGarguilo (via GitHub)" <gi...@apache.org>.
DomGarguilo commented on PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#issuecomment-1538645691

   > Looking at this it would be really nice if [Range](https://github.com/apache/accumulo/blob/56d49f15a05db9a46dbceb845918497760601c11/core/src/main/java/org/apache/accumulo/core/data/Range.java#L39) and `RowRange` could share code. Two ideas I had:
   > 
   > 1. I am wondering if it makes more sense to refactor `Range` to extend `RowRange`. At first glance it seems like the two should be able to share a lot of code as the main difference is comparing Text start/end row vs Key start/end otherwise the core methods/logic for the Range should be pretty similar. But again, I haven't looked in detail or thought through it all the way so maybe there are other reasons to keep it separate.
   > 2. Another option, which I like but may not work if we consider Range public API, is to refactor things and rename the existing Range object to KeyRange and have a more generic abstract Range object and then have `RowRange` and `KeyRange` extend that and share code. Having two descriptive objects of `RowRange` and `KeyRange` and have them share a common abstract class would be quite nice.
   
   It definitely seems like there should be some refactoring to share code. Of these two ideas the first seems like it would be a lot less disruptive as the Range API would not need to change. But the second idea seems maybe more complete(?) or optimal(?) (maybe how things would be done if creating both range and rowrange from scratch at this point) but more disruptive due to changing of the existing API. 


-- 
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


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

Posted by "DomGarguilo (via GitHub)" <gi...@apache.org>.
DomGarguilo commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1178254871


##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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 org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ */
+public class RowRange {
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  private final boolean infiniteStartRow;
+  private final boolean infiniteEndRow;
+
+  /**
+   * Creates a range of rows that goes from negative to positive infinity
+   */
+  public RowRange() {
+    this(null, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for negative infinity

Review Comment:
   Addressed the javadoc fix in 46db626



-- 
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


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

Posted by "DomGarguilo (via GitHub)" <gi...@apache.org>.
DomGarguilo commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1180750901


##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import org.apache.accumulo.core.dataImpl.thrift.TRowRange;
+import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.core.util.TextUtil;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ */
+public class RowRange {
+  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 create(null, true, null, 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 open(Text startRow, Text endRow) {
+    return create(startRow, true, endRow, 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 closed(Text startRow, Text endRow) {
+    return create(startRow, false, 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 openClosed(Text startRow, Text endRow) {
+    return create(startRow, true, endRow, false);
+  }
+
+  /**
+   * 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 closedOpen(Text startRow, Text endRow) {
+    return create(startRow, false, endRow, 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(Text startRow) {
+    return create(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) {
+    return create(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) {
+    return create(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) {
+    return create(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 create(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
+   */
+  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 && beforeStartKeyImpl(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 Thrift.
+   *
+   * @return Thrift row range
+   */
+  public TRowRange toThrift() {
+    final ByteBuffer startRow = TextUtil.getByteBuffer(this.startRow);
+    final ByteBuffer endRow = TextUtil.getByteBuffer(this.endRow);
+    return new TRowRange(startRow, endRow);
+  }

Review Comment:
   Addressed in dc894f9



-- 
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


[GitHub] [accumulo] keith-turner commented on a diff in pull request #3342: WIP - Add RowRange object

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1184133395


##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.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> {
+  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) {

Review Comment:
   This method was not covered by unit test.



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.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> {
+  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()?");

Review Comment:
   This method was not covered by unit test.



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.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> {
+  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) {
+    return beforeStartRowImpl(row);
+  }
+
+  /**
+   * Determines if the given row is after the end row of this row range.
+   *
+   * @param row row to check
+   * @return true if the given row is after the row range, otherwise false
+   */
+  public boolean afterEndRow(Text row) {
+    if (infiniteEndRow) {
+      return false;
+    }
+
+    if (endRowInclusive) {
+      return row.compareTo(endRow) > 0;
+    }
+    return row.compareTo(endRow) >= 0;
+  }
+
+  /**
+   * Implements logic of {@link #beforeStartRow(Text)}, but in a private method, so that it can be
+   * safely used by constructors if a subclass overrides that {@link #beforeStartRow(Text)}
+   */
+  private boolean beforeStartRowImpl(Text row) {
+    if (this.infiniteStartRow) {
+      return false;
+    }
+
+    if (startRowInclusive) {
+      return row.compareTo(startRow) < 0;
+    }
+    return row.compareTo(startRow) <= 0;
+  }
+
+  @Override
+  public String toString() {
+    final String startRangeSymbol = (startRowInclusive && startRow != null) ? "[" : "(";
+    final String startRow = this.startRow == null ? "-inf" : this.startRow.toString();
+    final String endRow = this.endRow == null ? "+inf" : this.endRow.toString();
+    final String endRangeSymbol = (endRowInclusive && this.endRow != null) ? "]" : ")";
+    return startRangeSymbol + startRow + "," + endRow + endRangeSymbol;
+  }
+
+  @Override
+  public int hashCode() {
+    int startHash = infiniteStartRow ? 0 : startRow.hashCode() + (startRowInclusive ? 1 : 0);
+    int stopHash = infiniteEndRow ? 0 : endRow.hashCode() + (endRowInclusive ? 1 : 0);
+
+    return startHash + stopHash;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof RowRange)) {
+      return false;
+    }
+    return equals((RowRange) other);
+  }
+
+  /**
+   * Determines if this row range equals another.
+   *
+   * @param other range to compare
+   * @return true if row ranges are equals, false otherwise
+   */
+  public boolean equals(RowRange other) {
+    return compareTo(other) == 0;
+  }
+
+  /**
+   * Compares this row range to another row range. Compares in order: start row, inclusiveness of
+   * start row, end row, inclusiveness of end row. Infinite rows sort first, and non-infinite rows
+   * are compared with {@link Text#compareTo(BinaryComparable)}. Inclusive sorts before
+   * non-inclusive.
+   *
+   * @param other range row to compare
+   * @return comparison result
+   */
+  @Override
+  public int compareTo(RowRange other) {
+    // Compare infinite start rows
+    int comp = Boolean.compare(this.infiniteStartRow, other.infiniteStartRow);
+
+    if (comp == 0) {
+      // Compare non-infinite start rows and start row inclusiveness
+      if (!this.infiniteStartRow) {
+        if (this.startRow == null && other.startRow == null) {
+          comp = 0;
+        } else if (this.startRow == null) {
+          comp = -1;
+        } else if (other.startRow == null) {
+          comp = 1;
+        } else {
+          comp = this.startRow.compareTo(other.startRow);
+        }
+        if (comp == 0) {
+          comp = Boolean.compare(other.startRowInclusive, this.startRowInclusive);
+        }
+      }
+    }
+
+    if (comp == 0) {
+      // Compare infinite end rows
+      comp = Boolean.compare(this.infiniteEndRow, other.infiniteEndRow);
+
+      // Compare non-infinite end rows and end row inclusiveness
+      if (comp == 0 && !this.infiniteEndRow) {
+        if (this.endRow == null && other.endRow == null) {
+          comp = 0;
+        } else if (this.endRow == null) {
+          comp = 1;
+        } else if (other.endRow == null) {
+          comp = -1;
+        } else {
+          comp = this.endRow.compareTo(other.endRow);
+        }
+        if (comp == 0) {
+          comp = Boolean.compare(this.endRowInclusive, other.endRowInclusive);
+        }
+      }
+    }
+
+    return comp;
+  }
+
+  /**
+   * Determines if this row range contains the given row.
+   *
+   * @param row row to check
+   * @return true if the row is contained in the row range, false otherwise
+   */
+  public boolean contains(Text row) {
+    if (infiniteStartRow) {
+      return !afterEndRow(row);
+    } else if (infiniteEndRow) {
+      return !beforeStartRow(row);
+    } else {
+      return !beforeStartRow(row) && !afterEndRow(row);
+    }
+  }
+
+  /**
+   * Merges overlapping and adjacent row ranges. For example, given the following input:
+   *
+   * <pre>
+   * [a,c], (c, d], (g,m), (j,t]
+   * </pre>
+   *
+   * the following row ranges would be returned:
+   *
+   * <pre>
+   * [a,d], (g,t]
+   * </pre>
+   *
+   * @param rowRanges the collection of row ranges to merge
+   * @return a list of merged row ranges
+   */
+  public static List<RowRange> mergeOverlapping(Collection<RowRange> rowRanges) {
+    if (rowRanges.isEmpty()) {
+      return Collections.emptyList();
+    }
+    if (rowRanges.size() == 1) {
+      return Collections.singletonList(rowRanges.iterator().next());
+    }
+
+    List<RowRange> sortedRowRanges = new ArrayList<>(rowRanges);
+    // Sort row ranges by their startRow values
+    sortedRowRanges.sort((r1, r2) -> {
+      if (r1.startRow == null && r2.startRow == null) {

Review Comment:
   Maybe could use the START_ROW_COMPARATOR suggested in another comment here.



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.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> {
+  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) {
+    return beforeStartRowImpl(row);
+  }
+
+  /**
+   * Determines if the given row is after the end row of this row range.
+   *
+   * @param row row to check
+   * @return true if the given row is after the row range, otherwise false
+   */
+  public boolean afterEndRow(Text row) {
+    if (infiniteEndRow) {
+      return false;
+    }
+
+    if (endRowInclusive) {
+      return row.compareTo(endRow) > 0;
+    }
+    return row.compareTo(endRow) >= 0;
+  }
+
+  /**
+   * Implements logic of {@link #beforeStartRow(Text)}, but in a private method, so that it can be
+   * safely used by constructors if a subclass overrides that {@link #beforeStartRow(Text)}
+   */
+  private boolean beforeStartRowImpl(Text row) {
+    if (this.infiniteStartRow) {
+      return false;
+    }
+
+    if (startRowInclusive) {
+      return row.compareTo(startRow) < 0;
+    }
+    return row.compareTo(startRow) <= 0;
+  }
+
+  @Override
+  public String toString() {
+    final String startRangeSymbol = (startRowInclusive && startRow != null) ? "[" : "(";
+    final String startRow = this.startRow == null ? "-inf" : this.startRow.toString();
+    final String endRow = this.endRow == null ? "+inf" : this.endRow.toString();
+    final String endRangeSymbol = (endRowInclusive && this.endRow != null) ? "]" : ")";
+    return startRangeSymbol + startRow + "," + endRow + endRangeSymbol;
+  }
+
+  @Override
+  public int hashCode() {
+    int startHash = infiniteStartRow ? 0 : startRow.hashCode() + (startRowInclusive ? 1 : 0);
+    int stopHash = infiniteEndRow ? 0 : endRow.hashCode() + (endRowInclusive ? 1 : 0);
+
+    return startHash + stopHash;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof RowRange)) {
+      return false;
+    }
+    return equals((RowRange) other);
+  }
+
+  /**
+   * Determines if this row range equals another.
+   *
+   * @param other range to compare
+   * @return true if row ranges are equals, false otherwise
+   */
+  public boolean equals(RowRange other) {
+    return compareTo(other) == 0;
+  }
+
+  /**
+   * Compares this row range to another row range. Compares in order: start row, inclusiveness of
+   * start row, end row, inclusiveness of end row. Infinite rows sort first, and non-infinite rows
+   * are compared with {@link Text#compareTo(BinaryComparable)}. Inclusive sorts before
+   * non-inclusive.
+   *
+   * @param other range row to compare
+   * @return comparison result
+   */
+  @Override
+  public int compareTo(RowRange other) {
+    // Compare infinite start rows
+    int comp = Boolean.compare(this.infiniteStartRow, other.infiniteStartRow);
+
+    if (comp == 0) {
+      // Compare non-infinite start rows and start row inclusiveness
+      if (!this.infiniteStartRow) {
+        if (this.startRow == null && other.startRow == null) {
+          comp = 0;
+        } else if (this.startRow == null) {
+          comp = -1;
+        } else if (other.startRow == null) {
+          comp = 1;
+        } else {
+          comp = this.startRow.compareTo(other.startRow);
+        }
+        if (comp == 0) {
+          comp = Boolean.compare(other.startRowInclusive, this.startRowInclusive);
+        }
+      }
+    }
+
+    if (comp == 0) {
+      // Compare infinite end rows
+      comp = Boolean.compare(this.infiniteEndRow, other.infiniteEndRow);
+
+      // Compare non-infinite end rows and end row inclusiveness
+      if (comp == 0 && !this.infiniteEndRow) {
+        if (this.endRow == null && other.endRow == null) {
+          comp = 0;
+        } else if (this.endRow == null) {
+          comp = 1;
+        } else if (other.endRow == null) {
+          comp = -1;
+        } else {
+          comp = this.endRow.compareTo(other.endRow);
+        }
+        if (comp == 0) {
+          comp = Boolean.compare(this.endRowInclusive, other.endRowInclusive);
+        }
+      }
+    }

Review Comment:
   With the following
   
   ```java
     // create once and reused to avoid object allocation on each compare call
     private static final Comparator<Text> START_ROW_COMPARATOR = Comparator.nullsFirst(Text::compareTo);
     private static final Comparator<Text> END_ROW_COMPARATOR = Comparator.nullsLast(Text::compareTo);
   ```
   
   could shorten comparison function
   
   ```suggestion
         if (!this.infiniteStartRow) {
          comp = START_ROW_COMPARATOR.compare(this.startRow, other.startRow);
           if (comp == 0) {
             comp = Boolean.compare(other.startRowInclusive, this.startRowInclusive);
           }
         }
       }
   
       if (comp == 0) {
         // Compare infinite end rows
         comp = Boolean.compare(this.infiniteEndRow, other.infiniteEndRow);
   
         // Compare non-infinite end rows and end row inclusiveness
         if (comp == 0 && !this.infiniteEndRow) {
          comp = END_ROW_COMPARATOR.compare(this.endRow, other.endRow);
          
           if (comp == 0) {
             comp = Boolean.compare(this.endRowInclusive, other.endRowInclusive);
           }
         }
       }
   ```
   



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.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> {
+  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);

Review Comment:
   This method is not covered by unit test.



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.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> {
+  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) {
+    return beforeStartRowImpl(row);
+  }
+
+  /**
+   * Determines if the given row is after the end row of this row range.
+   *
+   * @param row row to check
+   * @return true if the given row is after the row range, otherwise false
+   */
+  public boolean afterEndRow(Text row) {
+    if (infiniteEndRow) {
+      return false;
+    }
+
+    if (endRowInclusive) {
+      return row.compareTo(endRow) > 0;
+    }
+    return row.compareTo(endRow) >= 0;
+  }
+
+  /**
+   * Implements logic of {@link #beforeStartRow(Text)}, but in a private method, so that it can be
+   * safely used by constructors if a subclass overrides that {@link #beforeStartRow(Text)}
+   */
+  private boolean beforeStartRowImpl(Text row) {
+    if (this.infiniteStartRow) {
+      return false;
+    }
+
+    if (startRowInclusive) {
+      return row.compareTo(startRow) < 0;
+    }
+    return row.compareTo(startRow) <= 0;
+  }
+
+  @Override
+  public String toString() {
+    final String startRangeSymbol = (startRowInclusive && startRow != null) ? "[" : "(";
+    final String startRow = this.startRow == null ? "-inf" : this.startRow.toString();
+    final String endRow = this.endRow == null ? "+inf" : this.endRow.toString();
+    final String endRangeSymbol = (endRowInclusive && this.endRow != null) ? "]" : ")";
+    return startRangeSymbol + startRow + "," + endRow + endRangeSymbol;
+  }
+
+  @Override
+  public int hashCode() {
+    int startHash = infiniteStartRow ? 0 : startRow.hashCode() + (startRowInclusive ? 1 : 0);
+    int stopHash = infiniteEndRow ? 0 : endRow.hashCode() + (endRowInclusive ? 1 : 0);
+
+    return startHash + stopHash;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof RowRange)) {
+      return false;

Review Comment:
   I was curious about the code covered on the clip and merge overlapping method and I tried running the unit test with code coverage locally.  The class had really good coverage. Only a few places were not covered.  This line was one.  I commented on the other lines I saw that were not covered.



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.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> {
+  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);
+  }

Review Comment:
   These method were not covered.



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.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> {
+  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) {
+    return beforeStartRowImpl(row);
+  }
+
+  /**
+   * Determines if the given row is after the end row of this row range.
+   *
+   * @param row row to check
+   * @return true if the given row is after the row range, otherwise false
+   */
+  public boolean afterEndRow(Text row) {
+    if (infiniteEndRow) {
+      return false;
+    }
+
+    if (endRowInclusive) {
+      return row.compareTo(endRow) > 0;
+    }
+    return row.compareTo(endRow) >= 0;
+  }
+
+  /**
+   * Implements logic of {@link #beforeStartRow(Text)}, but in a private method, so that it can be
+   * safely used by constructors if a subclass overrides that {@link #beforeStartRow(Text)}
+   */
+  private boolean beforeStartRowImpl(Text row) {
+    if (this.infiniteStartRow) {
+      return false;

Review Comment:
   This was not covered



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.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> {
+  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(

Review Comment:
   This was not covered by unit test



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.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> {
+  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) {
+    return beforeStartRowImpl(row);
+  }
+
+  /**
+   * Determines if the given row is after the end row of this row range.
+   *
+   * @param row row to check
+   * @return true if the given row is after the row range, otherwise false
+   */
+  public boolean afterEndRow(Text row) {
+    if (infiniteEndRow) {
+      return false;
+    }
+
+    if (endRowInclusive) {
+      return row.compareTo(endRow) > 0;
+    }
+    return row.compareTo(endRow) >= 0;
+  }
+
+  /**
+   * Implements logic of {@link #beforeStartRow(Text)}, but in a private method, so that it can be
+   * safely used by constructors if a subclass overrides that {@link #beforeStartRow(Text)}
+   */
+  private boolean beforeStartRowImpl(Text row) {
+    if (this.infiniteStartRow) {
+      return false;
+    }
+
+    if (startRowInclusive) {
+      return row.compareTo(startRow) < 0;
+    }
+    return row.compareTo(startRow) <= 0;
+  }
+
+  @Override
+  public String toString() {
+    final String startRangeSymbol = (startRowInclusive && startRow != null) ? "[" : "(";
+    final String startRow = this.startRow == null ? "-inf" : this.startRow.toString();
+    final String endRow = this.endRow == null ? "+inf" : this.endRow.toString();
+    final String endRangeSymbol = (endRowInclusive && this.endRow != null) ? "]" : ")";
+    return startRangeSymbol + startRow + "," + endRow + endRangeSymbol;
+  }
+
+  @Override
+  public int hashCode() {
+    int startHash = infiniteStartRow ? 0 : startRow.hashCode() + (startRowInclusive ? 1 : 0);
+    int stopHash = infiniteEndRow ? 0 : endRow.hashCode() + (endRowInclusive ? 1 : 0);
+
+    return startHash + stopHash;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof RowRange)) {
+      return false;
+    }
+    return equals((RowRange) other);
+  }
+
+  /**
+   * Determines if this row range equals another.
+   *
+   * @param other range to compare
+   * @return true if row ranges are equals, false otherwise
+   */
+  public boolean equals(RowRange other) {
+    return compareTo(other) == 0;
+  }
+
+  /**
+   * Compares this row range to another row range. Compares in order: start row, inclusiveness of
+   * start row, end row, inclusiveness of end row. Infinite rows sort first, and non-infinite rows
+   * are compared with {@link Text#compareTo(BinaryComparable)}. Inclusive sorts before
+   * non-inclusive.
+   *
+   * @param other range row to compare
+   * @return comparison result
+   */
+  @Override
+  public int compareTo(RowRange other) {
+    // Compare infinite start rows
+    int comp = Boolean.compare(this.infiniteStartRow, other.infiniteStartRow);
+
+    if (comp == 0) {
+      // Compare non-infinite start rows and start row inclusiveness
+      if (!this.infiniteStartRow) {
+        if (this.startRow == null && other.startRow == null) {
+          comp = 0;
+        } else if (this.startRow == null) {
+          comp = -1;
+        } else if (other.startRow == null) {
+          comp = 1;
+        } else {
+          comp = this.startRow.compareTo(other.startRow);
+        }
+        if (comp == 0) {
+          comp = Boolean.compare(other.startRowInclusive, this.startRowInclusive);
+        }
+      }
+    }
+
+    if (comp == 0) {
+      // Compare infinite end rows
+      comp = Boolean.compare(this.infiniteEndRow, other.infiniteEndRow);
+
+      // Compare non-infinite end rows and end row inclusiveness
+      if (comp == 0 && !this.infiniteEndRow) {
+        if (this.endRow == null && other.endRow == null) {
+          comp = 0;
+        } else if (this.endRow == null) {
+          comp = 1;
+        } else if (other.endRow == null) {
+          comp = -1;
+        } else {
+          comp = this.endRow.compareTo(other.endRow);
+        }
+        if (comp == 0) {
+          comp = Boolean.compare(this.endRowInclusive, other.endRowInclusive);
+        }
+      }
+    }
+
+    return comp;
+  }
+
+  /**
+   * Determines if this row range contains the given row.
+   *
+   * @param row row to check
+   * @return true if the row is contained in the row range, false otherwise
+   */
+  public boolean contains(Text row) {
+    if (infiniteStartRow) {
+      return !afterEndRow(row);
+    } else if (infiniteEndRow) {
+      return !beforeStartRow(row);
+    } else {
+      return !beforeStartRow(row) && !afterEndRow(row);
+    }
+  }
+
+  /**
+   * Merges overlapping and adjacent row ranges. For example, given the following input:
+   *
+   * <pre>
+   * [a,c], (c, d], (g,m), (j,t]
+   * </pre>
+   *
+   * the following row ranges would be returned:
+   *
+   * <pre>
+   * [a,d], (g,t]
+   * </pre>
+   *
+   * @param rowRanges the collection of row ranges to merge
+   * @return a list of merged row ranges
+   */
+  public static List<RowRange> mergeOverlapping(Collection<RowRange> rowRanges) {
+    if (rowRanges.isEmpty()) {
+      return Collections.emptyList();
+    }
+    if (rowRanges.size() == 1) {
+      return Collections.singletonList(rowRanges.iterator().next());
+    }
+
+    List<RowRange> sortedRowRanges = new ArrayList<>(rowRanges);
+    // Sort row ranges by their startRow values
+    sortedRowRanges.sort((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);
+    });
+
+    ArrayList<RowRange> mergedRowRanges = new ArrayList<>(rowRanges.size());
+
+    // Initialize the current range for merging
+    RowRange currentRange = sortedRowRanges.get(0);
+    boolean currentStartRowInclusive = sortedRowRanges.get(0).startRowInclusive;
+
+    // Iterate through the sorted row ranges, merging overlapping and adjacent ranges
+    for (int i = 1; i < sortedRowRanges.size(); i++) {
+      if (currentRange.infiniteStartRow && currentRange.infiniteEndRow) {
+        // The current range covers all possible rows, no further merging needed
+        break;
+      }
+
+      RowRange nextRange = sortedRowRanges.get(i);
+
+      if (nextRange.infiniteStartRow && nextRange.infiniteEndRow) {
+        // The next range covers all possible rows, set the current range to cover all rows
+        currentRange = RowRange.all();
+        break;
+      }
+
+      boolean startRowsEqual = (currentRange.startRow == null && nextRange.startRow == null)
+          || (currentRange.startRow != null && currentRange.startRow.equals(nextRange.startRow));
+
+      int comparison;
+      if (startRowsEqual || currentRange.infiniteEndRow
+          || (nextRange.startRow != null && (currentRange.endRow == null
+              || currentRange.endRow.compareTo(nextRange.startRow) > 0
+              || (currentRange.endRow.equals(nextRange.startRow)
+                  && (!currentRange.endRowInclusive || nextRange.startRowInclusive))))) {
+        if (nextRange.infiniteEndRow) {
+          comparison = 1;
+        } else if (currentRange.endRow == null) {
+          comparison = -1;
+        } else {
+          comparison = nextRange.endRow.compareTo(currentRange.endRow);
+        }
+        if (comparison > 0 || (comparison == 0 && nextRange.endRowInclusive)) {
+          currentRange = RowRange.range(currentRange.startRow, currentStartRowInclusive,
+              nextRange.endRow, nextRange.endRowInclusive);
+        } // else current range contains the next range
+      } else {
+        // No overlap or adjacency, add the current range to the merged list and update the current
+        // range
+        mergedRowRanges.add(currentRange);
+        currentRange = nextRange;
+        currentStartRowInclusive = nextRange.startRowInclusive;
+      }
+    }
+
+    // Add the final current range to the merged list
+    mergedRowRanges.add(currentRange);
+
+    return mergedRowRanges;
+  }
+
+  /**
+   * Creates a row range which represents the intersection of this row range and the passed in row
+   * range. The following example will print true.
+   *
+   * <pre>
+   * RowRange rowRange1 = RowRange.closed(&quot;a&quot;, &quot;f&quot;);
+   * RowRange rowRange2 = RowRange.closed(&quot;c&quot;, &quot;n&quot;);
+   * RowRange rowRange3 = rowRange1.clip(rowRange2);
+   * System.out.println(rowRange3.equals(RowRange.closed(&quot;c&quot;, &quot;f&quot;)));
+   * </pre>
+   *
+   * @param rowRange row range to clip to
+   * @return the intersection of this row range and the given row range
+   * @throws IllegalArgumentException if row ranges do not overlap
+   */
+  public RowRange clip(RowRange rowRange) {
+    return clip(rowRange, false);
+  }
+
+  /**
+   * Creates a row range which represents the intersection of this row range and the passed in row
+   * range. Unlike {@link #clip(RowRange)}, this method can optionally return null if the row ranges
+   * do not overlap, instead of throwing an exception. The returnNullIfDisjoint parameter controls
+   * this behavior.
+   *
+   * @param rowRange row range to clip to
+   * @param returnNullIfDisjoint true to return null if row ranges are disjoint, false to throw an
+   *        exception
+   * @return the intersection of this row range and the given row range, or null if row ranges do
+   *         not overlap and returnNullIfDisjoint is true
+   * @throws IllegalArgumentException if row ranges do not overlap and returnNullIfDisjoint is false
+   * @see #clip(RowRange)
+   */
+  public RowRange clip(RowRange rowRange, boolean returnNullIfDisjoint) {
+    // Initialize start and end row values with the current instance's values
+    Text startRow = this.startRow;
+    boolean startRowInclusive = this.startRowInclusive;
+    Text endRow = this.endRow;
+    boolean endRowInclusive = this.endRowInclusive;
+
+    // If the input rowRange has a defined startRow, update startRow and startRowInclusive if needed
+    if (rowRange.startRow != null) {
+      // If the input rowRange's startRow is after this instance's endRow or equal but not
+      // inclusive, they do not overlap
+      if (afterEndRow(rowRange.startRow) || (rowRange.startRow.equals(this.endRow)
+          && !(rowRange.startRowInclusive && this.endRowInclusive))) {
+        if (returnNullIfDisjoint) {
+          return null;

Review Comment:
   This was not covered by unit test



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.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> {
+  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);

Review Comment:
   This was not covered by unit test.



##########
core/src/test/java/org/apache/accumulo/core/data/RowRangeTest.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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 static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.stream.Stream;
+
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.Nested;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+public class RowRangeTest {
+
+  @Nested
+  class StaticEntryPointTests {
+
+    @Test
+    void testClosedOpenEquality() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen(new Text("r1"), new Text("row5"));
+      RowRange range3 = RowRange.range(new Text("r1"), true, new Text("row5"), false);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testOpenClosedEquality() {
+      RowRange range1 = RowRange.openClosed("r1", "row5");
+      RowRange range2 = RowRange.openClosed(new Text("r1"), new Text("row5"));
+      RowRange range3 = RowRange.range(new Text("r1"), false, new Text("row5"), true);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testAtLeastEquality() {
+      RowRange range1 = RowRange.atLeast("r1");
+      RowRange range2 = RowRange.atLeast(new Text("r1"));
+      RowRange range3 = RowRange.range(new Text("r1"), true, null, false);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testLessThanEquality() {
+      RowRange range1 = RowRange.lessThan("row5");
+      RowRange range2 = RowRange.lessThan(new Text("row5"));
+      RowRange range3 = RowRange.range(null, false, new Text("row5"), false);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testAtMostEquality() {
+      RowRange range1 = RowRange.atMost("row5");
+      RowRange range2 = RowRange.atMost(new Text("row5"));
+      RowRange range3 = RowRange.range(null, false, new Text("row5"), true);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testAllEquality() {
+      RowRange range1 = RowRange.all();
+      RowRange range2 = RowRange.range((Text) null, false, null, false);
+
+      assertTrue(range1.equals(range2));
+    }
+  }
+
+  @Nested
+  class EqualsTests {
+
+    @Test
+    void testEqualsWithDifferentRanges() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen("r2", "row4");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithSameRange() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen("r1", "row5");
+      assertTrue(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithDifferentStartRowInclusiveness() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.openClosed("r1", "row5");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithDifferentEndRowInclusiveness() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen("r1", "row4");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithDifferentStartRowAndEndRowInclusiveness() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.openClosed("r1", "row4");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testOverloadEquality() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen(new Text("r1"), new Text("row5"));
+      RowRange range3 = RowRange.atLeast("row8");
+      RowRange range4 = RowRange.atLeast(new Text("row8"));
+      RowRange range5 = RowRange.lessThan("r2");
+      RowRange range6 = RowRange.lessThan(new Text("r2"));
+      RowRange range7 = RowRange.atMost("r3");
+      RowRange range8 = RowRange.atMost(new Text("r3"));
+
+      // Test that all ranges created using different entry point methods are equal
+      assertTrue(range1.equals(range2));
+      assertTrue(range3.equals(range4));
+      assertTrue(range5.equals(range6));
+      assertTrue(range7.equals(range8));
+
+      // Test that ranges with different properties are not equal
+      assertFalse(range1.equals(range3));
+      assertFalse(range1.equals(range5));
+      assertFalse(range1.equals(range7));
+      assertFalse(range3.equals(range5));
+      assertFalse(range3.equals(range7));
+      assertFalse(range5.equals(range7));
+    }
+  }
+
+  @Nested
+  class CompareToTests {
+
+    @Test
+    void testCompareWithSameRange() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.open("r1", "r3");
+      assertEquals(0, range1.compareTo(range2));
+    }
+
+    @Test
+    void testCompareWithDifferentStartRow() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.open("r2", "r3");
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithDifferentEndRow() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.open("r1", "r4");
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithDifferentStartRowInclusiveness() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.closedOpen("r1", "r3");
+      assertTrue(range1.compareTo(range2) > 0);
+    }
+
+    @Test
+    void testCompareWithDifferentEndRowInclusiveness() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.openClosed("r1", "r3");
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithInfiniteStartRow() {
+      RowRange range1 = RowRange.atLeast("r1");
+      RowRange range2 = RowRange.all();
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithInfiniteEndRow() {
+      RowRange range1 = RowRange.all();
+      RowRange range2 = RowRange.atLeast("r1");
+      assertTrue(range1.compareTo(range2) > 0);
+    }
+  }
+
+  @Nested
+  class ContainsTests {
+
+    @Test
+    void testContainsWithAllRange() {
+      RowRange range = RowRange.all();
+      assertTrue(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertTrue(range.contains(new Text("row3")));
+    }
+
+    @Test
+    void testContainsWithOpenRange() {
+      RowRange range = RowRange.open("r1", "r3");
+      assertFalse(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertFalse(range.contains(new Text("r3")));

Review Comment:
   Added some test data w/ same prefix and with binary zero that is next possible row.
   
   ```suggestion
         assertFalse(range.contains(new Text("r1")));
         assertTrue(range.contains(new Text(new byte[]{'r','1',0})));
         assertTrue(range.contains(new Text("r11")));
         assertTrue(range.contains(new Text("r2")));
         assertFalse(range.contains(new Text(new byte[]{'r','3',0})));
         assertFalse(range.contains(new Text("r3")));
   ```



##########
core/src/test/java/org/apache/accumulo/core/data/RowRangeTest.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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 static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.stream.Stream;
+
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.Nested;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+public class RowRangeTest {
+
+  @Nested
+  class StaticEntryPointTests {
+
+    @Test
+    void testClosedOpenEquality() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen(new Text("r1"), new Text("row5"));
+      RowRange range3 = RowRange.range(new Text("r1"), true, new Text("row5"), false);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testOpenClosedEquality() {
+      RowRange range1 = RowRange.openClosed("r1", "row5");
+      RowRange range2 = RowRange.openClosed(new Text("r1"), new Text("row5"));
+      RowRange range3 = RowRange.range(new Text("r1"), false, new Text("row5"), true);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testAtLeastEquality() {
+      RowRange range1 = RowRange.atLeast("r1");
+      RowRange range2 = RowRange.atLeast(new Text("r1"));
+      RowRange range3 = RowRange.range(new Text("r1"), true, null, false);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testLessThanEquality() {
+      RowRange range1 = RowRange.lessThan("row5");
+      RowRange range2 = RowRange.lessThan(new Text("row5"));
+      RowRange range3 = RowRange.range(null, false, new Text("row5"), false);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testAtMostEquality() {
+      RowRange range1 = RowRange.atMost("row5");
+      RowRange range2 = RowRange.atMost(new Text("row5"));
+      RowRange range3 = RowRange.range(null, false, new Text("row5"), true);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testAllEquality() {
+      RowRange range1 = RowRange.all();
+      RowRange range2 = RowRange.range((Text) null, false, null, false);
+
+      assertTrue(range1.equals(range2));
+    }
+  }
+
+  @Nested
+  class EqualsTests {
+
+    @Test
+    void testEqualsWithDifferentRanges() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen("r2", "row4");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithSameRange() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen("r1", "row5");
+      assertTrue(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithDifferentStartRowInclusiveness() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.openClosed("r1", "row5");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithDifferentEndRowInclusiveness() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen("r1", "row4");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithDifferentStartRowAndEndRowInclusiveness() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.openClosed("r1", "row4");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testOverloadEquality() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen(new Text("r1"), new Text("row5"));
+      RowRange range3 = RowRange.atLeast("row8");
+      RowRange range4 = RowRange.atLeast(new Text("row8"));
+      RowRange range5 = RowRange.lessThan("r2");
+      RowRange range6 = RowRange.lessThan(new Text("r2"));
+      RowRange range7 = RowRange.atMost("r3");
+      RowRange range8 = RowRange.atMost(new Text("r3"));
+
+      // Test that all ranges created using different entry point methods are equal
+      assertTrue(range1.equals(range2));
+      assertTrue(range3.equals(range4));
+      assertTrue(range5.equals(range6));
+      assertTrue(range7.equals(range8));
+
+      // Test that ranges with different properties are not equal
+      assertFalse(range1.equals(range3));
+      assertFalse(range1.equals(range5));
+      assertFalse(range1.equals(range7));
+      assertFalse(range3.equals(range5));
+      assertFalse(range3.equals(range7));
+      assertFalse(range5.equals(range7));
+    }
+  }
+
+  @Nested
+  class CompareToTests {
+
+    @Test
+    void testCompareWithSameRange() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.open("r1", "r3");
+      assertEquals(0, range1.compareTo(range2));
+    }
+
+    @Test
+    void testCompareWithDifferentStartRow() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.open("r2", "r3");
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithDifferentEndRow() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.open("r1", "r4");
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithDifferentStartRowInclusiveness() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.closedOpen("r1", "r3");
+      assertTrue(range1.compareTo(range2) > 0);
+    }
+
+    @Test
+    void testCompareWithDifferentEndRowInclusiveness() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.openClosed("r1", "r3");
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithInfiniteStartRow() {
+      RowRange range1 = RowRange.atLeast("r1");
+      RowRange range2 = RowRange.all();
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithInfiniteEndRow() {
+      RowRange range1 = RowRange.all();
+      RowRange range2 = RowRange.atLeast("r1");
+      assertTrue(range1.compareTo(range2) > 0);
+    }
+  }
+
+  @Nested
+  class ContainsTests {
+
+    @Test
+    void testContainsWithAllRange() {
+      RowRange range = RowRange.all();
+      assertTrue(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertTrue(range.contains(new Text("row3")));
+    }
+
+    @Test
+    void testContainsWithOpenRange() {
+      RowRange range = RowRange.open("r1", "r3");
+      assertFalse(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertFalse(range.contains(new Text("r3")));
+    }
+
+    @Test
+    void testContainsWithClosedRange() {
+      RowRange range = RowRange.closed("r1", "r3");
+      assertTrue(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertTrue(range.contains(new Text("r3")));

Review Comment:
   ```suggestion
         assertFalse(range.contains(new Text("r1")));
         assertTrue(range.contains(new Text("r1")));
         assertTrue(range.contains(new Text("r2")));
         assertTrue(range.contains(new Text("r3")));
        assertFalse(range.contains(new Text("r30")));
   ```



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.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> {
+  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) {
+    return beforeStartRowImpl(row);
+  }
+
+  /**
+   * Determines if the given row is after the end row of this row range.
+   *
+   * @param row row to check
+   * @return true if the given row is after the row range, otherwise false
+   */
+  public boolean afterEndRow(Text row) {
+    if (infiniteEndRow) {
+      return false;
+    }
+
+    if (endRowInclusive) {
+      return row.compareTo(endRow) > 0;
+    }
+    return row.compareTo(endRow) >= 0;
+  }
+
+  /**
+   * Implements logic of {@link #beforeStartRow(Text)}, but in a private method, so that it can be
+   * safely used by constructors if a subclass overrides that {@link #beforeStartRow(Text)}
+   */
+  private boolean beforeStartRowImpl(Text row) {
+    if (this.infiniteStartRow) {
+      return false;
+    }
+
+    if (startRowInclusive) {
+      return row.compareTo(startRow) < 0;
+    }
+    return row.compareTo(startRow) <= 0;
+  }
+
+  @Override
+  public String toString() {
+    final String startRangeSymbol = (startRowInclusive && startRow != null) ? "[" : "(";
+    final String startRow = this.startRow == null ? "-inf" : this.startRow.toString();
+    final String endRow = this.endRow == null ? "+inf" : this.endRow.toString();
+    final String endRangeSymbol = (endRowInclusive && this.endRow != null) ? "]" : ")";
+    return startRangeSymbol + startRow + "," + endRow + endRangeSymbol;
+  }
+
+  @Override
+  public int hashCode() {
+    int startHash = infiniteStartRow ? 0 : startRow.hashCode() + (startRowInclusive ? 1 : 0);
+    int stopHash = infiniteEndRow ? 0 : endRow.hashCode() + (endRowInclusive ? 1 : 0);
+
+    return startHash + stopHash;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof RowRange)) {
+      return false;
+    }
+    return equals((RowRange) other);
+  }
+
+  /**
+   * Determines if this row range equals another.
+   *
+   * @param other range to compare
+   * @return true if row ranges are equals, false otherwise
+   */
+  public boolean equals(RowRange other) {
+    return compareTo(other) == 0;
+  }
+
+  /**
+   * Compares this row range to another row range. Compares in order: start row, inclusiveness of
+   * start row, end row, inclusiveness of end row. Infinite rows sort first, and non-infinite rows
+   * are compared with {@link Text#compareTo(BinaryComparable)}. Inclusive sorts before
+   * non-inclusive.
+   *
+   * @param other range row to compare
+   * @return comparison result
+   */
+  @Override
+  public int compareTo(RowRange other) {
+    // Compare infinite start rows
+    int comp = Boolean.compare(this.infiniteStartRow, other.infiniteStartRow);
+
+    if (comp == 0) {
+      // Compare non-infinite start rows and start row inclusiveness
+      if (!this.infiniteStartRow) {
+        if (this.startRow == null && other.startRow == null) {
+          comp = 0;
+        } else if (this.startRow == null) {
+          comp = -1;
+        } else if (other.startRow == null) {
+          comp = 1;
+        } else {
+          comp = this.startRow.compareTo(other.startRow);
+        }
+        if (comp == 0) {
+          comp = Boolean.compare(other.startRowInclusive, this.startRowInclusive);
+        }
+      }
+    }
+
+    if (comp == 0) {
+      // Compare infinite end rows
+      comp = Boolean.compare(this.infiniteEndRow, other.infiniteEndRow);
+
+      // Compare non-infinite end rows and end row inclusiveness
+      if (comp == 0 && !this.infiniteEndRow) {
+        if (this.endRow == null && other.endRow == null) {
+          comp = 0;
+        } else if (this.endRow == null) {
+          comp = 1;
+        } else if (other.endRow == null) {
+          comp = -1;
+        } else {
+          comp = this.endRow.compareTo(other.endRow);
+        }
+        if (comp == 0) {
+          comp = Boolean.compare(this.endRowInclusive, other.endRowInclusive);
+        }
+      }
+    }
+
+    return comp;
+  }
+
+  /**
+   * Determines if this row range contains the given row.
+   *
+   * @param row row to check
+   * @return true if the row is contained in the row range, false otherwise
+   */
+  public boolean contains(Text row) {
+    if (infiniteStartRow) {
+      return !afterEndRow(row);
+    } else if (infiniteEndRow) {
+      return !beforeStartRow(row);
+    } else {
+      return !beforeStartRow(row) && !afterEndRow(row);
+    }
+  }
+
+  /**
+   * Merges overlapping and adjacent row ranges. For example, given the following input:
+   *
+   * <pre>
+   * [a,c], (c, d], (g,m), (j,t]
+   * </pre>
+   *
+   * the following row ranges would be returned:
+   *
+   * <pre>
+   * [a,d], (g,t]
+   * </pre>
+   *
+   * @param rowRanges the collection of row ranges to merge
+   * @return a list of merged row ranges
+   */
+  public static List<RowRange> mergeOverlapping(Collection<RowRange> rowRanges) {
+    if (rowRanges.isEmpty()) {
+      return Collections.emptyList();
+    }
+    if (rowRanges.size() == 1) {
+      return Collections.singletonList(rowRanges.iterator().next());
+    }
+
+    List<RowRange> sortedRowRanges = new ArrayList<>(rowRanges);
+    // Sort row ranges by their startRow values
+    sortedRowRanges.sort((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);
+    });
+
+    ArrayList<RowRange> mergedRowRanges = new ArrayList<>(rowRanges.size());
+
+    // Initialize the current range for merging
+    RowRange currentRange = sortedRowRanges.get(0);
+    boolean currentStartRowInclusive = sortedRowRanges.get(0).startRowInclusive;
+
+    // Iterate through the sorted row ranges, merging overlapping and adjacent ranges
+    for (int i = 1; i < sortedRowRanges.size(); i++) {
+      if (currentRange.infiniteStartRow && currentRange.infiniteEndRow) {
+        // The current range covers all possible rows, no further merging needed
+        break;
+      }
+
+      RowRange nextRange = sortedRowRanges.get(i);
+
+      if (nextRange.infiniteStartRow && nextRange.infiniteEndRow) {
+        // The next range covers all possible rows, set the current range to cover all rows
+        currentRange = RowRange.all();

Review Comment:
   This is not covered by unit test.



##########
core/src/test/java/org/apache/accumulo/core/data/RowRangeTest.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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 static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.stream.Stream;
+
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.Nested;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+public class RowRangeTest {
+
+  @Nested
+  class StaticEntryPointTests {
+
+    @Test
+    void testClosedOpenEquality() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen(new Text("r1"), new Text("row5"));
+      RowRange range3 = RowRange.range(new Text("r1"), true, new Text("row5"), false);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testOpenClosedEquality() {
+      RowRange range1 = RowRange.openClosed("r1", "row5");
+      RowRange range2 = RowRange.openClosed(new Text("r1"), new Text("row5"));
+      RowRange range3 = RowRange.range(new Text("r1"), false, new Text("row5"), true);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testAtLeastEquality() {
+      RowRange range1 = RowRange.atLeast("r1");
+      RowRange range2 = RowRange.atLeast(new Text("r1"));
+      RowRange range3 = RowRange.range(new Text("r1"), true, null, false);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testLessThanEquality() {
+      RowRange range1 = RowRange.lessThan("row5");
+      RowRange range2 = RowRange.lessThan(new Text("row5"));
+      RowRange range3 = RowRange.range(null, false, new Text("row5"), false);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testAtMostEquality() {
+      RowRange range1 = RowRange.atMost("row5");
+      RowRange range2 = RowRange.atMost(new Text("row5"));
+      RowRange range3 = RowRange.range(null, false, new Text("row5"), true);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testAllEquality() {
+      RowRange range1 = RowRange.all();
+      RowRange range2 = RowRange.range((Text) null, false, null, false);
+
+      assertTrue(range1.equals(range2));
+    }
+  }
+
+  @Nested
+  class EqualsTests {
+
+    @Test
+    void testEqualsWithDifferentRanges() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen("r2", "row4");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithSameRange() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen("r1", "row5");
+      assertTrue(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithDifferentStartRowInclusiveness() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.openClosed("r1", "row5");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithDifferentEndRowInclusiveness() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen("r1", "row4");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithDifferentStartRowAndEndRowInclusiveness() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.openClosed("r1", "row4");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testOverloadEquality() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen(new Text("r1"), new Text("row5"));
+      RowRange range3 = RowRange.atLeast("row8");
+      RowRange range4 = RowRange.atLeast(new Text("row8"));
+      RowRange range5 = RowRange.lessThan("r2");
+      RowRange range6 = RowRange.lessThan(new Text("r2"));
+      RowRange range7 = RowRange.atMost("r3");
+      RowRange range8 = RowRange.atMost(new Text("r3"));
+
+      // Test that all ranges created using different entry point methods are equal
+      assertTrue(range1.equals(range2));
+      assertTrue(range3.equals(range4));
+      assertTrue(range5.equals(range6));
+      assertTrue(range7.equals(range8));
+
+      // Test that ranges with different properties are not equal
+      assertFalse(range1.equals(range3));
+      assertFalse(range1.equals(range5));
+      assertFalse(range1.equals(range7));
+      assertFalse(range3.equals(range5));
+      assertFalse(range3.equals(range7));
+      assertFalse(range5.equals(range7));
+    }
+  }
+
+  @Nested
+  class CompareToTests {
+
+    @Test
+    void testCompareWithSameRange() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.open("r1", "r3");
+      assertEquals(0, range1.compareTo(range2));
+    }
+
+    @Test
+    void testCompareWithDifferentStartRow() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.open("r2", "r3");
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithDifferentEndRow() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.open("r1", "r4");
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithDifferentStartRowInclusiveness() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.closedOpen("r1", "r3");
+      assertTrue(range1.compareTo(range2) > 0);
+    }
+
+    @Test
+    void testCompareWithDifferentEndRowInclusiveness() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.openClosed("r1", "r3");
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithInfiniteStartRow() {
+      RowRange range1 = RowRange.atLeast("r1");
+      RowRange range2 = RowRange.all();
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithInfiniteEndRow() {
+      RowRange range1 = RowRange.all();
+      RowRange range2 = RowRange.atLeast("r1");
+      assertTrue(range1.compareTo(range2) > 0);
+    }
+  }
+
+  @Nested
+  class ContainsTests {
+
+    @Test
+    void testContainsWithAllRange() {
+      RowRange range = RowRange.all();
+      assertTrue(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertTrue(range.contains(new Text("row3")));
+    }
+
+    @Test
+    void testContainsWithOpenRange() {
+      RowRange range = RowRange.open("r1", "r3");
+      assertFalse(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertFalse(range.contains(new Text("r3")));
+    }
+
+    @Test
+    void testContainsWithClosedRange() {
+      RowRange range = RowRange.closed("r1", "r3");
+      assertTrue(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertTrue(range.contains(new Text("r3")));
+    }
+
+    @Test
+    void testContainsWithOpenClosedRange() {
+      RowRange range = RowRange.openClosed("r1", "r3");
+      assertFalse(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertTrue(range.contains(new Text("r3")));
+    }
+
+    @Test
+    void testContainsWithClosedOpenRange() {
+      RowRange range = RowRange.closedOpen("r1", "r3");
+      assertTrue(range.contains(new Text("r1")));

Review Comment:
   ```suggestion
        assertFalse(range.contains(new Text("r0")));
         assertTrue(range.contains(new Text("r1")));
   ```



##########
core/src/test/java/org/apache/accumulo/core/data/RowRangeTest.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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 static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.stream.Stream;
+
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.Nested;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+public class RowRangeTest {
+
+  @Nested
+  class StaticEntryPointTests {
+
+    @Test
+    void testClosedOpenEquality() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen(new Text("r1"), new Text("row5"));
+      RowRange range3 = RowRange.range(new Text("r1"), true, new Text("row5"), false);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testOpenClosedEquality() {
+      RowRange range1 = RowRange.openClosed("r1", "row5");
+      RowRange range2 = RowRange.openClosed(new Text("r1"), new Text("row5"));
+      RowRange range3 = RowRange.range(new Text("r1"), false, new Text("row5"), true);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testAtLeastEquality() {
+      RowRange range1 = RowRange.atLeast("r1");
+      RowRange range2 = RowRange.atLeast(new Text("r1"));
+      RowRange range3 = RowRange.range(new Text("r1"), true, null, false);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testLessThanEquality() {
+      RowRange range1 = RowRange.lessThan("row5");
+      RowRange range2 = RowRange.lessThan(new Text("row5"));
+      RowRange range3 = RowRange.range(null, false, new Text("row5"), false);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testAtMostEquality() {
+      RowRange range1 = RowRange.atMost("row5");
+      RowRange range2 = RowRange.atMost(new Text("row5"));
+      RowRange range3 = RowRange.range(null, false, new Text("row5"), true);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testAllEquality() {
+      RowRange range1 = RowRange.all();
+      RowRange range2 = RowRange.range((Text) null, false, null, false);
+
+      assertTrue(range1.equals(range2));
+    }
+  }
+
+  @Nested
+  class EqualsTests {
+
+    @Test
+    void testEqualsWithDifferentRanges() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen("r2", "row4");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithSameRange() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen("r1", "row5");
+      assertTrue(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithDifferentStartRowInclusiveness() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.openClosed("r1", "row5");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithDifferentEndRowInclusiveness() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen("r1", "row4");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithDifferentStartRowAndEndRowInclusiveness() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.openClosed("r1", "row4");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testOverloadEquality() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen(new Text("r1"), new Text("row5"));
+      RowRange range3 = RowRange.atLeast("row8");
+      RowRange range4 = RowRange.atLeast(new Text("row8"));
+      RowRange range5 = RowRange.lessThan("r2");
+      RowRange range6 = RowRange.lessThan(new Text("r2"));
+      RowRange range7 = RowRange.atMost("r3");
+      RowRange range8 = RowRange.atMost(new Text("r3"));
+
+      // Test that all ranges created using different entry point methods are equal
+      assertTrue(range1.equals(range2));
+      assertTrue(range3.equals(range4));
+      assertTrue(range5.equals(range6));
+      assertTrue(range7.equals(range8));
+
+      // Test that ranges with different properties are not equal
+      assertFalse(range1.equals(range3));
+      assertFalse(range1.equals(range5));
+      assertFalse(range1.equals(range7));
+      assertFalse(range3.equals(range5));
+      assertFalse(range3.equals(range7));
+      assertFalse(range5.equals(range7));
+    }
+  }
+
+  @Nested
+  class CompareToTests {
+
+    @Test
+    void testCompareWithSameRange() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.open("r1", "r3");
+      assertEquals(0, range1.compareTo(range2));
+    }
+
+    @Test
+    void testCompareWithDifferentStartRow() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.open("r2", "r3");
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithDifferentEndRow() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.open("r1", "r4");
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithDifferentStartRowInclusiveness() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.closedOpen("r1", "r3");
+      assertTrue(range1.compareTo(range2) > 0);
+    }
+
+    @Test
+    void testCompareWithDifferentEndRowInclusiveness() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.openClosed("r1", "r3");
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithInfiniteStartRow() {
+      RowRange range1 = RowRange.atLeast("r1");
+      RowRange range2 = RowRange.all();
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithInfiniteEndRow() {
+      RowRange range1 = RowRange.all();
+      RowRange range2 = RowRange.atLeast("r1");
+      assertTrue(range1.compareTo(range2) > 0);
+    }
+  }
+
+  @Nested
+  class ContainsTests {
+
+    @Test
+    void testContainsWithAllRange() {
+      RowRange range = RowRange.all();
+      assertTrue(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertTrue(range.contains(new Text("row3")));
+    }
+
+    @Test
+    void testContainsWithOpenRange() {
+      RowRange range = RowRange.open("r1", "r3");
+      assertFalse(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertFalse(range.contains(new Text("r3")));
+    }
+
+    @Test
+    void testContainsWithClosedRange() {
+      RowRange range = RowRange.closed("r1", "r3");
+      assertTrue(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertTrue(range.contains(new Text("r3")));
+    }
+
+    @Test
+    void testContainsWithOpenClosedRange() {
+      RowRange range = RowRange.openClosed("r1", "r3");
+      assertFalse(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertTrue(range.contains(new Text("r3")));
+    }
+
+    @Test
+    void testContainsWithClosedOpenRange() {
+      RowRange range = RowRange.closedOpen("r1", "r3");
+      assertTrue(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertFalse(range.contains(new Text("r3")));
+    }
+
+    @Test
+    void testContainsWithSingleRowRange() {
+      RowRange range = RowRange.closed("r1");
+      assertTrue(range.contains(new Text("r1")));
+      assertFalse(range.contains(new Text("r2")));
+    }
+
+    @Test
+    void testContainsWithAtLeastRange() {
+      RowRange range = RowRange.atLeast("r1");
+      assertTrue(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertFalse(range.contains(new Text("")));
+    }
+
+    @Test
+    void testContainsWithAtMostRange() {
+      RowRange range = RowRange.atMost("r1");
+      assertTrue(range.contains(new Text("r1")));
+      assertFalse(range.contains(new Text("r2")));

Review Comment:
   ```suggestion
         assertTrue(range.contains(new Text("r1")));
         assertFalse(range.contains(new Text("r10")));
         assertFalse(range.contains(new Text("r2")));
   ```



##########
core/src/test/java/org/apache/accumulo/core/data/RowRangeTest.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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 static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.stream.Stream;
+
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.Nested;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+public class RowRangeTest {
+
+  @Nested
+  class StaticEntryPointTests {
+
+    @Test
+    void testClosedOpenEquality() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen(new Text("r1"), new Text("row5"));
+      RowRange range3 = RowRange.range(new Text("r1"), true, new Text("row5"), false);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testOpenClosedEquality() {
+      RowRange range1 = RowRange.openClosed("r1", "row5");
+      RowRange range2 = RowRange.openClosed(new Text("r1"), new Text("row5"));
+      RowRange range3 = RowRange.range(new Text("r1"), false, new Text("row5"), true);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testAtLeastEquality() {
+      RowRange range1 = RowRange.atLeast("r1");
+      RowRange range2 = RowRange.atLeast(new Text("r1"));
+      RowRange range3 = RowRange.range(new Text("r1"), true, null, false);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testLessThanEquality() {
+      RowRange range1 = RowRange.lessThan("row5");
+      RowRange range2 = RowRange.lessThan(new Text("row5"));
+      RowRange range3 = RowRange.range(null, false, new Text("row5"), false);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testAtMostEquality() {
+      RowRange range1 = RowRange.atMost("row5");
+      RowRange range2 = RowRange.atMost(new Text("row5"));
+      RowRange range3 = RowRange.range(null, false, new Text("row5"), true);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testAllEquality() {
+      RowRange range1 = RowRange.all();
+      RowRange range2 = RowRange.range((Text) null, false, null, false);
+
+      assertTrue(range1.equals(range2));
+    }
+  }
+
+  @Nested
+  class EqualsTests {
+
+    @Test
+    void testEqualsWithDifferentRanges() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen("r2", "row4");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithSameRange() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen("r1", "row5");
+      assertTrue(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithDifferentStartRowInclusiveness() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.openClosed("r1", "row5");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithDifferentEndRowInclusiveness() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen("r1", "row4");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithDifferentStartRowAndEndRowInclusiveness() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.openClosed("r1", "row4");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testOverloadEquality() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen(new Text("r1"), new Text("row5"));
+      RowRange range3 = RowRange.atLeast("row8");
+      RowRange range4 = RowRange.atLeast(new Text("row8"));
+      RowRange range5 = RowRange.lessThan("r2");
+      RowRange range6 = RowRange.lessThan(new Text("r2"));
+      RowRange range7 = RowRange.atMost("r3");
+      RowRange range8 = RowRange.atMost(new Text("r3"));
+
+      // Test that all ranges created using different entry point methods are equal
+      assertTrue(range1.equals(range2));
+      assertTrue(range3.equals(range4));
+      assertTrue(range5.equals(range6));
+      assertTrue(range7.equals(range8));
+
+      // Test that ranges with different properties are not equal
+      assertFalse(range1.equals(range3));
+      assertFalse(range1.equals(range5));
+      assertFalse(range1.equals(range7));
+      assertFalse(range3.equals(range5));
+      assertFalse(range3.equals(range7));
+      assertFalse(range5.equals(range7));
+    }
+  }
+
+  @Nested
+  class CompareToTests {
+
+    @Test
+    void testCompareWithSameRange() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.open("r1", "r3");
+      assertEquals(0, range1.compareTo(range2));
+    }
+
+    @Test
+    void testCompareWithDifferentStartRow() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.open("r2", "r3");
+      assertTrue(range1.compareTo(range2) < 0);

Review Comment:
   For all of the compare to test it would be nice to try both directions.
   
   ```suggestion
         assertTrue(range1.compareTo(range2) < 0);
         assertTrue(range2.compareTo(range1) > 0);
   ```



##########
core/src/test/java/org/apache/accumulo/core/data/RowRangeTest.java:
##########
@@ -0,0 +1,569 @@
+/*
+ * 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 static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.stream.Stream;
+
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.Nested;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+public class RowRangeTest {
+
+  @Nested
+  class StaticEntryPointTests {
+
+    @Test
+    void testClosedOpenEquality() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen(new Text("r1"), new Text("row5"));
+      RowRange range3 = RowRange.range(new Text("r1"), true, new Text("row5"), false);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testOpenClosedEquality() {
+      RowRange range1 = RowRange.openClosed("r1", "row5");
+      RowRange range2 = RowRange.openClosed(new Text("r1"), new Text("row5"));
+      RowRange range3 = RowRange.range(new Text("r1"), false, new Text("row5"), true);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testAtLeastEquality() {
+      RowRange range1 = RowRange.atLeast("r1");
+      RowRange range2 = RowRange.atLeast(new Text("r1"));
+      RowRange range3 = RowRange.range(new Text("r1"), true, null, false);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testLessThanEquality() {
+      RowRange range1 = RowRange.lessThan("row5");
+      RowRange range2 = RowRange.lessThan(new Text("row5"));
+      RowRange range3 = RowRange.range(null, false, new Text("row5"), false);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testAtMostEquality() {
+      RowRange range1 = RowRange.atMost("row5");
+      RowRange range2 = RowRange.atMost(new Text("row5"));
+      RowRange range3 = RowRange.range(null, false, new Text("row5"), true);
+
+      assertTrue(range1.equals(range2));
+      assertTrue(range1.equals(range3));
+      assertTrue(range2.equals(range3));
+    }
+
+    @Test
+    void testAllEquality() {
+      RowRange range1 = RowRange.all();
+      RowRange range2 = RowRange.range((Text) null, false, null, false);
+
+      assertTrue(range1.equals(range2));
+    }
+  }
+
+  @Nested
+  class EqualsTests {
+
+    @Test
+    void testEqualsWithDifferentRanges() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen("r2", "row4");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithSameRange() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen("r1", "row5");
+      assertTrue(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithDifferentStartRowInclusiveness() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.openClosed("r1", "row5");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithDifferentEndRowInclusiveness() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen("r1", "row4");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testEqualsWithDifferentStartRowAndEndRowInclusiveness() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.openClosed("r1", "row4");
+      assertFalse(range1.equals(range2));
+    }
+
+    @Test
+    void testOverloadEquality() {
+      RowRange range1 = RowRange.closedOpen("r1", "row5");
+      RowRange range2 = RowRange.closedOpen(new Text("r1"), new Text("row5"));
+      RowRange range3 = RowRange.atLeast("row8");
+      RowRange range4 = RowRange.atLeast(new Text("row8"));
+      RowRange range5 = RowRange.lessThan("r2");
+      RowRange range6 = RowRange.lessThan(new Text("r2"));
+      RowRange range7 = RowRange.atMost("r3");
+      RowRange range8 = RowRange.atMost(new Text("r3"));
+
+      // Test that all ranges created using different entry point methods are equal
+      assertTrue(range1.equals(range2));
+      assertTrue(range3.equals(range4));
+      assertTrue(range5.equals(range6));
+      assertTrue(range7.equals(range8));
+
+      // Test that ranges with different properties are not equal
+      assertFalse(range1.equals(range3));
+      assertFalse(range1.equals(range5));
+      assertFalse(range1.equals(range7));
+      assertFalse(range3.equals(range5));
+      assertFalse(range3.equals(range7));
+      assertFalse(range5.equals(range7));
+    }
+  }
+
+  @Nested
+  class CompareToTests {
+
+    @Test
+    void testCompareWithSameRange() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.open("r1", "r3");
+      assertEquals(0, range1.compareTo(range2));
+    }
+
+    @Test
+    void testCompareWithDifferentStartRow() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.open("r2", "r3");
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithDifferentEndRow() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.open("r1", "r4");
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithDifferentStartRowInclusiveness() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.closedOpen("r1", "r3");
+      assertTrue(range1.compareTo(range2) > 0);
+    }
+
+    @Test
+    void testCompareWithDifferentEndRowInclusiveness() {
+      RowRange range1 = RowRange.open("r1", "r3");
+      RowRange range2 = RowRange.openClosed("r1", "r3");
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithInfiniteStartRow() {
+      RowRange range1 = RowRange.atLeast("r1");
+      RowRange range2 = RowRange.all();
+      assertTrue(range1.compareTo(range2) < 0);
+    }
+
+    @Test
+    void testCompareWithInfiniteEndRow() {
+      RowRange range1 = RowRange.all();
+      RowRange range2 = RowRange.atLeast("r1");
+      assertTrue(range1.compareTo(range2) > 0);
+    }
+  }
+
+  @Nested
+  class ContainsTests {
+
+    @Test
+    void testContainsWithAllRange() {
+      RowRange range = RowRange.all();
+      assertTrue(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertTrue(range.contains(new Text("row3")));
+    }
+
+    @Test
+    void testContainsWithOpenRange() {
+      RowRange range = RowRange.open("r1", "r3");
+      assertFalse(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertFalse(range.contains(new Text("r3")));
+    }
+
+    @Test
+    void testContainsWithClosedRange() {
+      RowRange range = RowRange.closed("r1", "r3");
+      assertTrue(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertTrue(range.contains(new Text("r3")));
+    }
+
+    @Test
+    void testContainsWithOpenClosedRange() {
+      RowRange range = RowRange.openClosed("r1", "r3");
+      assertFalse(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertTrue(range.contains(new Text("r3")));
+    }
+
+    @Test
+    void testContainsWithClosedOpenRange() {
+      RowRange range = RowRange.closedOpen("r1", "r3");
+      assertTrue(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertFalse(range.contains(new Text("r3")));
+    }
+
+    @Test
+    void testContainsWithSingleRowRange() {
+      RowRange range = RowRange.closed("r1");
+      assertTrue(range.contains(new Text("r1")));
+      assertFalse(range.contains(new Text("r2")));
+    }
+
+    @Test
+    void testContainsWithAtLeastRange() {
+      RowRange range = RowRange.atLeast("r1");
+      assertTrue(range.contains(new Text("r1")));
+      assertTrue(range.contains(new Text("r2")));
+      assertFalse(range.contains(new Text("")));
+    }
+
+    @Test
+    void testContainsWithAtMostRange() {
+      RowRange range = RowRange.atMost("r1");
+      assertTrue(range.contains(new Text("r1")));
+      assertFalse(range.contains(new Text("r2")));
+      assertTrue(range.contains(new Text("")));
+    }
+  }
+
+  @TestInstance(TestInstance.Lifecycle.PER_CLASS)
+  @Nested
+  class MergeOverlappingTests {
+
+    @ParameterizedTest
+    @MethodSource({"rowRangeProvider", "rowRangeProvider1"})
+    public void testMergeOverlapping(List<RowRange> rowRangesToMerge, List<RowRange> expected) {

Review Comment:
   This is neat, I have not see this before.



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.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> {
+  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) {
+    return beforeStartRowImpl(row);
+  }
+
+  /**
+   * Determines if the given row is after the end row of this row range.
+   *
+   * @param row row to check
+   * @return true if the given row is after the row range, otherwise false
+   */
+  public boolean afterEndRow(Text row) {
+    if (infiniteEndRow) {
+      return false;
+    }
+
+    if (endRowInclusive) {
+      return row.compareTo(endRow) > 0;
+    }
+    return row.compareTo(endRow) >= 0;
+  }
+
+  /**
+   * Implements logic of {@link #beforeStartRow(Text)}, but in a private method, so that it can be
+   * safely used by constructors if a subclass overrides that {@link #beforeStartRow(Text)}
+   */
+  private boolean beforeStartRowImpl(Text row) {
+    if (this.infiniteStartRow) {
+      return false;
+    }
+
+    if (startRowInclusive) {
+      return row.compareTo(startRow) < 0;
+    }
+    return row.compareTo(startRow) <= 0;
+  }
+
+  @Override
+  public String toString() {
+    final String startRangeSymbol = (startRowInclusive && startRow != null) ? "[" : "(";
+    final String startRow = this.startRow == null ? "-inf" : this.startRow.toString();
+    final String endRow = this.endRow == null ? "+inf" : this.endRow.toString();
+    final String endRangeSymbol = (endRowInclusive && this.endRow != null) ? "]" : ")";
+    return startRangeSymbol + startRow + "," + endRow + endRangeSymbol;
+  }
+
+  @Override
+  public int hashCode() {
+    int startHash = infiniteStartRow ? 0 : startRow.hashCode() + (startRowInclusive ? 1 : 0);
+    int stopHash = infiniteEndRow ? 0 : endRow.hashCode() + (endRowInclusive ? 1 : 0);
+
+    return startHash + stopHash;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof RowRange)) {
+      return false;
+    }
+    return equals((RowRange) other);
+  }
+
+  /**
+   * Determines if this row range equals another.
+   *
+   * @param other range to compare
+   * @return true if row ranges are equals, false otherwise
+   */
+  public boolean equals(RowRange other) {
+    return compareTo(other) == 0;
+  }
+
+  /**
+   * Compares this row range to another row range. Compares in order: start row, inclusiveness of
+   * start row, end row, inclusiveness of end row. Infinite rows sort first, and non-infinite rows
+   * are compared with {@link Text#compareTo(BinaryComparable)}. Inclusive sorts before
+   * non-inclusive.
+   *
+   * @param other range row to compare
+   * @return comparison result
+   */
+  @Override
+  public int compareTo(RowRange other) {
+    // Compare infinite start rows
+    int comp = Boolean.compare(this.infiniteStartRow, other.infiniteStartRow);
+
+    if (comp == 0) {
+      // Compare non-infinite start rows and start row inclusiveness
+      if (!this.infiniteStartRow) {
+        if (this.startRow == null && other.startRow == null) {
+          comp = 0;
+        } else if (this.startRow == null) {
+          comp = -1;
+        } else if (other.startRow == null) {
+          comp = 1;
+        } else {
+          comp = this.startRow.compareTo(other.startRow);
+        }
+        if (comp == 0) {
+          comp = Boolean.compare(other.startRowInclusive, this.startRowInclusive);
+        }
+      }
+    }
+
+    if (comp == 0) {
+      // Compare infinite end rows
+      comp = Boolean.compare(this.infiniteEndRow, other.infiniteEndRow);
+
+      // Compare non-infinite end rows and end row inclusiveness
+      if (comp == 0 && !this.infiniteEndRow) {
+        if (this.endRow == null && other.endRow == null) {
+          comp = 0;
+        } else if (this.endRow == null) {
+          comp = 1;
+        } else if (other.endRow == null) {
+          comp = -1;
+        } else {
+          comp = this.endRow.compareTo(other.endRow);
+        }
+        if (comp == 0) {
+          comp = Boolean.compare(this.endRowInclusive, other.endRowInclusive);
+        }
+      }
+    }
+
+    return comp;
+  }
+
+  /**
+   * Determines if this row range contains the given row.
+   *
+   * @param row row to check
+   * @return true if the row is contained in the row range, false otherwise
+   */
+  public boolean contains(Text row) {
+    if (infiniteStartRow) {
+      return !afterEndRow(row);
+    } else if (infiniteEndRow) {
+      return !beforeStartRow(row);
+    } else {
+      return !beforeStartRow(row) && !afterEndRow(row);
+    }
+  }
+
+  /**
+   * Merges overlapping and adjacent row ranges. For example, given the following input:
+   *
+   * <pre>
+   * [a,c], (c, d], (g,m), (j,t]
+   * </pre>
+   *
+   * the following row ranges would be returned:
+   *
+   * <pre>
+   * [a,d], (g,t]
+   * </pre>
+   *
+   * @param rowRanges the collection of row ranges to merge
+   * @return a list of merged row ranges
+   */
+  public static List<RowRange> mergeOverlapping(Collection<RowRange> rowRanges) {
+    if (rowRanges.isEmpty()) {
+      return Collections.emptyList();
+    }
+    if (rowRanges.size() == 1) {
+      return Collections.singletonList(rowRanges.iterator().next());
+    }
+
+    List<RowRange> sortedRowRanges = new ArrayList<>(rowRanges);
+    // Sort row ranges by their startRow values
+    sortedRowRanges.sort((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);
+    });
+
+    ArrayList<RowRange> mergedRowRanges = new ArrayList<>(rowRanges.size());
+
+    // Initialize the current range for merging
+    RowRange currentRange = sortedRowRanges.get(0);
+    boolean currentStartRowInclusive = sortedRowRanges.get(0).startRowInclusive;
+
+    // Iterate through the sorted row ranges, merging overlapping and adjacent ranges
+    for (int i = 1; i < sortedRowRanges.size(); i++) {
+      if (currentRange.infiniteStartRow && currentRange.infiniteEndRow) {
+        // The current range covers all possible rows, no further merging needed
+        break;
+      }
+
+      RowRange nextRange = sortedRowRanges.get(i);
+
+      if (nextRange.infiniteStartRow && nextRange.infiniteEndRow) {
+        // The next range covers all possible rows, set the current range to cover all rows
+        currentRange = RowRange.all();
+        break;
+      }
+
+      boolean startRowsEqual = (currentRange.startRow == null && nextRange.startRow == null)
+          || (currentRange.startRow != null && currentRange.startRow.equals(nextRange.startRow));
+
+      int comparison;
+      if (startRowsEqual || currentRange.infiniteEndRow
+          || (nextRange.startRow != null && (currentRange.endRow == null
+              || currentRange.endRow.compareTo(nextRange.startRow) > 0
+              || (currentRange.endRow.equals(nextRange.startRow)
+                  && (!currentRange.endRowInclusive || nextRange.startRowInclusive))))) {
+        if (nextRange.infiniteEndRow) {
+          comparison = 1;
+        } else if (currentRange.endRow == null) {
+          comparison = -1;
+        } else {
+          comparison = nextRange.endRow.compareTo(currentRange.endRow);
+        }
+        if (comparison > 0 || (comparison == 0 && nextRange.endRowInclusive)) {
+          currentRange = RowRange.range(currentRange.startRow, currentStartRowInclusive,
+              nextRange.endRow, nextRange.endRowInclusive);
+        } // else current range contains the next range
+      } else {
+        // No overlap or adjacency, add the current range to the merged list and update the current
+        // range
+        mergedRowRanges.add(currentRange);
+        currentRange = nextRange;
+        currentStartRowInclusive = nextRange.startRowInclusive;
+      }
+    }
+
+    // Add the final current range to the merged list
+    mergedRowRanges.add(currentRange);
+
+    return mergedRowRanges;
+  }
+
+  /**
+   * Creates a row range which represents the intersection of this row range and the passed in row
+   * range. The following example will print true.
+   *
+   * <pre>
+   * RowRange rowRange1 = RowRange.closed(&quot;a&quot;, &quot;f&quot;);
+   * RowRange rowRange2 = RowRange.closed(&quot;c&quot;, &quot;n&quot;);
+   * RowRange rowRange3 = rowRange1.clip(rowRange2);
+   * System.out.println(rowRange3.equals(RowRange.closed(&quot;c&quot;, &quot;f&quot;)));
+   * </pre>
+   *
+   * @param rowRange row range to clip to
+   * @return the intersection of this row range and the given row range
+   * @throws IllegalArgumentException if row ranges do not overlap
+   */
+  public RowRange clip(RowRange rowRange) {
+    return clip(rowRange, false);
+  }
+
+  /**
+   * Creates a row range which represents the intersection of this row range and the passed in row
+   * range. Unlike {@link #clip(RowRange)}, this method can optionally return null if the row ranges
+   * do not overlap, instead of throwing an exception. The returnNullIfDisjoint parameter controls
+   * this behavior.
+   *
+   * @param rowRange row range to clip to
+   * @param returnNullIfDisjoint true to return null if row ranges are disjoint, false to throw an
+   *        exception
+   * @return the intersection of this row range and the given row range, or null if row ranges do
+   *         not overlap and returnNullIfDisjoint is true
+   * @throws IllegalArgumentException if row ranges do not overlap and returnNullIfDisjoint is false
+   * @see #clip(RowRange)
+   */
+  public RowRange clip(RowRange rowRange, boolean returnNullIfDisjoint) {
+    // Initialize start and end row values with the current instance's values
+    Text startRow = this.startRow;
+    boolean startRowInclusive = this.startRowInclusive;
+    Text endRow = this.endRow;
+    boolean endRowInclusive = this.endRowInclusive;
+
+    // If the input rowRange has a defined startRow, update startRow and startRowInclusive if needed
+    if (rowRange.startRow != null) {
+      // If the input rowRange's startRow is after this instance's endRow or equal but not
+      // inclusive, they do not overlap
+      if (afterEndRow(rowRange.startRow) || (rowRange.startRow.equals(this.endRow)
+          && !(rowRange.startRowInclusive && this.endRowInclusive))) {
+        if (returnNullIfDisjoint) {
+          return null;
+        }
+        throw new IllegalArgumentException("RowRange " + rowRange + " does not overlap " + this);
+      } else if (!beforeStartRow(rowRange.startRow)) {
+        // If the input rowRange's startRow is within this instance's range, use it as the new
+        // startRow
+        startRow = rowRange.startRow;
+        startRowInclusive = rowRange.startRowInclusive;
+      }
+    }
+
+    // If the input rowRange has a defined endRow, update endRow and endRowInclusive if needed
+    if (rowRange.endRow != null) {
+      // If the input rowRange's endRow is before this instance's startRow or equal but not
+      // inclusive, they do not overlap
+      if (beforeStartRow(rowRange.endRow) || (rowRange.endRow.equals(this.startRow)
+          && !(rowRange.endRowInclusive && this.startRowInclusive))) {
+        if (returnNullIfDisjoint) {
+          return null;

Review Comment:
   This was not covered by unit test



-- 
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


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

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1177071781


##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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 org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ */
+public class RowRange {
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  private final boolean infiniteStartRow;
+  private final boolean infiniteEndRow;
+
+  /**
+   * Creates a range of rows that goes from negative to positive infinity
+   */

Review Comment:
   This is a common way we've expressed this, but it's technically incorrect. The range goes from an empty byte array to an infinitely large byte array of all 1 bits. We should try to find a better way to express this succinctly.
   
   Maybe "from null to positive infinity" or "from no bytes to positive infinity" or "range that includes all possible rows".



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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 org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ */
+public class RowRange {
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  private final boolean infiniteStartRow;
+  private final boolean infiniteEndRow;
+
+  /**
+   * Creates a range of rows that goes from negative to positive infinity
+   */
+  public RowRange() {
+    this(null, true, null, true);
+  }
+
+  /**
+   * Creates a range of rows from startRow inclusive to endRow inclusive.
+   *
+   * @param startRow starting row; set to null for negative infinity

Review Comment:
   ```suggestion
      * @param startRow starting row; set to null for the smallest possible row (an empty one)
   ```
   
   Instead of overloading constructors, you could have static entry points to more clearly express bounded and unbounded ranges. See [Guava's Range API](https://guava.dev/releases/19.0/api/docs/com/google/common/collect/Range.html) for a really good example of static entry points with phrases like "atLeast(lowerBound)", or "atMost(upperBound)" or "between(lowerBound,upperBound)". If we're adding any new Range API, these would be really useful to have right up front, rather than trying to add them in later.



-- 
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


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

Posted by "DomGarguilo (via GitHub)" <gi...@apache.org>.
DomGarguilo commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1177861209


##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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 org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ */
+public class RowRange {
+  final private Text startRow;
+  final private Text endRow;
+  final private boolean startRowInclusive;
+  final private boolean endRowInclusive;
+  private final boolean infiniteStartRow;
+  private final boolean infiniteEndRow;
+
+  /**
+   * Creates a range of rows that goes from negative to positive infinity
+   */

Review Comment:
   > "range that includes all possible rows"
   
   To me, this suggestion makes it the most clear so I might go with that one unless anyone else has another opinion.
   
   



-- 
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


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

Posted by "DomGarguilo (via GitHub)" <gi...@apache.org>.
DomGarguilo commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1185367053


##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.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> {
+  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) {
+    return beforeStartRowImpl(row);
+  }
+
+  /**
+   * Determines if the given row is after the end row of this row range.
+   *
+   * @param row row to check
+   * @return true if the given row is after the row range, otherwise false
+   */
+  public boolean afterEndRow(Text row) {
+    if (infiniteEndRow) {
+      return false;
+    }
+
+    if (endRowInclusive) {
+      return row.compareTo(endRow) > 0;
+    }
+    return row.compareTo(endRow) >= 0;
+  }
+
+  /**
+   * Implements logic of {@link #beforeStartRow(Text)}, but in a private method, so that it can be
+   * safely used by constructors if a subclass overrides that {@link #beforeStartRow(Text)}
+   */
+  private boolean beforeStartRowImpl(Text row) {
+    if (this.infiniteStartRow) {
+      return false;
+    }
+
+    if (startRowInclusive) {
+      return row.compareTo(startRow) < 0;
+    }
+    return row.compareTo(startRow) <= 0;
+  }
+
+  @Override
+  public String toString() {
+    final String startRangeSymbol = (startRowInclusive && startRow != null) ? "[" : "(";
+    final String startRow = this.startRow == null ? "-inf" : this.startRow.toString();
+    final String endRow = this.endRow == null ? "+inf" : this.endRow.toString();
+    final String endRangeSymbol = (endRowInclusive && this.endRow != null) ? "]" : ")";
+    return startRangeSymbol + startRow + "," + endRow + endRangeSymbol;
+  }
+
+  @Override
+  public int hashCode() {
+    int startHash = infiniteStartRow ? 0 : startRow.hashCode() + (startRowInclusive ? 1 : 0);
+    int stopHash = infiniteEndRow ? 0 : endRow.hashCode() + (endRowInclusive ? 1 : 0);
+
+    return startHash + stopHash;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof RowRange)) {
+      return false;
+    }
+    return equals((RowRange) other);
+  }
+
+  /**
+   * Determines if this row range equals another.
+   *
+   * @param other range to compare
+   * @return true if row ranges are equals, false otherwise
+   */
+  public boolean equals(RowRange other) {
+    return compareTo(other) == 0;
+  }
+
+  /**
+   * Compares this row range to another row range. Compares in order: start row, inclusiveness of
+   * start row, end row, inclusiveness of end row. Infinite rows sort first, and non-infinite rows
+   * are compared with {@link Text#compareTo(BinaryComparable)}. Inclusive sorts before
+   * non-inclusive.
+   *
+   * @param other range row to compare
+   * @return comparison result
+   */
+  @Override
+  public int compareTo(RowRange other) {
+    // Compare infinite start rows
+    int comp = Boolean.compare(this.infiniteStartRow, other.infiniteStartRow);
+
+    if (comp == 0) {
+      // Compare non-infinite start rows and start row inclusiveness
+      if (!this.infiniteStartRow) {
+        if (this.startRow == null && other.startRow == null) {
+          comp = 0;
+        } else if (this.startRow == null) {
+          comp = -1;
+        } else if (other.startRow == null) {
+          comp = 1;
+        } else {
+          comp = this.startRow.compareTo(other.startRow);
+        }
+        if (comp == 0) {
+          comp = Boolean.compare(other.startRowInclusive, this.startRowInclusive);
+        }
+      }
+    }
+
+    if (comp == 0) {
+      // Compare infinite end rows
+      comp = Boolean.compare(this.infiniteEndRow, other.infiniteEndRow);
+
+      // Compare non-infinite end rows and end row inclusiveness
+      if (comp == 0 && !this.infiniteEndRow) {
+        if (this.endRow == null && other.endRow == null) {
+          comp = 0;
+        } else if (this.endRow == null) {
+          comp = 1;
+        } else if (other.endRow == null) {
+          comp = -1;
+        } else {
+          comp = this.endRow.compareTo(other.endRow);
+        }
+        if (comp == 0) {
+          comp = Boolean.compare(this.endRowInclusive, other.endRowInclusive);
+        }
+      }
+    }
+
+    return comp;
+  }
+
+  /**
+   * Determines if this row range contains the given row.
+   *
+   * @param row row to check
+   * @return true if the row is contained in the row range, false otherwise
+   */
+  public boolean contains(Text row) {
+    if (infiniteStartRow) {
+      return !afterEndRow(row);
+    } else if (infiniteEndRow) {
+      return !beforeStartRow(row);
+    } else {
+      return !beforeStartRow(row) && !afterEndRow(row);
+    }
+  }
+
+  /**
+   * Merges overlapping and adjacent row ranges. For example, given the following input:
+   *
+   * <pre>
+   * [a,c], (c, d], (g,m), (j,t]
+   * </pre>
+   *
+   * the following row ranges would be returned:
+   *
+   * <pre>
+   * [a,d], (g,t]
+   * </pre>
+   *
+   * @param rowRanges the collection of row ranges to merge
+   * @return a list of merged row ranges
+   */
+  public static List<RowRange> mergeOverlapping(Collection<RowRange> rowRanges) {
+    if (rowRanges.isEmpty()) {
+      return Collections.emptyList();
+    }
+    if (rowRanges.size() == 1) {
+      return Collections.singletonList(rowRanges.iterator().next());
+    }
+
+    List<RowRange> sortedRowRanges = new ArrayList<>(rowRanges);
+    // Sort row ranges by their startRow values
+    sortedRowRanges.sort((r1, r2) -> {
+      if (r1.startRow == null && r2.startRow == null) {

Review Comment:
   I moved this comparator to a constant like you suggested for the others in 3c329d7. But I could not get it working when trying to use the `START_ROW_COMPARATOR`. I'm not sure if I was incorporating it incorrectly into the new Comparator or what but I couldnt get any version working without causing some test failures.
   



-- 
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


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

Posted by "DomGarguilo (via GitHub)" <gi...@apache.org>.
DomGarguilo commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1180667689


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -1490,7 +1490,7 @@ public Map<String,String> tableIdMap() {
   public Text getMaxRow(String tableName, Authorizations auths, Text startRow,
       boolean startInclusive, Text endRow, boolean endInclusive) throws TableNotFoundException {
     return getMaxRow(tableName, auths,
-        new RowRange(startRow, startInclusive, endRow, endInclusive));
+        RowRange.create(startRow, startInclusive, endRow, endInclusive));

Review Comment:
   Addressed in a527552



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -45,8 +45,91 @@ public RowRange() {
    * @param endRow ending row; set to null for positive infinity
    * @throws IllegalArgumentException if end row is before start row
    */
-  public RowRange(Text startRow, Text endRow) {
-    this(startRow, true, endRow, true);
+  public static RowRange open(Text startRow, Text endRow) {

Review Comment:
   Addressed in a527552



-- 
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


[GitHub] [accumulo] keith-turner commented on a diff in pull request #3342: WIP - Add RowRange object

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1180632249


##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -45,8 +45,91 @@ public RowRange() {
    * @param endRow ending row; set to null for positive infinity
    * @throws IllegalArgumentException if end row is before start row
    */
-  public RowRange(Text startRow, Text endRow) {
-    this(startRow, true, endRow, true);
+  public static RowRange open(Text startRow, Text endRow) {

Review Comment:
   Charsequence versions of the static methods would be nice.



-- 
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


[GitHub] [accumulo] keith-turner commented on a diff in pull request #3342: WIP - Add RowRange object

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1185495088


##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,641 @@
+/*
+ * 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);

Review Comment:
   Looking at this comparison I see how it differs.  It compares the start row for null, then if those are both non-null it compares the entire range. I missed that at the end, I thought it was consistently comparing the start row throughout.



-- 
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


[GitHub] [accumulo] cshannon commented on pull request #3342: Add RowRange object

Posted by "cshannon (via GitHub)" <gi...@apache.org>.
cshannon commented on PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#issuecomment-1537162205

   Looking at this it would be really nice if [Range](https://github.com/apache/accumulo/blob/56d49f15a05db9a46dbceb845918497760601c11/core/src/main/java/org/apache/accumulo/core/data/Range.java#L39) and `RowRange` could share code. Two ideas I had:
   
   1. I am wondering if it makes more sense to refactor `Range` to extend `RowRange`. At first glance it seems like the two should be able to share a lot of code as the main difference is comparing Text start/end row vs Key start/end otherwise the core methods/logic for the Range should be pretty similar. But again, I haven't looked in detail or thought through it all the way so maybe there are other reasons to keep it separate.
   
   2. Another option, which I like but may not work if we consider Range public API, is to refactor things and rename the existing Range object to KeyRange and have a more generic abstract Range object and then have `RowRange` and `KeyRange` extend that and share code. Having two descriptive objects of `RowRange` and `KeyRange` and have them share a common abstract class would be quite nice.
   
   


-- 
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


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

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1191917352


##########
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:
   Ah. Making it private is fine. I thought maybe you wanted to make this available to users.



-- 
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


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

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
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


[GitHub] [accumulo] DomGarguilo commented on pull request #3342: Add RowRange object

Posted by "DomGarguilo (via GitHub)" <gi...@apache.org>.
DomGarguilo commented on PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#issuecomment-1544283227

   @ctubbsii, I tried to resolve all of your suggestions in 6a60c98


-- 
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


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

Posted by "DomGarguilo (via GitHub)" <gi...@apache.org>.
DomGarguilo commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1186343155


##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.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> {
+  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) {
+    return beforeStartRowImpl(row);
+  }
+
+  /**
+   * Determines if the given row is after the end row of this row range.
+   *
+   * @param row row to check
+   * @return true if the given row is after the row range, otherwise false
+   */
+  public boolean afterEndRow(Text row) {
+    if (infiniteEndRow) {
+      return false;
+    }
+
+    if (endRowInclusive) {
+      return row.compareTo(endRow) > 0;
+    }
+    return row.compareTo(endRow) >= 0;
+  }
+
+  /**
+   * Implements logic of {@link #beforeStartRow(Text)}, but in a private method, so that it can be
+   * safely used by constructors if a subclass overrides that {@link #beforeStartRow(Text)}
+   */
+  private boolean beforeStartRowImpl(Text row) {
+    if (this.infiniteStartRow) {
+      return false;
+    }
+
+    if (startRowInclusive) {
+      return row.compareTo(startRow) < 0;
+    }
+    return row.compareTo(startRow) <= 0;
+  }
+
+  @Override
+  public String toString() {
+    final String startRangeSymbol = (startRowInclusive && startRow != null) ? "[" : "(";
+    final String startRow = this.startRow == null ? "-inf" : this.startRow.toString();
+    final String endRow = this.endRow == null ? "+inf" : this.endRow.toString();
+    final String endRangeSymbol = (endRowInclusive && this.endRow != null) ? "]" : ")";
+    return startRangeSymbol + startRow + "," + endRow + endRangeSymbol;
+  }
+
+  @Override
+  public int hashCode() {
+    int startHash = infiniteStartRow ? 0 : startRow.hashCode() + (startRowInclusive ? 1 : 0);
+    int stopHash = infiniteEndRow ? 0 : endRow.hashCode() + (endRowInclusive ? 1 : 0);
+
+    return startHash + stopHash;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof RowRange)) {
+      return false;

Review Comment:
   I added the tests you suggested as well as some others and the class now has 100% test coverage.



-- 
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


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

Posted by "DomGarguilo (via GitHub)" <gi...@apache.org>.
DomGarguilo commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1180724330


##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import org.apache.accumulo.core.dataImpl.thrift.TRowRange;
+import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.core.util.TextUtil;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.
+ */
+public class RowRange {

Review Comment:
   adfc5cd



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import org.apache.accumulo.core.dataImpl.thrift.TRowRange;
+import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.core.util.TextUtil;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This class is used to specify a range of rows.

Review Comment:
   adfc5cd



-- 
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


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

Posted by "DomGarguilo (via GitHub)" <gi...@apache.org>.
DomGarguilo commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1185362681


##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -0,0 +1,650 @@
+/*
+ * 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.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> {
+  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) {
+    return beforeStartRowImpl(row);
+  }
+
+  /**
+   * Determines if the given row is after the end row of this row range.
+   *
+   * @param row row to check
+   * @return true if the given row is after the row range, otherwise false
+   */
+  public boolean afterEndRow(Text row) {
+    if (infiniteEndRow) {
+      return false;
+    }
+
+    if (endRowInclusive) {
+      return row.compareTo(endRow) > 0;
+    }
+    return row.compareTo(endRow) >= 0;
+  }
+
+  /**
+   * Implements logic of {@link #beforeStartRow(Text)}, but in a private method, so that it can be
+   * safely used by constructors if a subclass overrides that {@link #beforeStartRow(Text)}
+   */
+  private boolean beforeStartRowImpl(Text row) {
+    if (this.infiniteStartRow) {
+      return false;
+    }
+
+    if (startRowInclusive) {
+      return row.compareTo(startRow) < 0;
+    }
+    return row.compareTo(startRow) <= 0;
+  }
+
+  @Override
+  public String toString() {
+    final String startRangeSymbol = (startRowInclusive && startRow != null) ? "[" : "(";
+    final String startRow = this.startRow == null ? "-inf" : this.startRow.toString();
+    final String endRow = this.endRow == null ? "+inf" : this.endRow.toString();
+    final String endRangeSymbol = (endRowInclusive && this.endRow != null) ? "]" : ")";
+    return startRangeSymbol + startRow + "," + endRow + endRangeSymbol;
+  }
+
+  @Override
+  public int hashCode() {
+    int startHash = infiniteStartRow ? 0 : startRow.hashCode() + (startRowInclusive ? 1 : 0);
+    int stopHash = infiniteEndRow ? 0 : endRow.hashCode() + (endRowInclusive ? 1 : 0);
+
+    return startHash + stopHash;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof RowRange)) {
+      return false;
+    }
+    return equals((RowRange) other);
+  }
+
+  /**
+   * Determines if this row range equals another.
+   *
+   * @param other range to compare
+   * @return true if row ranges are equals, false otherwise
+   */
+  public boolean equals(RowRange other) {
+    return compareTo(other) == 0;
+  }
+
+  /**
+   * Compares this row range to another row range. Compares in order: start row, inclusiveness of
+   * start row, end row, inclusiveness of end row. Infinite rows sort first, and non-infinite rows
+   * are compared with {@link Text#compareTo(BinaryComparable)}. Inclusive sorts before
+   * non-inclusive.
+   *
+   * @param other range row to compare
+   * @return comparison result
+   */
+  @Override
+  public int compareTo(RowRange other) {
+    // Compare infinite start rows
+    int comp = Boolean.compare(this.infiniteStartRow, other.infiniteStartRow);
+
+    if (comp == 0) {
+      // Compare non-infinite start rows and start row inclusiveness
+      if (!this.infiniteStartRow) {
+        if (this.startRow == null && other.startRow == null) {
+          comp = 0;
+        } else if (this.startRow == null) {
+          comp = -1;
+        } else if (other.startRow == null) {
+          comp = 1;
+        } else {
+          comp = this.startRow.compareTo(other.startRow);
+        }
+        if (comp == 0) {
+          comp = Boolean.compare(other.startRowInclusive, this.startRowInclusive);
+        }
+      }
+    }
+
+    if (comp == 0) {
+      // Compare infinite end rows
+      comp = Boolean.compare(this.infiniteEndRow, other.infiniteEndRow);
+
+      // Compare non-infinite end rows and end row inclusiveness
+      if (comp == 0 && !this.infiniteEndRow) {
+        if (this.endRow == null && other.endRow == null) {
+          comp = 0;
+        } else if (this.endRow == null) {
+          comp = 1;
+        } else if (other.endRow == null) {
+          comp = -1;
+        } else {
+          comp = this.endRow.compareTo(other.endRow);
+        }
+        if (comp == 0) {
+          comp = Boolean.compare(this.endRowInclusive, other.endRowInclusive);
+        }
+      }
+    }

Review Comment:
   Added in 9f51966



-- 
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


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

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1191917352


##########
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:
   Ah. Making it private is fine. I thought maybe you wanted to make this available to users, but users can just do:
   `Comparator.comparing(RowRange::compareTo)`, so they don't need this public.



-- 
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


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

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on code in PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#discussion_r1179735010


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java:
##########
@@ -1490,7 +1490,7 @@ public Map<String,String> tableIdMap() {
   public Text getMaxRow(String tableName, Authorizations auths, Text startRow,
       boolean startInclusive, Text endRow, boolean endInclusive) throws TableNotFoundException {
     return getMaxRow(tableName, auths,
-        new RowRange(startRow, startInclusive, endRow, endInclusive));
+        RowRange.create(startRow, startInclusive, endRow, endInclusive));

Review Comment:
   The equivalent Guava API would call this "range" instead of "create". And they also use "open" and "closed" instead of "inclusive" and its negation. I think it would be good to use the "range" method name here, but keep our use of "inclusive" that matches with the key Range API.



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -45,8 +45,91 @@ public RowRange() {
    * @param endRow ending row; set to null for positive infinity
    * @throws IllegalArgumentException if end row is before start row
    */
-  public RowRange(Text startRow, Text endRow) {
-    this(startRow, true, endRow, true);
+  public static RowRange open(Text startRow, Text endRow) {
+    return create(startRow, true, endRow, 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 closed(Text startRow, Text endRow) {
+    return create(startRow, false, endRow, false);
+  }

Review Comment:
   This is incorrect. Closed means "inclusive", not "exclusive". Open means "exclusive". This is backwards throughout.



##########
core/src/main/java/org/apache/accumulo/core/data/RowRange.java:
##########
@@ -45,8 +45,91 @@ public RowRange() {
    * @param endRow ending row; set to null for positive infinity
    * @throws IllegalArgumentException if end row is before start row
    */
-  public RowRange(Text startRow, Text endRow) {
-    this(startRow, true, endRow, true);
+  public static RowRange open(Text startRow, Text endRow) {

Review Comment:
   It kinda sucks that we're creating a new API with Text, as it would be nice to try to phase out that dependency on Hadoop types in our APIs. But, we don't have an alternative right now. You may want to consider offering overloaded methods for CharSequence, though, so users aren't forced to use Text if their endpoints are human-readable as Strings.



-- 
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


[GitHub] [accumulo] ctubbsii commented on pull request #3342: WIP - Add RowRange object

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on PR #3342:
URL: https://github.com/apache/accumulo/pull/3342#issuecomment-1526630191

   When this is done, the key Range object could also be a improved with some similar API changes. Although, we might want to wait until we have a better ByteSequence public API to use instead of Text so we're not churning the old API so much.


-- 
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