You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by gu...@apache.org on 2022/12/08 15:51:13 UTC

[lucene] branch main updated: Use ByteArrayComparator to replace Arrays#compareUnsigned in some other places (#11880)

This is an automated email from the ASF dual-hosted git repository.

guofeng pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/lucene.git


The following commit(s) were added to refs/heads/main by this push:
     new 9ff989ec001 Use ByteArrayComparator to replace Arrays#compareUnsigned in some other places (#11880)
9ff989ec001 is described below

commit 9ff989ec001c97e031e630d390905e7831984b64
Author: gf2121 <52...@users.noreply.github.com>
AuthorDate: Thu Dec 8 23:51:08 2022 +0800

    Use ByteArrayComparator to replace Arrays#compareUnsigned in some other places (#11880)
---
 lucene/CHANGES.txt                                 |   3 +
 .../document/BinaryRangeFieldRangeQuery.java       |  10 +-
 .../document/LatLonPointDistanceFeatureQuery.java  | 130 ++--------
 .../apache/lucene/document/RangeFieldQuery.java    | 268 ++++++++-------------
 .../java/org/apache/lucene/index/CheckIndex.java   |  70 +-----
 5 files changed, 143 insertions(+), 338 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 2b4836a8c41..8bc6891efd8 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -200,6 +200,9 @@ Optimizations
 * GITHUB#11876: Use ByteArrayComparator to speed up PointInSetQuery in single dimension case.
   (Guo Feng)
 
+* GITHUB#11880: Use ByteArrayComparator to speed up BinaryRangeFieldRangeQuery, RangeFieldQuery
+  LatLonPointDistanceFeatureQuery and CheckIndex. (Guo Feng)
+
 * GITHUB#11881: Further optimize drill-sideways scoring by specializing the single dimension case
   and borrowing some concepts from "min should match" scoring. (Greg Miller)
 
