You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2021/05/24 18:49:50 UTC

[GitHub] [lucene] mikemccand commented on a change in pull request #127: LUCENE-9946: Support multi-value fields in range facet counting

mikemccand commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r638127417



##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/OverlappingLongRangeCounter.java
##########
@@ -0,0 +1,364 @@
+/*
+ * 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
+ *
+ *     http://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.lucene.facet.range;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * This implementation supports requested ranges that overlap. Because of this, we use a
+ * segment-tree to more efficiently aggregate counts into ranges at the end of processing. We also
+ * need to worry about double-counting issues since it's possible that multiple elementary segments,
+ * although mutually-exclusive, can roll-up to the same requested range. This creates some
+ * complexity with how we need to handle multi-valued documents.
+ */
+class OverlappingLongRangeCounter extends LongRangeCounter {
+
+  /** segment tree root node */
+  private final LongRangeNode root;
+  /** elementary segment boundaries used for efficient counting (bsearch to find interval) */
+  private final long[] boundaries;
+  /** whether-or-not there are leaf counts that still need to be rolled up at the end */
+  private boolean hasUnflushedCounts = false;
+
+  // Needed only for counting single-valued docs:
+  /** counts seen in each elementary interval leaf */
+  private int[] singleValuedLeafCounts;
+
+  // Needed only for counting multi-valued docs:
+  /** whether-or-not an elementary interval has seen at least one match for a single doc */
+  private FixedBitSet multiValuedDocLeafHits;
+  /** whether-or-not a range has seen at least one match for a single doc */
+  private FixedBitSet multiValuedDocRangeHits;
+
+  // Used during rollup
+  private int leafUpto;
+  /** number of counted documents that haven't matched any requested ranges */
+  private int missingCount = 0;

Review comment:
       Also this `= 0`.

##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeCounter.java
##########
@@ -148,170 +90,139 @@ public void add(long v) {
     // are guaranteed to find a match because the last
     // boundary is Long.MAX_VALUE:
 
+    long[] boundaries = boundaries();
+
     int lo = 0;
     int hi = boundaries.length - 1;
     while (true) {
       int mid = (lo + hi) >>> 1;
-      // System.out.println("  cycle lo=" + lo + " hi=" + hi + " mid=" + mid + " boundary=" +
-      // boundaries[mid] + " to " + boundaries[mid+1]);
       if (v <= boundaries[mid]) {
         if (mid == 0) {
-          leafCounts[0]++;
+          processSingleValuedHit(mid);
           return;
         } else {
           hi = mid - 1;
         }
       } else if (v > boundaries[mid + 1]) {
         lo = mid + 1;
       } else {
-        leafCounts[mid + 1]++;
-        // System.out.println("  incr @ " + (mid+1) + "; now " + leafCounts[mid+1]);
+        processSingleValuedHit(mid + 1);
         return;
       }
     }
   }
 
-  /**
-   * Fills counts corresponding to the original input ranges, returning the missing count (how many
-   * hits didn't match any ranges).
-   */
-  public int fillCounts(int[] counts) {
-    // System.out.println("  rollup");
-    missingCount = 0;
-    leafUpto = 0;
-    rollup(root, counts, false);
-    return missingCount;
-  }
+  /** Count a multi-valued doc value */
+  void addMultiValued(long v) {
 
-  private int rollup(LongRangeNode node, int[] counts, boolean sawOutputs) {
-    int count;
-    sawOutputs |= node.outputs != null;
-    if (node.left != null) {
-      count = rollup(node.left, counts, sawOutputs);
-      count += rollup(node.right, counts, sawOutputs);
-    } else {
-      // Leaf:
-      count = leafCounts[leafUpto];
-      leafUpto++;
-      if (!sawOutputs) {
-        // This is a missing count (no output ranges were
-        // seen "above" us):
-        missingCount += count;
-      }
+    if (rangeCount() == 0) {
+      return; // don't bother if there aren't any requested ranges
+    }
+
+    long[] boundaries = boundaries();
+
+    // First check if we've "advanced" beyond the last leaf we counted for this doc. If
+    // we haven't, there's no sense doing anything else:
+    if (multiValuedDocLastSeenLeaf != -1 && v <= boundaries[multiValuedDocLastSeenLeaf]) {
+      return;
     }
-    if (node.outputs != null) {
-      for (int rangeIndex : node.outputs) {
-        counts[rangeIndex] += count;
+
+    // Also check if we've already counted the last leaf. If so, there's nothing else to count
+    // for this doc:
+    final int nextCandidateLeaf = multiValuedDocLastSeenLeaf + 1;
+    if (nextCandidateLeaf == boundaries.length) {
+      return;
+    }
+
+    // Binary search in the range of the next candidate leaf up to the last leaf:
+    int lo = nextCandidateLeaf;
+    int hi = boundaries.length - 1;
+    while (true) {
+      int mid = (lo + hi) >>> 1;
+      if (v <= boundaries[mid]) {
+        if (mid == nextCandidateLeaf) {
+          processMultiValuedHit(mid);
+          multiValuedDocLastSeenLeaf = mid;
+          return;
+        } else {
+          hi = mid - 1;
+        }
+      } else if (v > boundaries[mid + 1]) {
+        lo = mid + 1;
+      } else {
+        int idx = mid + 1;
+        processMultiValuedHit(idx);
+        multiValuedDocLastSeenLeaf = idx;
+        return;
       }
     }
-    // System.out.println("  rollup node=" + node.start + " to " + node.end + ": count=" + count);
-    return count;
   }
 
-  private static LongRangeNode split(int start, int end, List<InclusiveRange> elementaryIntervals) {
-    if (start == end - 1) {
-      // leaf
-      InclusiveRange range = elementaryIntervals.get(start);
-      return new LongRangeNode(range.start, range.end, null, null, start);
-    } else {
-      int mid = (start + end) >>> 1;
-      LongRangeNode left = split(start, mid, elementaryIntervals);
-      LongRangeNode right = split(mid, end, elementaryIntervals);
-      return new LongRangeNode(left.start, right.end, left, right, -1);
-    }
+  /**
+   * Finish processing all documents. This will return the number of docs that didn't contribute to
+   * any ranges (that weren't already reported when calling endMultiValuedDoc()).
+   */
+  abstract int finish();
+
+  /** Provide boundary information for elementary segments (max inclusive value per range) */
+  protected abstract long[] boundaries();
+
+  /** Process a single-value "hit" against an elementary segment. */
+  protected abstract void processSingleValuedHit(int elementarySegmentNum);
+
+  /** Process a multi-value "hit" against an elementary segment. */
+  protected abstract void processMultiValuedHit(int elementarySegmentNum);
+
+  /** Increment the specified range by one. */
+  protected final void increment(int rangeNum) {
+    countBuffer[rangeNum]++;
   }
 
-  private static final class InclusiveRange {
-    public final long start;
-    public final long end;
+  /** Increment the specified range by the specified count. */
+  protected final void increment(int rangeNum, int count) {
+    countBuffer[rangeNum] += count;
+  }
 
-    public InclusiveRange(long start, long end) {
-      assert end >= start;
-      this.start = start;
-      this.end = end;
+  /** Number of ranges requested by the caller. */
+  protected final int rangeCount() {
+    return countBuffer.length;
+  }
+
+  /** Determine whether-or-not any requested ranges overlap */
+  private static boolean hasOverlappingRanges(LongRange[] ranges) {
+    if (ranges.length == 0) {
+      return false;
     }
 
-    @Override
-    public String toString() {
-      return start + " to " + end;
+    // Copy before sorting so we don't mess with the caller's original ranges:
+    LongRange[] sortedRanges = new LongRange[ranges.length];
+    System.arraycopy(ranges, 0, sortedRanges, 0, ranges.length);
+    Arrays.sort(sortedRanges, Comparator.comparingLong(r -> r.min));
+
+    long prev = sortedRanges[0].max;
+    for (int i = 1; i < sortedRanges.length; i++) {
+      if (sortedRanges[i].min <= prev) {

Review comment:
       Hmm are the ranges inclusive on both ends? Edit: yeah looks like they are (looking at `LongRange.java` sources).

##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/OverlappingLongRangeCounter.java
##########
@@ -0,0 +1,364 @@
+/*
+ * 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
+ *
+ *     http://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.lucene.facet.range;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * This implementation supports requested ranges that overlap. Because of this, we use a
+ * segment-tree to more efficiently aggregate counts into ranges at the end of processing. We also
+ * need to worry about double-counting issues since it's possible that multiple elementary segments,
+ * although mutually-exclusive, can roll-up to the same requested range. This creates some
+ * complexity with how we need to handle multi-valued documents.
+ */
+class OverlappingLongRangeCounter extends LongRangeCounter {
+
+  /** segment tree root node */
+  private final LongRangeNode root;
+  /** elementary segment boundaries used for efficient counting (bsearch to find interval) */
+  private final long[] boundaries;
+  /** whether-or-not there are leaf counts that still need to be rolled up at the end */
+  private boolean hasUnflushedCounts = false;
+
+  // Needed only for counting single-valued docs:
+  /** counts seen in each elementary interval leaf */
+  private int[] singleValuedLeafCounts;
+
+  // Needed only for counting multi-valued docs:
+  /** whether-or-not an elementary interval has seen at least one match for a single doc */
+  private FixedBitSet multiValuedDocLeafHits;
+  /** whether-or-not a range has seen at least one match for a single doc */
+  private FixedBitSet multiValuedDocRangeHits;
+
+  // Used during rollup
+  private int leafUpto;
+  /** number of counted documents that haven't matched any requested ranges */
+  private int missingCount = 0;
+
+  OverlappingLongRangeCounter(LongRange[] ranges, int[] countBuffer) {
+    super(countBuffer);
+
+    // Build elementary intervals:
+    List<InclusiveRange> elementaryIntervals = buildElementaryIntervals(ranges);
+
+    // Build binary tree on top of intervals:
+    root = split(0, elementaryIntervals.size(), elementaryIntervals);
+
+    // Set outputs, so we know which range to output for each node in the tree:
+    for (int i = 0; i < ranges.length; i++) {
+      root.addOutputs(i, ranges[i]);
+    }
+
+    // Keep track of elementary interval max boundaries for bsearch:
+    boundaries = new long[elementaryIntervals.size()];
+    for (int i = 0; i < boundaries.length; i++) {
+      boundaries[i] = elementaryIntervals.get(i).end;
+    }
+  }
+
+  @Override
+  void startMultiValuedDoc() {
+    super.startMultiValuedDoc();
+    // Lazy init a bitset to track the elementary segments we see of a multi-valued doc:
+    if (multiValuedDocLeafHits == null) {
+      multiValuedDocLeafHits = new FixedBitSet(boundaries.length);
+    } else {
+      multiValuedDocLeafHits.clear(0, multiValuedDocLeafHits.length());
+    }
+  }
+
+  @Override
+  boolean endMultiValuedDoc() {
+    assert multiValuedDocLeafHits != null : "must call startDoc() first";
+
+    // Short-circuit if the caller didn't specify any ranges to count:
+    if (rangeCount() == 0) {
+      return false;
+    }
+
+    // Do the rollup for this doc:
+    // Lazy init a bitset to track the requested ranges seen for this multi-valued doc:
+    if (multiValuedDocRangeHits == null) {
+      multiValuedDocRangeHits = new FixedBitSet(rangeCount());
+    } else {
+      multiValuedDocRangeHits.clear(0, multiValuedDocRangeHits.length());
+    }
+    leafUpto = 0;
+    rollupMultiValued(root);
+
+    // Actually increment the count for each matching range, and see if the doc contributed to
+    // at least one:
+    boolean docContributedToAtLeastOneRange = false;
+    for (int i = multiValuedDocRangeHits.nextSetBit(0); i < multiValuedDocRangeHits.length(); ) {
+      increment(i);
+      docContributedToAtLeastOneRange = true;
+      if (++i < multiValuedDocRangeHits.length()) {
+        i = multiValuedDocRangeHits.nextSetBit(i);
+      }
+    }
+
+    return docContributedToAtLeastOneRange;
+  }
+
+  @Override
+  int finish() {
+    if (hasUnflushedCounts) {
+      // Rollup any outstanding counts from single-valued cases:
+      missingCount = 0;
+      leafUpto = 0;
+      rollupSingleValued(root, false);
+
+      return missingCount;
+    } else {
+      return 0;
+    }
+  }
+
+  @Override
+  protected long[] boundaries() {
+    return boundaries;
+  }
+
+  @Override
+  protected void processSingleValuedHit(int elementarySegmentNum) {
+    // Lazy init:
+    if (singleValuedLeafCounts == null) {

Review comment:
       Do we expect caller to only ever call either the multi- or single- valued methods here, for each segment?  Like, caller should not mix/match depending on the hit (e.g. even in the multi-valued case, some docs might have only one value)?  In which case, maybe inside here we should assert `multiValuedDocLeaf/RangeHits == null` and vice/versa?

##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/OverlappingLongRangeCounter.java
##########
@@ -0,0 +1,364 @@
+/*
+ * 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
+ *
+ *     http://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.lucene.facet.range;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * This implementation supports requested ranges that overlap. Because of this, we use a
+ * segment-tree to more efficiently aggregate counts into ranges at the end of processing. We also
+ * need to worry about double-counting issues since it's possible that multiple elementary segments,
+ * although mutually-exclusive, can roll-up to the same requested range. This creates some
+ * complexity with how we need to handle multi-valued documents.
+ */
+class OverlappingLongRangeCounter extends LongRangeCounter {
+
+  /** segment tree root node */
+  private final LongRangeNode root;
+  /** elementary segment boundaries used for efficient counting (bsearch to find interval) */
+  private final long[] boundaries;
+  /** whether-or-not there are leaf counts that still need to be rolled up at the end */
+  private boolean hasUnflushedCounts = false;
+
+  // Needed only for counting single-valued docs:
+  /** counts seen in each elementary interval leaf */
+  private int[] singleValuedLeafCounts;
+
+  // Needed only for counting multi-valued docs:
+  /** whether-or-not an elementary interval has seen at least one match for a single doc */
+  private FixedBitSet multiValuedDocLeafHits;
+  /** whether-or-not a range has seen at least one match for a single doc */
+  private FixedBitSet multiValuedDocRangeHits;
+
+  // Used during rollup
+  private int leafUpto;
+  /** number of counted documents that haven't matched any requested ranges */
+  private int missingCount = 0;
+
+  OverlappingLongRangeCounter(LongRange[] ranges, int[] countBuffer) {
+    super(countBuffer);
+
+    // Build elementary intervals:
+    List<InclusiveRange> elementaryIntervals = buildElementaryIntervals(ranges);
+
+    // Build binary tree on top of intervals:
+    root = split(0, elementaryIntervals.size(), elementaryIntervals);
+
+    // Set outputs, so we know which range to output for each node in the tree:
+    for (int i = 0; i < ranges.length; i++) {
+      root.addOutputs(i, ranges[i]);
+    }
+
+    // Keep track of elementary interval max boundaries for bsearch:
+    boundaries = new long[elementaryIntervals.size()];
+    for (int i = 0; i < boundaries.length; i++) {
+      boundaries[i] = elementaryIntervals.get(i).end;
+    }
+  }
+
+  @Override
+  void startMultiValuedDoc() {
+    super.startMultiValuedDoc();
+    // Lazy init a bitset to track the elementary segments we see of a multi-valued doc:
+    if (multiValuedDocLeafHits == null) {
+      multiValuedDocLeafHits = new FixedBitSet(boundaries.length);
+    } else {
+      multiValuedDocLeafHits.clear(0, multiValuedDocLeafHits.length());
+    }
+  }
+
+  @Override
+  boolean endMultiValuedDoc() {
+    assert multiValuedDocLeafHits != null : "must call startDoc() first";
+
+    // Short-circuit if the caller didn't specify any ranges to count:
+    if (rangeCount() == 0) {
+      return false;
+    }
+
+    // Do the rollup for this doc:
+    // Lazy init a bitset to track the requested ranges seen for this multi-valued doc:
+    if (multiValuedDocRangeHits == null) {
+      multiValuedDocRangeHits = new FixedBitSet(rangeCount());
+    } else {
+      multiValuedDocRangeHits.clear(0, multiValuedDocRangeHits.length());
+    }
+    leafUpto = 0;
+    rollupMultiValued(root);
+
+    // Actually increment the count for each matching range, and see if the doc contributed to
+    // at least one:
+    boolean docContributedToAtLeastOneRange = false;
+    for (int i = multiValuedDocRangeHits.nextSetBit(0); i < multiValuedDocRangeHits.length(); ) {
+      increment(i);
+      docContributedToAtLeastOneRange = true;
+      if (++i < multiValuedDocRangeHits.length()) {
+        i = multiValuedDocRangeHits.nextSetBit(i);
+      }
+    }
+
+    return docContributedToAtLeastOneRange;
+  }
+
+  @Override
+  int finish() {
+    if (hasUnflushedCounts) {
+      // Rollup any outstanding counts from single-valued cases:
+      missingCount = 0;
+      leafUpto = 0;
+      rollupSingleValued(root, false);
+
+      return missingCount;
+    } else {
+      return 0;
+    }
+  }
+
+  @Override
+  protected long[] boundaries() {
+    return boundaries;
+  }
+
+  @Override
+  protected void processSingleValuedHit(int elementarySegmentNum) {
+    // Lazy init:
+    if (singleValuedLeafCounts == null) {
+      singleValuedLeafCounts = new int[boundaries.length];
+    }
+
+    singleValuedLeafCounts[elementarySegmentNum]++;
+    hasUnflushedCounts = true;
+  }
+
+  @Override
+  protected void processMultiValuedHit(int elementarySegmentNum) {
+    assert multiValuedDocLeafHits != null : "must call startDoc() first";
+    multiValuedDocLeafHits.set(elementarySegmentNum);
+  }
+
+  private static LongRangeNode split(int start, int end, List<InclusiveRange> elementaryIntervals) {
+    if (start == end - 1) {
+      // leaf
+      InclusiveRange range = elementaryIntervals.get(start);
+      return new LongRangeNode(range.start, range.end, null, null, start);
+    } else {
+      int mid = (start + end) >>> 1;
+      LongRangeNode left = split(start, mid, elementaryIntervals);
+      LongRangeNode right = split(mid, end, elementaryIntervals);
+      return new LongRangeNode(left.start, right.end, left, right, -1);
+    }
+  }
+
+  private int rollupSingleValued(LongRangeNode node, boolean sawOutputs) {
+    int count;
+    sawOutputs |= node.outputs != null;
+    if (node.left != null) {
+      count = rollupSingleValued(node.left, sawOutputs);
+      count += rollupSingleValued(node.right, sawOutputs);
+    } else {
+      // Leaf:
+      count = singleValuedLeafCounts[leafUpto];
+      leafUpto++;
+      if (!sawOutputs) {
+        // This is a missing count (no output ranges were
+        // seen "above" us):
+        missingCount += count;
+      }
+    }
+    if (node.outputs != null) {
+      for (int rangeIndex : node.outputs) {
+        increment(rangeIndex, count);
+      }
+    }
+
+    return count;
+  }
+
+  private boolean rollupMultiValued(LongRangeNode node) {

Review comment:
       This is called per-hit, while `rollupSingleValued` is called once at the end of the segment?

##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/OverlappingLongRangeCounter.java
##########
@@ -0,0 +1,364 @@
+/*
+ * 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
+ *
+ *     http://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.lucene.facet.range;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * This implementation supports requested ranges that overlap. Because of this, we use a
+ * segment-tree to more efficiently aggregate counts into ranges at the end of processing. We also
+ * need to worry about double-counting issues since it's possible that multiple elementary segments,
+ * although mutually-exclusive, can roll-up to the same requested range. This creates some
+ * complexity with how we need to handle multi-valued documents.
+ */
+class OverlappingLongRangeCounter extends LongRangeCounter {
+
+  /** segment tree root node */
+  private final LongRangeNode root;
+  /** elementary segment boundaries used for efficient counting (bsearch to find interval) */
+  private final long[] boundaries;
+  /** whether-or-not there are leaf counts that still need to be rolled up at the end */
+  private boolean hasUnflushedCounts = false;

Review comment:
       You don't need the `= false` -- it's java's default already.

##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/SimpleLongRangeCounter.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+ *
+ *     http://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.lucene.facet.range;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+
+/**
+ * This implementation assumes the requested ranges _do not_ overlap. With this assumption, we're
+ * able to take a simpler approach to accumulating range counts by just binary searching for the
+ * appropriate range and counting directly as each value comes in.
+ */
+class SimpleLongRangeCounter extends LongRangeCounter {
+
+  /** elementary segment boundaries used for efficient counting (bsearch to find interval) */
+  private final long[] boundaries;
+  /** original range number each elementary segment corresponds to (index into countBuffer) */
+  private final int[] rangeNums;
+  /** number of counted documents that haven't matched any requested ranges */
+  private int missingCount = 0;

Review comment:
       We don't need the `= 0` -- it's java's default.

##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeCounter.java
##########
@@ -16,122 +16,64 @@
  */
 package org.apache.lucene.facet.range;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import java.util.Arrays;
+import java.util.Comparator;
 
 /**
- * Counts how many times each range was seen; per-hit it's just a binary search ({@link #add})
- * against the elementary intervals, and in the end we rollup back to the original ranges.
+ * Counter for numeric ranges. Works for both single- and multi-valued cases (assuming you use it
+ * correctly).
+ *
+ * <p>Usage notes: When counting a document field that only has a single value, callers should call
+ * addSingleValued() with the value. Whenever a document field has multiple values, callers should
+ * call startMultiValuedDoc() at the beginning of processing the document, followed by
+ * addMultiValued() with each value before finally calling endMultiValuedDoc() at the end of
+ * processing the document. The call to endMultiValuedDoc() will respond with a boolean indicating
+ * whether-or-not the specific document matched against at least one of the ranges being counted.
+ * Finally, after processing all documents, the caller should call finish(). This final call will
+ * ensure the contents of the user-provided {@code countBuffer} contains accurate counts (each index
+ * corresponding to the provided {@code LongRange} in {@code ranges}). The final call to finish()
+ * will also report how many additional documents did not match against any ranges. The combination
+ * of the endMultiValuedDoc() boolean responses and the number reported by finish() communicates the
+ * total number of missing documents. Note that the call to finish() will not report any documents
+ * already reported missing by endMultiValuedDoc().
  */
-final class LongRangeCounter {
-
-  final LongRangeNode root;
-  final long[] boundaries;
-  final int[] leafCounts;
-
-  // Used during rollup
-  private int leafUpto;
-  private int missingCount;
-
-  public LongRangeCounter(LongRange[] ranges) {
-    // Maps all range inclusive endpoints to int flags; 1
-    // = start of interval, 2 = end of interval.  We need to
-    // track the start vs end case separately because if a
-    // given point is both, then it must be its own
-    // elementary interval:
-    Map<Long, Integer> endsMap = new HashMap<>();
-
-    endsMap.put(Long.MIN_VALUE, 1);
-    endsMap.put(Long.MAX_VALUE, 2);
-
-    for (LongRange range : ranges) {
-      Integer cur = endsMap.get(range.min);
-      if (cur == null) {
-        endsMap.put(range.min, 1);
-      } else {
-        endsMap.put(range.min, cur.intValue() | 1);
-      }
-      cur = endsMap.get(range.max);
-      if (cur == null) {
-        endsMap.put(range.max, 2);
-      } else {
-        endsMap.put(range.max, cur.intValue() | 2);
-      }
-    }
-
-    List<Long> endsList = new ArrayList<>(endsMap.keySet());
-    Collections.sort(endsList);
-
-    // Build elementaryIntervals (a 1D Venn diagram):
-    List<InclusiveRange> elementaryIntervals = new ArrayList<>();
-    int upto0 = 1;
-    long v = endsList.get(0);
-    long prev;
-    if (endsMap.get(v) == 3) {
-      elementaryIntervals.add(new InclusiveRange(v, v));
-      prev = v + 1;
-    } else {
-      prev = v;
-    }
-
-    while (upto0 < endsList.size()) {
-      v = endsList.get(upto0);
-      int flags = endsMap.get(v);
-      // System.out.println("  v=" + v + " flags=" + flags);
-      if (flags == 3) {
-        // This point is both an end and a start; we need to
-        // separate it:
-        if (v > prev) {
-          elementaryIntervals.add(new InclusiveRange(prev, v - 1));
-        }
-        elementaryIntervals.add(new InclusiveRange(v, v));
-        prev = v + 1;
-      } else if (flags == 1) {
-        // This point is only the start of an interval;
-        // attach it to next interval:
-        if (v > prev) {
-          elementaryIntervals.add(new InclusiveRange(prev, v - 1));
-        }
-        prev = v;
-      } else {
-        assert flags == 2;
-        // This point is only the end of an interval; attach
-        // it to last interval:
-        elementaryIntervals.add(new InclusiveRange(prev, v));
-        prev = v + 1;
-      }
-      // System.out.println("    ints=" + elementaryIntervals);
-      upto0++;
-    }
+abstract class LongRangeCounter {
 
-    // Build binary tree on top of intervals:
-    root = split(0, elementaryIntervals.size(), elementaryIntervals);
+  /** accumulated counts for all of the ranges */
+  private final int[] countBuffer;
 
-    // Set outputs, so we know which range to output for
-    // each node in the tree:
-    for (int i = 0; i < ranges.length; i++) {
-      root.addOutputs(i, ranges[i]);
-    }
+  /**
+   * track the last counted leaf so we can skip over ones we've already counted for multi-value doc

Review comment:
       Is a "leaf" the same as an "elemental range"?

##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/SimpleLongRangeCounter.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+ *
+ *     http://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.lucene.facet.range;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+
+/**
+ * This implementation assumes the requested ranges _do not_ overlap. With this assumption, we're
+ * able to take a simpler approach to accumulating range counts by just binary searching for the
+ * appropriate range and counting directly as each value comes in.
+ */
+class SimpleLongRangeCounter extends LongRangeCounter {
+
+  /** elementary segment boundaries used for efficient counting (bsearch to find interval) */
+  private final long[] boundaries;
+  /** original range number each elementary segment corresponds to (index into countBuffer) */
+  private final int[] rangeNums;
+  /** number of counted documents that haven't matched any requested ranges */
+  private int missingCount = 0;
+  /** whether-or-not the multi-valued doc currently being counted has matched any ranges */
+  private boolean multiValuedDocMatchedRange;
+
+  SimpleLongRangeCounter(LongRange[] ranges, int[] countBuffer) {
+    super(countBuffer);
+
+    // Create a copy of the requested ranges, sorted by min, and keeping track of the original
+    // position:
+    ReferencingLongRange[] sortedRanges = new ReferencingLongRange[ranges.length];
+    for (int i = 0; i < ranges.length; i++) {
+      sortedRanges[i] = new ReferencingLongRange(ranges[i], i);
+    }
+    Arrays.sort(sortedRanges, Comparator.comparingLong(r -> r.range.min));
+
+    // Create elementary intervals, which include requested ranges and "gaps" in-between:
+    List<InclusiveRange> elementaryIntervals = buildElementaryIntervals(sortedRanges);
+
+    // Keep track of elementary interval boundary ends (for bsearching) along with the requested
+    // range they map back to (and -1 when they map to a "gap" range):
+    boundaries = new long[elementaryIntervals.size()];
+    rangeNums = new int[elementaryIntervals.size()];
+    Arrays.fill(rangeNums, -1);
+    int currRange = 0;
+    for (int i = 0; i < boundaries.length; i++) {
+      boundaries[i] = elementaryIntervals.get(i).end;
+      if (currRange < sortedRanges.length) {
+        ReferencingLongRange curr = sortedRanges[currRange];
+        if (boundaries[i] == curr.range.max) {
+          rangeNums[i] = curr.pos;
+          currRange++;
+        }
+      }
+    }
+  }
+
+  @Override
+  void startMultiValuedDoc() {
+    super.startMultiValuedDoc();
+    multiValuedDocMatchedRange = false;
+  }
+
+  @Override
+  boolean endMultiValuedDoc() {
+    return multiValuedDocMatchedRange;
+  }
+
+  @Override
+  void addSingleValued(long v) {
+    if (rangeCount() == 0) {
+      missingCount++;
+      return;
+    }
+
+    super.addSingleValued(v);
+  }
+
+  @Override
+  int finish() {
+    // Nothing much to do in this case since we're able to count directly into the requested
+    // ranges as we go in this implementation. Just report any missing count:
+    return missingCount;
+  }
+
+  @Override
+  protected long[] boundaries() {
+    return boundaries;
+  }
+
+  @Override
+  protected void processSingleValuedHit(int elementarySegmentNum) {
+    int rangeNum = rangeNums[elementarySegmentNum];
+    if (rangeNum != -1) {
+      // The elementary segment we matched against corresponds to a requested
+      // range, so increment it:
+      increment(rangeNum);
+    } else {
+      // The matched elementary segment is a "gap" range, so the doc isn't
+      // present in any requested ranges:
+      missingCount++;
+    }
+  }
+
+  @Override
+  protected void processMultiValuedHit(int elementarySegmentNum) {
+    int rangeNum = rangeNums[elementarySegmentNum];
+    if (rangeNum != -1) {
+      // The elementary segment we matched against corresponds to a requested
+      // range, so increment it. We can do this without fear of double-counting
+      // since we know the requested ranges don't overlap:
+      increment(rangeNum);
+      multiValuedDocMatchedRange = true;
+    }
+  }
+
+  /**
+   * Create elementary intervals, which include requested ranges and "gaps" in-between. This logic
+   * assumes no requested ranges overlap, and that the incoming ranges have already been sorted.
+   */
+  private static List<InclusiveRange> buildElementaryIntervals(
+      ReferencingLongRange[] sortedRanges) {
+    List<InclusiveRange> elementaryIntervals = new ArrayList<>();
+    long prev = Long.MIN_VALUE;
+    for (ReferencingLongRange range : sortedRanges) {
+      if (range.range.min > prev) {
+        // add a "gap" range preceding requested range if necessary:
+        elementaryIntervals.add(new InclusiveRange(prev, range.range.min - 1));
+      }
+      // add the requested range:
+      elementaryIntervals.add(new InclusiveRange(range.range.min, range.range.max));
+      prev = range.range.max + 1;
+    }
+    if (elementaryIntervals.isEmpty() == false) {
+      long lastEnd = elementaryIntervals.get(elementaryIntervals.size() - 1).end;
+      if (lastEnd < Long.MAX_VALUE) {
+        elementaryIntervals.add(new InclusiveRange(lastEnd + 1, Long.MAX_VALUE));
+      }
+    } else {
+      // If no ranges were requested, create a single entry from MIN_VALUE to MAX_VALUE:
+      elementaryIntervals.add(new InclusiveRange(Long.MIN_VALUE, Long.MAX_VALUE));
+    }
+
+    return elementaryIntervals;
+  }
+
+  /** Simple container for a requested range and it's original position */

Review comment:
       s/`it's`/`its`

##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/OverlappingLongRangeCounter.java
##########
@@ -0,0 +1,364 @@
+/*
+ * 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
+ *
+ *     http://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.lucene.facet.range;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * This implementation supports requested ranges that overlap. Because of this, we use a
+ * segment-tree to more efficiently aggregate counts into ranges at the end of processing. We also
+ * need to worry about double-counting issues since it's possible that multiple elementary segments,
+ * although mutually-exclusive, can roll-up to the same requested range. This creates some
+ * complexity with how we need to handle multi-valued documents.
+ */
+class OverlappingLongRangeCounter extends LongRangeCounter {
+
+  /** segment tree root node */
+  private final LongRangeNode root;
+  /** elementary segment boundaries used for efficient counting (bsearch to find interval) */
+  private final long[] boundaries;
+  /** whether-or-not there are leaf counts that still need to be rolled up at the end */
+  private boolean hasUnflushedCounts = false;
+
+  // Needed only for counting single-valued docs:
+  /** counts seen in each elementary interval leaf */
+  private int[] singleValuedLeafCounts;
+
+  // Needed only for counting multi-valued docs:
+  /** whether-or-not an elementary interval has seen at least one match for a single doc */
+  private FixedBitSet multiValuedDocLeafHits;
+  /** whether-or-not a range has seen at least one match for a single doc */
+  private FixedBitSet multiValuedDocRangeHits;
+
+  // Used during rollup
+  private int leafUpto;
+  /** number of counted documents that haven't matched any requested ranges */
+  private int missingCount = 0;
+
+  OverlappingLongRangeCounter(LongRange[] ranges, int[] countBuffer) {
+    super(countBuffer);
+
+    // Build elementary intervals:
+    List<InclusiveRange> elementaryIntervals = buildElementaryIntervals(ranges);
+
+    // Build binary tree on top of intervals:
+    root = split(0, elementaryIntervals.size(), elementaryIntervals);
+
+    // Set outputs, so we know which range to output for each node in the tree:
+    for (int i = 0; i < ranges.length; i++) {
+      root.addOutputs(i, ranges[i]);
+    }
+
+    // Keep track of elementary interval max boundaries for bsearch:
+    boundaries = new long[elementaryIntervals.size()];
+    for (int i = 0; i < boundaries.length; i++) {
+      boundaries[i] = elementaryIntervals.get(i).end;
+    }
+  }
+
+  @Override
+  void startMultiValuedDoc() {
+    super.startMultiValuedDoc();
+    // Lazy init a bitset to track the elementary segments we see of a multi-valued doc:
+    if (multiValuedDocLeafHits == null) {
+      multiValuedDocLeafHits = new FixedBitSet(boundaries.length);
+    } else {
+      multiValuedDocLeafHits.clear(0, multiValuedDocLeafHits.length());
+    }
+  }
+
+  @Override
+  boolean endMultiValuedDoc() {
+    assert multiValuedDocLeafHits != null : "must call startDoc() first";
+
+    // Short-circuit if the caller didn't specify any ranges to count:
+    if (rangeCount() == 0) {
+      return false;
+    }
+
+    // Do the rollup for this doc:
+    // Lazy init a bitset to track the requested ranges seen for this multi-valued doc:
+    if (multiValuedDocRangeHits == null) {
+      multiValuedDocRangeHits = new FixedBitSet(rangeCount());
+    } else {
+      multiValuedDocRangeHits.clear(0, multiValuedDocRangeHits.length());
+    }
+    leafUpto = 0;
+    rollupMultiValued(root);
+
+    // Actually increment the count for each matching range, and see if the doc contributed to
+    // at least one:
+    boolean docContributedToAtLeastOneRange = false;
+    for (int i = multiValuedDocRangeHits.nextSetBit(0); i < multiValuedDocRangeHits.length(); ) {
+      increment(i);
+      docContributedToAtLeastOneRange = true;
+      if (++i < multiValuedDocRangeHits.length()) {
+        i = multiValuedDocRangeHits.nextSetBit(i);
+      }
+    }
+
+    return docContributedToAtLeastOneRange;
+  }
+
+  @Override
+  int finish() {
+    if (hasUnflushedCounts) {
+      // Rollup any outstanding counts from single-valued cases:
+      missingCount = 0;

Review comment:
       Ahh, maybe also `assert singleValuedLeafCounts != null`?

##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/SimpleLongRangeCounter.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+ *
+ *     http://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.lucene.facet.range;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+
+/**
+ * This implementation assumes the requested ranges _do not_ overlap. With this assumption, we're
+ * able to take a simpler approach to accumulating range counts by just binary searching for the
+ * appropriate range and counting directly as each value comes in.
+ */
+class SimpleLongRangeCounter extends LongRangeCounter {

Review comment:
       Maybe rename to `ExclusiveLongRangeCounter`, trying to express that it's the opposite of "overlapping"?

##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/SimpleLongRangeCounter.java
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+ *
+ *     http://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.lucene.facet.range;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+
+/**
+ * This implementation assumes the requested ranges _do not_ overlap. With this assumption, we're
+ * able to take a simpler approach to accumulating range counts by just binary searching for the
+ * appropriate range and counting directly as each value comes in.
+ */
+class SimpleLongRangeCounter extends LongRangeCounter {
+
+  /** elementary segment boundaries used for efficient counting (bsearch to find interval) */
+  private final long[] boundaries;
+  /** original range number each elementary segment corresponds to (index into countBuffer) */
+  private final int[] rangeNums;
+  /** number of counted documents that haven't matched any requested ranges */
+  private int missingCount = 0;
+  /** whether-or-not the multi-valued doc currently being counted has matched any ranges */
+  private boolean multiValuedDocMatchedRange;
+
+  SimpleLongRangeCounter(LongRange[] ranges, int[] countBuffer) {
+    super(countBuffer);
+
+    // Create a copy of the requested ranges, sorted by min, and keeping track of the original
+    // position:
+    ReferencingLongRange[] sortedRanges = new ReferencingLongRange[ranges.length];
+    for (int i = 0; i < ranges.length; i++) {
+      sortedRanges[i] = new ReferencingLongRange(ranges[i], i);
+    }
+    Arrays.sort(sortedRanges, Comparator.comparingLong(r -> r.range.min));
+
+    // Create elementary intervals, which include requested ranges and "gaps" in-between:
+    List<InclusiveRange> elementaryIntervals = buildElementaryIntervals(sortedRanges);
+
+    // Keep track of elementary interval boundary ends (for bsearching) along with the requested
+    // range they map back to (and -1 when they map to a "gap" range):
+    boundaries = new long[elementaryIntervals.size()];
+    rangeNums = new int[elementaryIntervals.size()];
+    Arrays.fill(rangeNums, -1);
+    int currRange = 0;
+    for (int i = 0; i < boundaries.length; i++) {
+      boundaries[i] = elementaryIntervals.get(i).end;
+      if (currRange < sortedRanges.length) {
+        ReferencingLongRange curr = sortedRanges[currRange];
+        if (boundaries[i] == curr.range.max) {
+          rangeNums[i] = curr.pos;
+          currRange++;
+        }
+      }
+    }
+  }
+
+  @Override
+  void startMultiValuedDoc() {
+    super.startMultiValuedDoc();
+    multiValuedDocMatchedRange = false;
+  }
+
+  @Override
+  boolean endMultiValuedDoc() {
+    return multiValuedDocMatchedRange;
+  }
+
+  @Override
+  void addSingleValued(long v) {
+    if (rangeCount() == 0) {
+      missingCount++;
+      return;
+    }
+
+    super.addSingleValued(v);
+  }
+
+  @Override
+  int finish() {
+    // Nothing much to do in this case since we're able to count directly into the requested
+    // ranges as we go in this implementation. Just report any missing count:
+    return missingCount;
+  }
+
+  @Override
+  protected long[] boundaries() {
+    return boundaries;
+  }
+
+  @Override
+  protected void processSingleValuedHit(int elementarySegmentNum) {
+    int rangeNum = rangeNums[elementarySegmentNum];
+    if (rangeNum != -1) {
+      // The elementary segment we matched against corresponds to a requested
+      // range, so increment it:
+      increment(rangeNum);
+    } else {
+      // The matched elementary segment is a "gap" range, so the doc isn't
+      // present in any requested ranges:
+      missingCount++;
+    }
+  }
+
+  @Override
+  protected void processMultiValuedHit(int elementarySegmentNum) {
+    int rangeNum = rangeNums[elementarySegmentNum];
+    if (rangeNum != -1) {
+      // The elementary segment we matched against corresponds to a requested
+      // range, so increment it. We can do this without fear of double-counting
+      // since we know the requested ranges don't overlap:
+      increment(rangeNum);
+      multiValuedDocMatchedRange = true;
+    }
+  }
+
+  /**
+   * Create elementary intervals, which include requested ranges and "gaps" in-between. This logic
+   * assumes no requested ranges overlap, and that the incoming ranges have already been sorted.
+   */
+  private static List<InclusiveRange> buildElementaryIntervals(
+      ReferencingLongRange[] sortedRanges) {
+    List<InclusiveRange> elementaryIntervals = new ArrayList<>();
+    long prev = Long.MIN_VALUE;
+    for (ReferencingLongRange range : sortedRanges) {
+      if (range.range.min > prev) {
+        // add a "gap" range preceding requested range if necessary:
+        elementaryIntervals.add(new InclusiveRange(prev, range.range.min - 1));
+      }
+      // add the requested range:
+      elementaryIntervals.add(new InclusiveRange(range.range.min, range.range.max));
+      prev = range.range.max + 1;
+    }
+    if (elementaryIntervals.isEmpty() == false) {
+      long lastEnd = elementaryIntervals.get(elementaryIntervals.size() - 1).end;
+      if (lastEnd < Long.MAX_VALUE) {
+        elementaryIntervals.add(new InclusiveRange(lastEnd + 1, Long.MAX_VALUE));
+      }
+    } else {
+      // If no ranges were requested, create a single entry from MIN_VALUE to MAX_VALUE:
+      elementaryIntervals.add(new InclusiveRange(Long.MIN_VALUE, Long.MAX_VALUE));
+    }
+
+    return elementaryIntervals;
+  }
+
+  /** Simple container for a requested range and it's original position */
+  private static final class ReferencingLongRange {

Review comment:
       Maybe rename to `LongRangeAndPos`?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org