diff --git a/lucene/core/src/java/org/apache/lucene/document/BinaryRangeFieldRangeQuery.java b/lucene/core/src/java/org/apache/lucene/document/BinaryRangeFieldRangeQuery.java
index 621c55f65b0..b225f4d98fe 100644
--- a/lucene/core/src/java/org/apache/lucene/document/BinaryRangeFieldRangeQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/document/BinaryRangeFieldRangeQuery.java
@@ -32,11 +32,14 @@ import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.TwoPhaseIterator;
 import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.ArrayUtil.ByteArrayComparator;
 
 abstract class BinaryRangeFieldRangeQuery extends Query {
   private final String field;
   private byte[] queryPackedValue;
   private final int numBytesPerDimension;
+  private final ByteArrayComparator comparator;
   private final int numDims;
   private final RangeFieldQuery.QueryType queryType;
 
@@ -49,6 +52,7 @@ abstract class BinaryRangeFieldRangeQuery extends Query {
     this.field = field;
     this.queryPackedValue = queryPackedValue;
     this.numBytesPerDimension = numBytesPerDimension;
+    this.comparator = ArrayUtil.getUnsignedComparator(numBytesPerDimension);
     this.numDims = numDims;
 
     if (!(queryType == RangeFieldQuery.QueryType.INTERSECTS)) {
@@ -119,7 +123,11 @@ abstract class BinaryRangeFieldRangeQuery extends Query {
               @Override
               public boolean matches() {
                 return queryType.matches(
-                    queryPackedValue, values.getPackedValue(), numDims, numBytesPerDimension);
+                    queryPackedValue,
+                    values.getPackedValue(),
+                    numDims,
+                    numBytesPerDimension,
+                    comparator);
               }
 
               @Override
diff --git a/lucene/core/src/java/org/apache/lucene/document/LatLonPointDistanceFeatureQuery.java b/lucene/core/src/java/org/apache/lucene/document/LatLonPointDistanceFeatureQuery.java
index 8508cb13761..beadd92f31c 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LatLonPointDistanceFeatureQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LatLonPointDistanceFeatureQuery.java
@@ -17,7 +17,6 @@
 package org.apache.lucene.document;
 
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.Objects;
 import org.apache.lucene.geo.GeoEncodingUtils;
 import org.apache.lucene.geo.GeoUtils;
@@ -38,6 +37,7 @@ import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.ScorerSupplier;
 import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.DocIdSetBuilder;
 import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.SloppyMath;
@@ -425,53 +425,21 @@ final class LatLonPointDistanceFeatureQuery extends Query {
                 // Already visited or skipped
                 return;
               }
-              if (Arrays.compareUnsigned(
-                          packedValue, 0, LatLonPoint.BYTES, maxLat, 0, LatLonPoint.BYTES)
-                      > 0
-                  || Arrays.compareUnsigned(
-                          packedValue, 0, LatLonPoint.BYTES, minLat, 0, LatLonPoint.BYTES)
-                      < 0) {
+              if (ArrayUtil.compareUnsigned4(packedValue, 0, maxLat, 0) > 0
+                  || ArrayUtil.compareUnsigned4(packedValue, 0, minLat, 0) < 0) {
                 // Latitude out of range
                 return;
               }
               if (crossDateLine) {
-                if (Arrays.compareUnsigned(
-                            packedValue,
-                            LatLonPoint.BYTES,
-                            2 * LatLonPoint.BYTES,
-                            minLon,
-                            0,
-                            LatLonPoint.BYTES)
-                        < 0
-                    && Arrays.compareUnsigned(
-                            packedValue,
-                            LatLonPoint.BYTES,
-                            2 * LatLonPoint.BYTES,
-                            maxLon,
-                            0,
-                            LatLonPoint.BYTES)
-                        > 0) {
+                if (ArrayUtil.compareUnsigned4(packedValue, LatLonPoint.BYTES, minLon, 0) < 0
+                    && ArrayUtil.compareUnsigned4(packedValue, LatLonPoint.BYTES, maxLon, 0) > 0) {
                   // Longitude out of range
                   return;
                 }
 
               } else {
-                if (Arrays.compareUnsigned(
-                            packedValue,
-                            LatLonPoint.BYTES,
-                            2 * LatLonPoint.BYTES,
-                            maxLon,
-                            0,
-                            LatLonPoint.BYTES)
-                        > 0
-                    || Arrays.compareUnsigned(
-                            packedValue,
-                            LatLonPoint.BYTES,
-                            2 * LatLonPoint.BYTES,
-                            minLon,
-                            0,
-                            LatLonPoint.BYTES)
-                        < 0) {
+                if (ArrayUtil.compareUnsigned4(packedValue, LatLonPoint.BYTES, maxLon, 0) > 0
+                    || ArrayUtil.compareUnsigned4(packedValue, LatLonPoint.BYTES, minLon, 0) < 0) {
                   // Longitude out of range
                   return;
                 }
@@ -482,94 +450,34 @@ final class LatLonPointDistanceFeatureQuery extends Query {
             @Override
             public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
 
-              if (Arrays.compareUnsigned(
-                          minPackedValue, 0, LatLonPoint.BYTES, maxLat, 0, LatLonPoint.BYTES)
-                      > 0
-                  || Arrays.compareUnsigned(
-                          maxPackedValue, 0, LatLonPoint.BYTES, minLat, 0, LatLonPoint.BYTES)
-                      < 0) {
+              if (ArrayUtil.compareUnsigned4(minPackedValue, 0, maxLat, 0) > 0
+                  || ArrayUtil.compareUnsigned4(maxPackedValue, 0, minLat, 0) < 0) {
                 return Relation.CELL_OUTSIDE_QUERY;
               }
               boolean crosses =
-                  Arrays.compareUnsigned(
-                              minPackedValue, 0, LatLonPoint.BYTES, minLat, 0, LatLonPoint.BYTES)
-                          < 0
-                      || Arrays.compareUnsigned(
-                              maxPackedValue, 0, LatLonPoint.BYTES, maxLat, 0, LatLonPoint.BYTES)
-                          > 0;
+                  ArrayUtil.compareUnsigned4(minPackedValue, 0, minLat, 0) < 0
+                      || ArrayUtil.compareUnsigned4(maxPackedValue, 0, maxLat, 0) > 0;
 
               if (crossDateLine) {
-                if (Arrays.compareUnsigned(
-                            minPackedValue,
-                            LatLonPoint.BYTES,
-                            2 * LatLonPoint.BYTES,
-                            maxLon,
-                            0,
-                            LatLonPoint.BYTES)
-                        > 0
-                    && Arrays.compareUnsigned(
-                            maxPackedValue,
-                            LatLonPoint.BYTES,
-                            2 * LatLonPoint.BYTES,
-                            minLon,
-                            0,
-                            LatLonPoint.BYTES)
+                if (ArrayUtil.compareUnsigned4(minPackedValue, LatLonPoint.BYTES, maxLon, 0) > 0
+                    && ArrayUtil.compareUnsigned4(maxPackedValue, LatLonPoint.BYTES, minLon, 0)
                         < 0) {
                   return Relation.CELL_OUTSIDE_QUERY;
                 }
                 crosses |=
-                    Arrays.compareUnsigned(
-                                minPackedValue,
-                                LatLonPoint.BYTES,
-                                2 * LatLonPoint.BYTES,
-                                maxLon,
-                                0,
-                                LatLonPoint.BYTES)
-                            < 0
-                        || Arrays.compareUnsigned(
-                                maxPackedValue,
-                                LatLonPoint.BYTES,
-                                2 * LatLonPoint.BYTES,
-                                minLon,
-                                0,
-                                LatLonPoint.BYTES)
+                    ArrayUtil.compareUnsigned4(minPackedValue, LatLonPoint.BYTES, maxLon, 0) < 0
+                        || ArrayUtil.compareUnsigned4(maxPackedValue, LatLonPoint.BYTES, minLon, 0)
                             > 0;
 
               } else {
-                if (Arrays.compareUnsigned(
-                            minPackedValue,
-                            LatLonPoint.BYTES,
-                            2 * LatLonPoint.BYTES,
-                            maxLon,
-                            0,
-                            LatLonPoint.BYTES)
-                        > 0
-                    || Arrays.compareUnsigned(
-                            maxPackedValue,
-                            LatLonPoint.BYTES,
-                            2 * LatLonPoint.BYTES,
-                            minLon,
-                            0,
-                            LatLonPoint.BYTES)
+                if (ArrayUtil.compareUnsigned4(minPackedValue, LatLonPoint.BYTES, maxLon, 0) > 0
+                    || ArrayUtil.compareUnsigned4(maxPackedValue, LatLonPoint.BYTES, minLon, 0)
                         < 0) {
                   return Relation.CELL_OUTSIDE_QUERY;
                 }
                 crosses |=
-                    Arrays.compareUnsigned(
-                                minPackedValue,
-                                LatLonPoint.BYTES,
-                                2 * LatLonPoint.BYTES,
-                                minLon,
-                                0,
-                                LatLonPoint.BYTES)
-                            < 0
-                        || Arrays.compareUnsigned(
-                                maxPackedValue,
-                                LatLonPoint.BYTES,
-                                2 * LatLonPoint.BYTES,
-                                maxLon,
-                                0,
-                                LatLonPoint.BYTES)
+                    ArrayUtil.compareUnsigned4(minPackedValue, LatLonPoint.BYTES, minLon, 0) < 0
+                        || ArrayUtil.compareUnsigned4(maxPackedValue, LatLonPoint.BYTES, maxLon, 0)
                             > 0;
               }
               if (crosses) {
diff --git a/lucene/core/src/java/org/apache/lucene/document/RangeFieldQuery.java b/lucene/core/src/java/org/apache/lucene/document/RangeFieldQuery.java
index 9070a82211c..6c14e0a9d07 100644
--- a/lucene/core/src/java/org/apache/lucene/document/RangeFieldQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/document/RangeFieldQuery.java
@@ -35,6 +35,8 @@ import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.ScorerSupplier;
 import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.ArrayUtil.ByteArrayComparator;
 import org.apache.lucene.util.DocIdSetBuilder;
 
 /**
@@ -56,6 +58,8 @@ public abstract class RangeFieldQuery extends Query {
   final byte[] ranges;
   /** number of bytes per dimension */
   final int bytesPerDim;
+  /** ByteArrayComparator selected by bytesPerDim */
+  final ByteArrayComparator comparator;
 
   /**
    * Used by {@code RangeFieldQuery} to check how each internal or leaf node relates to the query.
@@ -71,46 +75,19 @@ public abstract class RangeFieldQuery extends Query {
           byte[] maxPackedValue,
           int numDims,
           int bytesPerDim,
-          int dim) {
+          int dim,
+          ByteArrayComparator comparator) {
         int minOffset = dim * bytesPerDim;
         int maxOffset = minOffset + bytesPerDim * numDims;
 
-        if (Arrays.compareUnsigned(
-                    queryPackedValue,
-                    maxOffset,
-                    maxOffset + bytesPerDim,
-                    minPackedValue,
-                    minOffset,
-                    minOffset + bytesPerDim)
-                < 0
-            || Arrays.compareUnsigned(
-                    queryPackedValue,
-                    minOffset,
-                    minOffset + bytesPerDim,
-                    maxPackedValue,
-                    maxOffset,
-                    maxOffset + bytesPerDim)
-                > 0) {
+        if (comparator.compare(queryPackedValue, maxOffset, minPackedValue, minOffset) < 0
+            || comparator.compare(queryPackedValue, minOffset, maxPackedValue, maxOffset) > 0) {
           // disjoint
           return Relation.CELL_OUTSIDE_QUERY;
         }
 
-        if (Arrays.compareUnsigned(
-                    queryPackedValue,
-                    maxOffset,
-                    maxOffset + bytesPerDim,
-                    maxPackedValue,
-                    minOffset,
-                    minOffset + bytesPerDim)
-                >= 0
-            && Arrays.compareUnsigned(
-                    queryPackedValue,
-                    minOffset,
-                    minOffset + bytesPerDim,
-                    minPackedValue,
-                    maxOffset,
-                    maxOffset + bytesPerDim)
-                <= 0) {
+        if (comparator.compare(queryPackedValue, maxOffset, maxPackedValue, minOffset) >= 0
+            && comparator.compare(queryPackedValue, minOffset, minPackedValue, maxOffset) <= 0) {
           return Relation.CELL_INSIDE_QUERY;
         }
 
@@ -119,25 +96,16 @@ public abstract class RangeFieldQuery extends Query {
 
       @Override
       boolean matches(
-          byte[] queryPackedValue, byte[] packedValue, int numDims, int bytesPerDim, int dim) {
+          byte[] queryPackedValue,
+          byte[] packedValue,
+          int numDims,
+          int bytesPerDim,
+          int dim,
+          ByteArrayComparator comparator) {
         int minOffset = dim * bytesPerDim;
         int maxOffset = minOffset + bytesPerDim * numDims;
-        return Arrays.compareUnsigned(
-                    queryPackedValue,
-                    maxOffset,
-                    maxOffset + bytesPerDim,
-                    packedValue,
-                    minOffset,
-                    minOffset + bytesPerDim)
-                >= 0
-            && Arrays.compareUnsigned(
-                    queryPackedValue,
-                    minOffset,
-                    minOffset + bytesPerDim,
-                    packedValue,
-                    maxOffset,
-                    maxOffset + bytesPerDim)
-                <= 0;
+        return comparator.compare(queryPackedValue, maxOffset, packedValue, minOffset) >= 0
+            && comparator.compare(queryPackedValue, minOffset, packedValue, maxOffset) <= 0;
       }
     },
     /** Use this for within queries. */
@@ -150,46 +118,19 @@ public abstract class RangeFieldQuery extends Query {
           byte[] maxPackedValue,
           int numDims,
           int bytesPerDim,
-          int dim) {
+          int dim,
+          ByteArrayComparator comparator) {
         int minOffset = dim * bytesPerDim;
         int maxOffset = minOffset + bytesPerDim * numDims;
 
-        if (Arrays.compareUnsigned(
-                    queryPackedValue,
-                    maxOffset,
-                    maxOffset + bytesPerDim,
-                    minPackedValue,
-                    maxOffset,
-                    maxOffset + bytesPerDim)
-                < 0
-            || Arrays.compareUnsigned(
-                    queryPackedValue,
-                    minOffset,
-                    minOffset + bytesPerDim,
-                    maxPackedValue,
-                    minOffset,
-                    minOffset + bytesPerDim)
-                > 0) {
+        if (comparator.compare(queryPackedValue, maxOffset, minPackedValue, maxOffset) < 0
+            || comparator.compare(queryPackedValue, minOffset, maxPackedValue, minOffset) > 0) {
           // all ranges have at least one point outside of the query
           return Relation.CELL_OUTSIDE_QUERY;
         }
 
-        if (Arrays.compareUnsigned(
-                    queryPackedValue,
-                    maxOffset,
-                    maxOffset + bytesPerDim,
-                    maxPackedValue,
-                    maxOffset,
-                    maxOffset + bytesPerDim)
-                >= 0
-            && Arrays.compareUnsigned(
-                    queryPackedValue,
-                    minOffset,
-                    minOffset + bytesPerDim,
-                    minPackedValue,
-                    minOffset,
-                    minOffset + bytesPerDim)
-                <= 0) {
+        if (comparator.compare(queryPackedValue, maxOffset, maxPackedValue, maxOffset) >= 0
+            && comparator.compare(queryPackedValue, minOffset, minPackedValue, minOffset) <= 0) {
           return Relation.CELL_INSIDE_QUERY;
         }
 
@@ -198,25 +139,16 @@ public abstract class RangeFieldQuery extends Query {
 
       @Override
       boolean matches(
-          byte[] queryPackedValue, byte[] packedValue, int numDims, int bytesPerDim, int dim) {
+          byte[] queryPackedValue,
+          byte[] packedValue,
+          int numDims,
+          int bytesPerDim,
+          int dim,
+          ByteArrayComparator comparator) {
         int minOffset = dim * bytesPerDim;
         int maxOffset = minOffset + bytesPerDim * numDims;
-        return Arrays.compareUnsigned(
-                    queryPackedValue,
-                    minOffset,
-                    minOffset + bytesPerDim,
-                    packedValue,
-                    minOffset,
-                    minOffset + bytesPerDim)
-                <= 0
-            && Arrays.compareUnsigned(
-                    queryPackedValue,
-                    maxOffset,
-                    maxOffset + bytesPerDim,
-                    packedValue,
-                    maxOffset,
-                    maxOffset + bytesPerDim)
-                >= 0;
+        return comparator.compare(queryPackedValue, minOffset, packedValue, minOffset) <= 0
+            && comparator.compare(queryPackedValue, maxOffset, packedValue, maxOffset) >= 0;
       }
     },
     /** Use this for contains */
@@ -229,46 +161,19 @@ public abstract class RangeFieldQuery extends Query {
           byte[] maxPackedValue,
           int numDims,
           int bytesPerDim,
-          int dim) {
+          int dim,
+          ByteArrayComparator comparator) {
         int minOffset = dim * bytesPerDim;
         int maxOffset = minOffset + bytesPerDim * numDims;
 
-        if (Arrays.compareUnsigned(
-                    queryPackedValue,
-                    maxOffset,
-                    maxOffset + bytesPerDim,
-                    maxPackedValue,
-                    maxOffset,
-                    maxOffset + bytesPerDim)
-                > 0
-            || Arrays.compareUnsigned(
-                    queryPackedValue,
-                    minOffset,
-                    minOffset + bytesPerDim,
-                    minPackedValue,
-                    minOffset,
-                    minOffset + bytesPerDim)
-                < 0) {
+        if (comparator.compare(queryPackedValue, maxOffset, maxPackedValue, maxOffset) > 0
+            || comparator.compare(queryPackedValue, minOffset, minPackedValue, minOffset) < 0) {
           // all ranges are either less than the query max or greater than the query min
           return Relation.CELL_OUTSIDE_QUERY;
         }
 
-        if (Arrays.compareUnsigned(
-                    queryPackedValue,
-                    maxOffset,
-                    maxOffset + bytesPerDim,
-                    minPackedValue,
-                    maxOffset,
-                    maxOffset + bytesPerDim)
-                <= 0
-            && Arrays.compareUnsigned(
-                    queryPackedValue,
-                    minOffset,
-                    minOffset + bytesPerDim,
-                    maxPackedValue,
-                    minOffset,
-                    minOffset + bytesPerDim)
-                >= 0) {
+        if (comparator.compare(queryPackedValue, maxOffset, minPackedValue, maxOffset) <= 0
+            && comparator.compare(queryPackedValue, minOffset, maxPackedValue, minOffset) >= 0) {
           return Relation.CELL_INSIDE_QUERY;
         }
 
@@ -277,25 +182,16 @@ public abstract class RangeFieldQuery extends Query {
 
       @Override
       boolean matches(
-          byte[] queryPackedValue, byte[] packedValue, int numDims, int bytesPerDim, int dim) {
+          byte[] queryPackedValue,
+          byte[] packedValue,
+          int numDims,
+          int bytesPerDim,
+          int dim,
+          ByteArrayComparator comparator) {
         int minOffset = dim * bytesPerDim;
         int maxOffset = minOffset + bytesPerDim * numDims;
-        return Arrays.compareUnsigned(
-                    queryPackedValue,
-                    minOffset,
-                    minOffset + bytesPerDim,
-                    packedValue,
-                    minOffset,
-                    minOffset + bytesPerDim)
-                >= 0
-            && Arrays.compareUnsigned(
-                    queryPackedValue,
-                    maxOffset,
-                    maxOffset + bytesPerDim,
-                    packedValue,
-                    maxOffset,
-                    maxOffset + bytesPerDim)
-                <= 0;
+        return comparator.compare(queryPackedValue, minOffset, packedValue, minOffset) >= 0
+            && comparator.compare(queryPackedValue, maxOffset, packedValue, maxOffset) <= 0;
       }
     },
     /** Use this for crosses queries */
@@ -308,13 +204,19 @@ public abstract class RangeFieldQuery extends Query {
           byte[] maxPackedValue,
           int numDims,
           int bytesPerDim,
-          int dim) {
+          int dim,
+          ByteArrayComparator comparator) {
         throw new UnsupportedOperationException();
       }
 
       @Override
       boolean matches(
-          byte[] queryPackedValue, byte[] packedValue, int numDims, int bytesPerDim, int dim) {
+          byte[] queryPackedValue,
+          byte[] packedValue,
+          int numDims,
+          int bytesPerDim,
+          int dim,
+          ByteArrayComparator comparator) {
         throw new UnsupportedOperationException();
       }
 
@@ -324,17 +226,18 @@ public abstract class RangeFieldQuery extends Query {
           byte[] minPackedValue,
           byte[] maxPackedValue,
           int numDims,
-          int bytesPerDim) {
+          int bytesPerDim,
+          ByteArrayComparator comparator) {
         Relation intersectRelation =
             QueryType.INTERSECTS.compare(
-                queryPackedValue, minPackedValue, maxPackedValue, numDims, bytesPerDim);
+                queryPackedValue, minPackedValue, maxPackedValue, numDims, bytesPerDim, comparator);
         if (intersectRelation == Relation.CELL_OUTSIDE_QUERY) {
           return Relation.CELL_OUTSIDE_QUERY;
         }
 
         Relation withinRelation =
             QueryType.WITHIN.compare(
-                queryPackedValue, minPackedValue, maxPackedValue, numDims, bytesPerDim);
+                queryPackedValue, minPackedValue, maxPackedValue, numDims, bytesPerDim, comparator);
         if (withinRelation == Relation.CELL_INSIDE_QUERY) {
           return Relation.CELL_OUTSIDE_QUERY;
         }
@@ -348,9 +251,15 @@ public abstract class RangeFieldQuery extends Query {
       }
 
       @Override
-      boolean matches(byte[] queryPackedValue, byte[] packedValue, int numDims, int bytesPerDim) {
-        return INTERSECTS.matches(queryPackedValue, packedValue, numDims, bytesPerDim)
-            && WITHIN.matches(queryPackedValue, packedValue, numDims, bytesPerDim) == false;
+      boolean matches(
+          byte[] queryPackedValue,
+          byte[] packedValue,
+          int numDims,
+          int bytesPerDim,
+          ByteArrayComparator comparator) {
+        return INTERSECTS.matches(queryPackedValue, packedValue, numDims, bytesPerDim, comparator)
+            && WITHIN.matches(queryPackedValue, packedValue, numDims, bytesPerDim, comparator)
+                == false;
       }
     };
 
@@ -360,18 +269,27 @@ public abstract class RangeFieldQuery extends Query {
         byte[] maxPackedValue,
         int numDims,
         int bytesPerDim,
-        int dim);
+        int dim,
+        ByteArrayComparator comparator);
 
     Relation compare(
         byte[] queryPackedValue,
         byte[] minPackedValue,
         byte[] maxPackedValue,
         int numDims,
-        int bytesPerDim) {
+        int bytesPerDim,
+        ByteArrayComparator comparator) {
       boolean inside = true;
       for (int dim = 0; dim < numDims; ++dim) {
         Relation relation =
-            compare(queryPackedValue, minPackedValue, maxPackedValue, numDims, bytesPerDim, dim);
+            compare(
+                queryPackedValue,
+                minPackedValue,
+                maxPackedValue,
+                numDims,
+                bytesPerDim,
+                dim,
+                comparator);
         if (relation == Relation.CELL_OUTSIDE_QUERY) {
           return Relation.CELL_OUTSIDE_QUERY;
         } else if (relation != Relation.CELL_INSIDE_QUERY) {
@@ -382,11 +300,22 @@ public abstract class RangeFieldQuery extends Query {
     }
 
     abstract boolean matches(
-        byte[] queryPackedValue, byte[] packedValue, int numDims, int bytesPerDim, int dim);
+        byte[] queryPackedValue,
+        byte[] packedValue,
+        int numDims,
+        int bytesPerDim,
+        int dim,
+        ByteArrayComparator comparator);
 
-    boolean matches(byte[] queryPackedValue, byte[] packedValue, int numDims, int bytesPerDim) {
+    boolean matches(
+        byte[] queryPackedValue,
+        byte[] packedValue,
+        int numDims,
+        int bytesPerDim,
+        ByteArrayComparator comparator) {
       for (int dim = 0; dim < numDims; ++dim) {
-        if (matches(queryPackedValue, packedValue, numDims, bytesPerDim, dim) == false) {
+        if (matches(queryPackedValue, packedValue, numDims, bytesPerDim, dim, comparator)
+            == false) {
           return false;
         }
       }
@@ -412,6 +341,7 @@ public abstract class RangeFieldQuery extends Query {
     this.numDims = numDims;
     this.ranges = ranges;
     this.bytesPerDim = ranges.length / (2 * numDims);
+    this.comparator = ArrayUtil.getUnsignedComparator(bytesPerDim);
   }
 
   /** check input arguments */
@@ -473,21 +403,22 @@ public abstract class RangeFieldQuery extends Query {
 
           @Override
           public void visit(int docID, byte[] leaf) throws IOException {
-            if (queryType.matches(ranges, leaf, numDims, bytesPerDim)) {
+            if (queryType.matches(ranges, leaf, numDims, bytesPerDim, comparator)) {
               visit(docID);
             }
           }
 
           @Override
           public void visit(DocIdSetIterator iterator, byte[] leaf) throws IOException {
-            if (queryType.matches(ranges, leaf, numDims, bytesPerDim)) {
+            if (queryType.matches(ranges, leaf, numDims, bytesPerDim, comparator)) {
               adder.add(iterator);
             }
           }
 
           @Override
           public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-            return queryType.compare(ranges, minPackedValue, maxPackedValue, numDims, bytesPerDim);
+            return queryType.compare(
+                ranges, minPackedValue, maxPackedValue, numDims, bytesPerDim, comparator);
           }
         };
       }
@@ -513,7 +444,8 @@ public abstract class RangeFieldQuery extends Query {
                     values.getMinPackedValue(),
                     values.getMaxPackedValue(),
                     numDims,
-                    bytesPerDim)
+                    bytesPerDim,
+                    comparator)
                 == Relation.CELL_INSIDE_QUERY) {
           allDocsMatch = true;
         }
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index 51be1b46578..39abf4145b8 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -66,6 +66,8 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.Lock;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.ArrayUtil.ByteArrayComparator;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -2678,6 +2680,7 @@ public final class CheckIndex implements Closeable {
     private final int numDataDims;
     private final int numIndexDims;
     private final int bytesPerDim;
+    private final ByteArrayComparator comparator;
     private final String fieldName;
 
     /** Sole constructor */
@@ -2687,6 +2690,7 @@ public final class CheckIndex implements Closeable {
       numDataDims = values.getNumDimensions();
       numIndexDims = values.getNumIndexDimensions();
       bytesPerDim = values.getBytesPerDimension();
+      comparator = ArrayUtil.getUnsignedComparator(bytesPerDim);
       packedBytesCount = numDataDims * bytesPerDim;
       packedIndexBytesCount = numIndexDims * bytesPerDim;
       globalMinPackedValue = values.getMinPackedValue();
@@ -2778,14 +2782,7 @@ public final class CheckIndex implements Closeable {
         int offset = bytesPerDim * dim;
 
         // Compare to last cell:
-        if (Arrays.compareUnsigned(
-                packedValue,
-                offset,
-                offset + bytesPerDim,
-                lastMinPackedValue,
-                offset,
-                offset + bytesPerDim)
-            < 0) {
+        if (comparator.compare(packedValue, offset, lastMinPackedValue, offset) < 0) {
           // This doc's point, in this dimension, is lower than the minimum value of the last cell
           // checked:
           throw new CheckIndexException(
@@ -2803,14 +2800,7 @@ public final class CheckIndex implements Closeable {
                   + dim);
         }
 
-        if (Arrays.compareUnsigned(
-                packedValue,
-                offset,
-                offset + bytesPerDim,
-                lastMaxPackedValue,
-                offset,
-                offset + bytesPerDim)
-            > 0) {
+        if (comparator.compare(packedValue, offset, lastMaxPackedValue, offset) > 0) {
           // This doc's point, in this dimension, is greater than the maximum value of the last cell
           // checked:
           throw new CheckIndexException(
@@ -2835,8 +2825,7 @@ public final class CheckIndex implements Closeable {
       // for data dimension > 1, leaves are sorted by the dimension with the lowest cardinality to
       // improve block compression
       if (numDataDims == 1) {
-        int cmp =
-            Arrays.compareUnsigned(lastPackedValue, 0, bytesPerDim, packedValue, 0, bytesPerDim);
+        int cmp = comparator.compare(lastPackedValue, 0, packedValue, 0);
         if (cmp > 0) {
           throw new CheckIndexException(
               "packed points value "
@@ -2874,14 +2863,7 @@ public final class CheckIndex implements Closeable {
       for (int dim = 0; dim < numIndexDims; dim++) {
         int offset = bytesPerDim * dim;
 
-        if (Arrays.compareUnsigned(
-                minPackedValue,
-                offset,
-                offset + bytesPerDim,
-                maxPackedValue,
-                offset,
-                offset + bytesPerDim)
-            > 0) {
+        if (comparator.compare(minPackedValue, offset, maxPackedValue, offset) > 0) {
           throw new CheckIndexException(
               "packed points cell minPackedValue "
                   + Arrays.toString(minPackedValue)
@@ -2895,14 +2877,7 @@ public final class CheckIndex implements Closeable {
         }
 
         // Make sure this cell is not outside of the global min/max:
-        if (Arrays.compareUnsigned(
-                minPackedValue,
-                offset,
-                offset + bytesPerDim,
-                globalMinPackedValue,
-                offset,
-                offset + bytesPerDim)
-            < 0) {
+        if (comparator.compare(minPackedValue, offset, globalMinPackedValue, offset) < 0) {
           throw new CheckIndexException(
               "packed points cell minPackedValue "
                   + Arrays.toString(minPackedValue)
@@ -2915,14 +2890,7 @@ public final class CheckIndex implements Closeable {
                   + "\"");
         }
 
-        if (Arrays.compareUnsigned(
-                maxPackedValue,
-                offset,
-                offset + bytesPerDim,
-                globalMinPackedValue,
-                offset,
-                offset + bytesPerDim)
-            < 0) {
+        if (comparator.compare(maxPackedValue, offset, globalMinPackedValue, offset) < 0) {
           throw new CheckIndexException(
               "packed points cell maxPackedValue "
                   + Arrays.toString(maxPackedValue)
@@ -2935,14 +2903,7 @@ public final class CheckIndex implements Closeable {
                   + "\"");
         }
 
-        if (Arrays.compareUnsigned(
-                minPackedValue,
-                offset,
-                offset + bytesPerDim,
-                globalMaxPackedValue,
-                offset,
-                offset + bytesPerDim)
-            > 0) {
+        if (comparator.compare(minPackedValue, offset, globalMaxPackedValue, offset) > 0) {
           throw new CheckIndexException(
               "packed points cell minPackedValue "
                   + Arrays.toString(minPackedValue)
@@ -2954,14 +2915,7 @@ public final class CheckIndex implements Closeable {
                   + fieldName
                   + "\"");
         }
-        if (Arrays.compareUnsigned(
-                maxPackedValue,
-                offset,
-                offset + bytesPerDim,
-                globalMaxPackedValue,
-                offset,
-                offset + bytesPerDim)
-            > 0) {
+        if (comparator.compare(maxPackedValue, offset, globalMaxPackedValue, offset) > 0) {
           throw new CheckIndexException(
               "packed points cell maxPackedValue "
                   + Arrays.toString(maxPackedValue)