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/05 19:58:48 UTC

[GitHub] [lucene] gsmiller opened a new pull request #127: LUCENE-9946: Support multi-value fields in range facet counting

gsmiller opened a new pull request #127:
URL: https://github.com/apache/lucene/pull/127


   # Description
   
   Add support for multi-value fields in range facet counting implementations (LongRangeFacetCounts and DoubleRangeFacetCounts).
   
   # Solution
   
   The segment-tree based approach to range counting was extended to handle docs with multiple values, and the facet counting implementations automatically detect whether-or-not the specified field is multi-valued.
   
   # Tests
   
   Added new tests in TestRangeFacetCounts to test multi-valued cases in addition to the existing single-value test cases.
   
   # Checklist
   
   Please review the following and check all that apply:
   
   - [x] I have reviewed the guidelines for [How to Contribute](https://wiki.apache.org/lucene/HowToContribute) and my code conforms to the standards described there to the best of my ability.
   - [x] I have created a Jira issue and added the issue ID to my pull request title.
   - [x] I have given Lucene maintainers [access](https://help.github.com/en/articles/allowing-changes-to-a-pull-request-branch-created-from-a-fork) to contribute to my PR branch. (optional but recommended)
   - [x] I have developed this patch against the `main` branch.
   - [x] I have run `./gradlew check`.
   - [x] I have added tests for my changes.
   


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r638250252



##########
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:
       So the caller can actually mix/match without issue. What you describe is probably simpler, but it's technically valid to mix/match, and could be more performant in situations where a doc actually has only one value. With that pattern, the multi-value cases need to get rolled up as each doc is counted, but rollup of all the single value docs can be deferred until the end. It makes describing usage a bit less straight forward, but I suppose it could be a meaningful optimization in cases where the majority of the hits are single-valued but a few outliers have multiple values.




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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r638251032



##########
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:
       I like it, thanks!




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


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

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r638833063



##########
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:
       Yes!  Naming is often the hardest part ;)




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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on pull request #127:
URL: https://github.com/apache/lucene/pull/127#issuecomment-834352411


   (In the meantime, I just pushed a small revision that does a more efficient binary search for multi-valued cases by keeping track of the last seen elementary interval, taking advantage of the sorted nature of the values within each doc.)


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r638250840



##########
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:
       Ack, done.




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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on pull request #127:
URL: https://github.com/apache/lucene/pull/127#issuecomment-848107460


   Thanks @mikemccand! Added some more Javadoc to make some of the rollup methods more clear. I think that covers everything, but please let me know if there's anything I overlooked. Thanks again!


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


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

Posted by GitBox <gi...@apache.org>.
rmuir commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r627497501



##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
##########
@@ -79,76 +94,150 @@ public DoubleRangeFacetCounts(
       DoubleRange... ranges)
       throws IOException {
     super(field, ranges, fastMatchQuery);
-    count(valueSource, hits.getMatchingDocs());
+    // use the provided valueSource if non-null, otherwise use the doc values associated with the
+    // field
+    if (valueSource != null) {
+      count(valueSource, hits.getMatchingDocs());
+    } else {
+      count(field, hits.getMatchingDocs());
+    }
   }
 
+  /** Counts from the provided valueSource. */
   private void count(DoubleValuesSource valueSource, List<MatchingDocs> matchingDocs)
       throws IOException {
 
-    DoubleRange[] ranges = (DoubleRange[]) this.ranges;
-
-    LongRange[] longRanges = new LongRange[ranges.length];
-    for (int i = 0; i < ranges.length; i++) {
-      DoubleRange range = ranges[i];
-      longRanges[i] =
-          new LongRange(
-              range.label,
-              NumericUtils.doubleToSortableLong(range.min),
-              true,
-              NumericUtils.doubleToSortableLong(range.max),
-              true);
-    }
+    LongRange[] longRanges = createLongRanges();
 
-    LongRangeCounter counter = new LongRangeCounter(longRanges);
+    LongRangeCounter counter = new LongRangeCounter(longRanges, counts, false);
 
     int missingCount = 0;
     for (MatchingDocs hits : matchingDocs) {
       DoubleValues fv = valueSource.getValues(hits.context, null);
-
       totCount += hits.totalHits;
-      final DocIdSetIterator fastMatchDocs;
+
+      final DocIdSetIterator it;
       if (fastMatchQuery != null) {
-        final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(hits.context);
-        final IndexSearcher searcher = new IndexSearcher(topLevelContext);
-        searcher.setQueryCache(null);
-        final Weight fastMatchWeight =
-            searcher.createWeight(
-                searcher.rewrite(fastMatchQuery), ScoreMode.COMPLETE_NO_SCORES, 1);
-        Scorer s = fastMatchWeight.scorer(hits.context);
-        if (s == null) {
+        DocIdSetIterator fastMatchDocs = createFastMatchDisi(hits.context);
+        if (fastMatchDocs == null) {
           continue;
+        } else {
+          it =
+              ConjunctionDISI.intersectIterators(
+                  Arrays.asList(hits.bits.iterator(), fastMatchDocs));
         }
-        fastMatchDocs = s.iterator();
       } else {
-        fastMatchDocs = null;
+        it = hits.bits.iterator();
       }
 
-      DocIdSetIterator docs = hits.bits.iterator();
-
-      for (int doc = docs.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; ) {
-        if (fastMatchDocs != null) {
-          int fastMatchDoc = fastMatchDocs.docID();
-          if (fastMatchDoc < doc) {
-            fastMatchDoc = fastMatchDocs.advance(doc);
-          }
-
-          if (doc != fastMatchDoc) {
-            doc = docs.advance(fastMatchDoc);
-            continue;
-          }
-        }
+      for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; ) {
         // Skip missing docs:
         if (fv.advanceExact(doc)) {
           counter.add(NumericUtils.doubleToSortableLong(fv.doubleValue()));
         } else {
           missingCount++;
         }
 
-        doc = docs.nextDoc();
+        doc = it.nextDoc();
       }
     }
 
-    missingCount += counter.fillCounts(counts);
+    missingCount += counter.finish();
     totCount -= missingCount;
   }
+
+  /** Counts from the provided field. */
+  private void count(String field, List<MatchingDocs> matchingDocs) throws IOException {
+
+    LongRange[] longRanges = createLongRanges();
+
+    LongRangeCounter counter = null;
+
+    int missingCount = 0;
+
+    for (MatchingDocs hits : matchingDocs) {
+
+      final DocIdSetIterator it;
+      if (fastMatchQuery != null) {
+        DocIdSetIterator fastMatchDocs = createFastMatchDisi(hits.context);
+        if (fastMatchDocs == null) {
+          continue;
+        } else {
+          it =
+              ConjunctionDISI.intersectIterators(
+                  Arrays.asList(hits.bits.iterator(), fastMatchDocs));
+        }
+      } else {
+        it = hits.bits.iterator();
+      }
+
+      SortedNumericDocValues multiValues = DocValues.getSortedNumeric(hits.context.reader(), field);
+      NumericDocValues singleValues = DocValues.unwrapSingleton(multiValues);
+
+      if (singleValues != null) {
+
+        if (counter == null) {
+          counter = new LongRangeCounter(longRanges, counts, false);
+        }
+        assert counter.isMultiValued == false;

Review comment:
       You are correct. If you choose `SORTED_NUMERIC` or `SORTED_SET` types, you can index 0..n values per document. But if all documents in the segment only have at most one value, the codec will write it the same as if you specified `NUMERIC` or `SORTED` and wrap with singleton:
    https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java#L1394-L1396




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


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

Posted by GitBox <gi...@apache.org>.
rmuir commented on pull request #127:
URL: https://github.com/apache/lucene/pull/127#issuecomment-833971467


   @gsmiller do you think we are taking advantage of the sortedness of `SORTED_NUMERIC` here? e.g. do we really need a bitset or could we record hits high-level like this:
   
   ```
   int lastRangeID = -1; // prevents double counting a range within multivalued doc
   for (value : values) {
     rangeID = segmentTree.lookup(value);
     if (rangeID != lastRangeID) {
       hits[rangeID]++;
       lastRangeID = rangeID;
     }
   }
   ```
   
   I'm trying to see if we can accelerate (e.g. avoid bitsets) and maybe reduce overhead between single and multi-valued algorithms, it may simplify code. Of course such an optimization is harder if range can be overlapping.


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on pull request #127:
URL: https://github.com/apache/lucene/pull/127#issuecomment-834034324


   @rmuir interesting thought. There might be something to optimize here with the knowledge that each value (within the context of a doc) is sorted, but I don't think it will fully resolve the need to rollup within the context of a doc (e.g., calling startDoc() and endDoc()) or the need for bit sets completely. It's also possible I'm not fully understanding your suggestion though or overlooking something.
   
   So for starters, the ranges requested for counting can overlap, but the elementary ranges created when setting up the segment tree cannot. For single-value docs, the algorithm first increments elementary interval counts for each doc. Of course, in this step, there's no risk of double-counting since each doc only has one value and the elementary intervals don't overlap. Then, at the end, the "rollup" step increments the requested ranges by rolling up the elementary intervals into the requested ranges. Note that each elementary interval can contribute to multiple requested ranges, and more than one elementary interval con contribute to a single requested range. But, even though multiple elementary intervals can roll up to the same requested range, it's accurate to sum these counts for the requested range since we know each elementary interval count came from a single doc (since the docs are single-valued).
   
   For multi-value cases, there are two places where double-counting can happen: First, when accumulating into the elementary intervals, it's possible a doc will have multiple values that land in the same interval. This is the more obvious case, and the one I'm solving with `multiValuedDocLeafHits`. This is also where I think your suggestion could be used, not bothering to even set an elementary range if we know we've already set it (by taking advantage of the sorted property of the doc values). The trickier double-counting case is when rolling up the elementary intervals into the requested ranges. Because multiple elementary intervals can roll into one requested range, if the same document contributed to multiple elementary intervals for one range, we'd double-count the requested range. So if we waiting until the end to do the rollup, if multiple elementary intervals rolled up to one requested range, we don't know if we can actually sum the counts since it's possible they came from 
 the same doc. This is the root problem I'm trying to address by doing the rollup after each doc, and also why I need `multiValuedDocRangeHits` (so multiple elementary intervals don't double-count the same requested range for the same doc).
   
   Apologies if I'm telling you a bunch of stuff you already know about this code or completely missing your point.
   


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r638247788



##########
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:
       Yes. Maybe there's an opportunity to tidy up terminology here. The leaves in the segment tree are all of the elementary ranges. I'll see if I can clean up some of the terminology.




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


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

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r638843546



##########
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:
       Ahh, OK, that's interesting.  So let's leave out those `assert`s I suggested, and maybe in the future we could mix/match this kind of optimization.  `SortedNumericDocValues` has a `.valueCount()` method that tells you per-hit how many values there are ... I guess we could optimize the `.valueCount() == 1` case then.




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


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

Posted by GitBox <gi...@apache.org>.
rmuir commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r626892485



##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeCounter.java
##########
@@ -17,26 +17,153 @@
 package org.apache.lucene.facet.range;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 /**
- * 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.
+ * Segment tree for counting numeric ranges. Works for both single- and multi-valued cases (assuming
+ * you use it correctly).
+ *
+ * <p>Usage notes: For counting against a single value field/source, callers should call add() for
+ * each value and then call finish() after all documents have been processed. The call to finish()
+ * will inform the caller how many documents didn't match against any ranges. After finish() has
+ * been called, the caller-provided count buffer (passed into the ctor) will be populated with
+ * accurate range counts.
+ *
+ * <p>For counting against a multi-valued field, callers should call startDoc() at the beginning of
+ * processing each doc, followed by add() for each value, and then endDoc() at the end of the doc.
+ * The call to endDoc() will inform the caller if that document matched against any ranges. It is
+ * not necessary to call finish() for multi-valued cases. After each call to endDoc(), the
+ * caller-provided count buffer (passed into the ctor) will be populated with accurate range counts.
  */
 final class LongRangeCounter {
 
-  final LongRangeNode root;
-  final long[] boundaries;
-  final int[] leafCounts;
+  /** segment tree root node */
+  private final LongRangeNode root;
+  /** elementary segment boundaries used for efficient counting (bsearch to find interval) */
+  private final long[] boundaries;
+  /** accumulated counts for all of the ranges */
+  private final int[] countBuffer;
+  /** whether-or-not we're counting docs that could be multi-valued */
+  final boolean isMultiValued;
+
+  // Needed only for counting single-valued docs:
+  /** counts seen in each elementary interval leaf */
+  private final 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 final boolean[] multiValuedDocLeafHits;
+  /** whether-or-not a range has seen at least one match for a single doc */
+  private final boolean[] multiValuedDocRangeHits;

Review comment:
       rather than `boolean[]`, have you experimented with FixedBitSet? I'm suspicious that `boolean[]` may take 8x as much memory since there isn't really a java boolean type at the .class level. personally I can't think of where i've ever used a `boolean[]`, i just usually think bitset in my head, so maybe my fears are unfounded...




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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r638252232



##########
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:
       Thanks! Naming is tricky...




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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on pull request #127:
URL: https://github.com/apache/lucene/pull/127#issuecomment-840197275


   No worries @rmuir! Whenever you have some time. No particular rush.


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on pull request #127:
URL: https://github.com/apache/lucene/pull/127#issuecomment-834346001


   Oh yeah, good point @rmuir! I actually wonder if a separate implementation of the segment-tree counter (`LongRangeCounter`) would make sense for this typical case. The entire "rollup" step can be avoided if there are no overlapping ranges. Counting each value becomes a simple binary search for the elementary interval, then a check if that interval is a requested range or "in between" requested ranges. Multi-value cases can utilize the optimization you suggest to avoid double-counting. Hmm... let me try that out.


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on pull request #127:
URL: https://github.com/apache/lucene/pull/127#issuecomment-847051886


   @rmuir just a friendly reminder this is still open whenever you have some time. Still no rush, just making sure it didn't slip through the cracks. Thanks again!


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


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

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r638833979



##########
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:
       OK maybe add a comment in each calling out this subtlety?




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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r638246848



##########
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:
       Looks like you sorted this out, but I'll leave a comment to make this more readable so a future developer doesn't have to jump around to confirm.




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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on pull request #127:
URL: https://github.com/apache/lucene/pull/127#issuecomment-847398425


   Thanks for the feedback @mikemccand! Posted a new revision to address feedback, primarily around naming consistency (using "elementary interval" instead of "leaf" or "elementary segment").


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r626951309



##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeCounter.java
##########
@@ -17,26 +17,153 @@
 package org.apache.lucene.facet.range;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 /**
- * 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.
+ * Segment tree for counting numeric ranges. Works for both single- and multi-valued cases (assuming
+ * you use it correctly).
+ *
+ * <p>Usage notes: For counting against a single value field/source, callers should call add() for
+ * each value and then call finish() after all documents have been processed. The call to finish()
+ * will inform the caller how many documents didn't match against any ranges. After finish() has
+ * been called, the caller-provided count buffer (passed into the ctor) will be populated with
+ * accurate range counts.
+ *
+ * <p>For counting against a multi-valued field, callers should call startDoc() at the beginning of
+ * processing each doc, followed by add() for each value, and then endDoc() at the end of the doc.
+ * The call to endDoc() will inform the caller if that document matched against any ranges. It is
+ * not necessary to call finish() for multi-valued cases. After each call to endDoc(), the
+ * caller-provided count buffer (passed into the ctor) will be populated with accurate range counts.
  */
 final class LongRangeCounter {
 
-  final LongRangeNode root;
-  final long[] boundaries;
-  final int[] leafCounts;
+  /** segment tree root node */
+  private final LongRangeNode root;
+  /** elementary segment boundaries used for efficient counting (bsearch to find interval) */
+  private final long[] boundaries;
+  /** accumulated counts for all of the ranges */
+  private final int[] countBuffer;
+  /** whether-or-not we're counting docs that could be multi-valued */
+  final boolean isMultiValued;
+
+  // Needed only for counting single-valued docs:
+  /** counts seen in each elementary interval leaf */
+  private final 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 final boolean[] multiValuedDocLeafHits;
+  /** whether-or-not a range has seen at least one match for a single doc */
+  private final boolean[] multiValuedDocRangeHits;

Review comment:
       I like the suggestion (responded in more detail on your related comment below).




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


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

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #127:
URL: https://github.com/apache/lucene/pull/127#issuecomment-847216192


   This change looks awesome!  First class support for tricky multi-valued long/double range facet counting!
   
   I understand the need for per-hit rollup in the multi-valued case ... but are we also doing that per-hit rollup for the single valued case (for code simplicity)?


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r638244889



##########
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:
       Fixed. This is a habit I got into a while back that I'm still working to break. Thanks!

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




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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on pull request #127:
URL: https://github.com/apache/lucene/pull/127#issuecomment-832970411


   I also ran a luceneutil benchmark (-source wikimedium10m) to check that the existing single-valued cases didn't regress in performance. Looks reasonable to my eye:
   ```
                       TaskQPS baseline      StdDevQPS range_facet_tweaks      StdDev                Pct diff p-value
               OrHighNotLow      762.13      (5.9%)      744.85      (6.0%)   -2.3% ( -13% -   10%) 0.231
                    LowTerm     1149.90      (3.5%)     1136.75      (2.7%)   -1.1% (  -7% -    5%) 0.254
                   PKLookup      149.74      (2.5%)      148.10      (2.8%)   -1.1% (  -6% -    4%) 0.191
                   HighTerm     1277.63      (3.7%)     1263.61      (3.2%)   -1.1% (  -7% -    5%) 0.312
                     Fuzzy2       28.99      (7.1%)       28.68      (6.6%)   -1.0% ( -13% -   13%) 0.633
                  OrHighLow      325.96      (4.5%)      323.50      (4.1%)   -0.8% (  -8% -    8%) 0.581
                    Respell       69.99      (1.8%)       69.64      (1.8%)   -0.5% (  -4% -    3%) 0.383
               OrNotHighLow      726.65      (2.8%)      723.62      (3.6%)   -0.4% (  -6% -    6%) 0.686
                 AndHighLow      516.40      (3.1%)      514.33      (2.5%)   -0.4% (  -5% -    5%) 0.655
            MedSloppyPhrase       94.03      (3.0%)       93.67      (3.7%)   -0.4% (  -6% -    6%) 0.724
                    MedTerm     1071.48      (2.8%)     1067.94      (3.0%)   -0.3% (  -5% -    5%) 0.719
                   Wildcard       74.23      (3.3%)       74.01      (3.5%)   -0.3% (  -6% -    6%) 0.785
                  MedPhrase      305.87      (3.1%)      305.02      (3.5%)   -0.3% (  -6% -    6%) 0.792
      BrowseMonthTaxoFacets        4.88      (3.6%)        4.87      (3.6%)   -0.2% (  -7% -    7%) 0.845
              OrNotHighHigh      525.04      (4.2%)      523.90      (4.8%)   -0.2% (  -8% -    9%) 0.878
       HighIntervalsOrdered       23.56      (2.0%)       23.53      (1.8%)   -0.1% (  -3% -    3%) 0.813
           HighSloppyPhrase       24.69      (4.5%)       24.66      (5.1%)   -0.1% (  -9% -    9%) 0.936
       BrowseDateTaxoFacets        4.12      (3.8%)        4.12      (3.9%)   -0.1% (  -7% -    7%) 0.928
              OrHighNotHigh      489.22      (4.6%)      488.71      (4.4%)   -0.1% (  -8% -    9%) 0.942
   BrowseDayOfYearTaxoFacets        4.12      (3.7%)        4.12      (3.9%)   -0.1% (  -7% -    7%) 0.956
                AndHighHigh       50.27      (3.4%)       50.24      (3.1%)   -0.1% (  -6% -    6%) 0.959
               OrHighNotMed      554.73      (3.9%)      554.58      (4.4%)   -0.0% (  -7% -    8%) 0.984
                  LowPhrase       94.40      (2.0%)       94.40      (1.8%)    0.0% (  -3% -    3%) 1.000
                     IntNRQ       65.05      (0.7%)       65.06      (0.4%)    0.0% (  -1% -    1%) 0.911
                 AndHighMed      139.73      (3.5%)      139.76      (3.4%)    0.0% (  -6% -    7%) 0.981
            LowSloppyPhrase       47.74      (2.8%)       47.77      (3.4%)    0.1% (  -5% -    6%) 0.951
                    Prefix3      180.61      (3.5%)      180.88      (3.9%)    0.1% (  -7% -    7%) 0.899
                 HighPhrase       70.08      (2.8%)       70.23      (2.1%)    0.2% (  -4% -    5%) 0.778
                LowSpanNear      129.15      (2.5%)      129.47      (3.0%)    0.2% (  -5% -    5%) 0.780
                     Fuzzy1       61.26     (13.7%)       61.47     (13.6%)    0.3% ( -23% -   32%) 0.937
               OrNotHighMed      469.03      (3.8%)      470.84      (3.6%)    0.4% (  -6% -    8%) 0.742
   BrowseDayOfYearSSDVFacets       11.06      (5.5%)       11.11      (5.1%)    0.4% (  -9% -   11%) 0.807
                MedSpanNear       15.05      (2.0%)       15.12      (2.0%)    0.5% (  -3% -    4%) 0.479
                 OrHighHigh       19.61      (4.2%)       19.72      (2.3%)    0.5% (  -5% -    7%) 0.624
               HighSpanNear       21.58      (2.4%)       21.70      (2.5%)    0.6% (  -4% -    5%) 0.467
                  OrHighMed       90.57      (3.7%)       91.19      (2.7%)    0.7% (  -5% -    7%) 0.502
      BrowseMonthSSDVFacets       12.31      (7.2%)       12.42      (5.9%)    0.9% ( -11% -   15%) 0.682
                 TermDTSort      100.38     (13.4%)      101.43     (14.1%)    1.0% ( -23% -   32%) 0.810
          HighTermMonthSort      128.35      (9.7%)      130.05     (11.9%)    1.3% ( -18% -   25%) 0.700
       HighTermTitleBDVSort      121.08     (12.3%)      123.32     (10.9%)    1.8% ( -19% -   28%) 0.616
      HighTermDayOfYearSort       92.38     (10.2%)       95.79     (12.7%)    3.7% ( -17% -   29%) 0.312
   ```


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


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

Posted by GitBox <gi...@apache.org>.
rmuir commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r626897700



##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeCounter.java
##########
@@ -17,26 +17,153 @@
 package org.apache.lucene.facet.range;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 /**
- * 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.
+ * Segment tree for counting numeric ranges. Works for both single- and multi-valued cases (assuming
+ * you use it correctly).
+ *
+ * <p>Usage notes: For counting against a single value field/source, callers should call add() for
+ * each value and then call finish() after all documents have been processed. The call to finish()
+ * will inform the caller how many documents didn't match against any ranges. After finish() has
+ * been called, the caller-provided count buffer (passed into the ctor) will be populated with
+ * accurate range counts.
+ *
+ * <p>For counting against a multi-valued field, callers should call startDoc() at the beginning of
+ * processing each doc, followed by add() for each value, and then endDoc() at the end of the doc.
+ * The call to endDoc() will inform the caller if that document matched against any ranges. It is
+ * not necessary to call finish() for multi-valued cases. After each call to endDoc(), the
+ * caller-provided count buffer (passed into the ctor) will be populated with accurate range counts.
  */
 final class LongRangeCounter {
 
-  final LongRangeNode root;
-  final long[] boundaries;
-  final int[] leafCounts;
+  /** segment tree root node */
+  private final LongRangeNode root;
+  /** elementary segment boundaries used for efficient counting (bsearch to find interval) */
+  private final long[] boundaries;
+  /** accumulated counts for all of the ranges */
+  private final int[] countBuffer;
+  /** whether-or-not we're counting docs that could be multi-valued */
+  final boolean isMultiValued;
+
+  // Needed only for counting single-valued docs:
+  /** counts seen in each elementary interval leaf */
+  private final 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 final boolean[] multiValuedDocLeafHits;
+  /** whether-or-not a range has seen at least one match for a single doc */
+  private final boolean[] multiValuedDocRangeHits;
 
   // Used during rollup
   private int leafUpto;
   private int missingCount;
 
-  public LongRangeCounter(LongRange[] ranges) {
+  LongRangeCounter(LongRange[] ranges, int[] countBuffer, boolean isMultiValued) {
+    // Whether-or-not we're processing docs that could be multi-valued:
+    this.isMultiValued = isMultiValued;
+
+    // We'll populate the user-provided count buffer with range counts:
+    this.countBuffer = 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]);
+    }
+
+    // Set boundaries (ends of each elementary interval):
+    boundaries = new long[elementaryIntervals.size()];
+    for (int i = 0; i < boundaries.length; i++) {
+      boundaries[i] = elementaryIntervals.get(i).end;
+    }
+
+    // Setup to count:
+    if (isMultiValued == false) {
+      // Setup to count single-valued docs only:
+      singleValuedLeafCounts = new int[boundaries.length];
+      multiValuedDocLeafHits = null;
+      multiValuedDocRangeHits = null;
+    } else {
+      // Setup to count multi-valued docs:
+      singleValuedLeafCounts = null;
+      multiValuedDocLeafHits = new boolean[boundaries.length];
+      multiValuedDocRangeHits = new boolean[ranges.length];
+    }
+  }
+
+  /**
+   * Start processing a new doc. It's unnecessary to call this for single-value cases (but it
+   * doesn't cause problems if you do).
+   */
+  void startDoc() {
+    if (isMultiValued) {
+      Arrays.fill(multiValuedDocLeafHits, false);
+    }
+  }
+
+  /**
+   * Finish processing a new doc. Returns whether-or-not the document contributed a count to at
+   * least one range. It's unnecessary to call this for single-value cases, and the return value in
+   * such cases will always be {@code true} (but calling it doesn't cause any problems).
+   */
+  boolean endDoc() {
+    // Necessary to rollup after each doc for multi-valued case:
+    if (isMultiValued) {
+      leafUpto = 0;
+      Arrays.fill(multiValuedDocRangeHits, false);
+      rollupMultiValued(root);
+      boolean docContributedToAtLeastOneRange = false;
+      for (int i = 0; i < multiValuedDocRangeHits.length; i++) {

Review comment:
       this loop could instead use `bitset.nextSetBit` instead of checking every array element, which might make bitset thing pay off in performance, especially if arrays get large and sparse (as it will check 64 bits at a time). but i don't know how big these arrays are "typically".




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


[GitHub] [lucene] gsmiller merged pull request #127: LUCENE-9946: Support multi-value fields in range facet counting

Posted by GitBox <gi...@apache.org>.
gsmiller merged pull request #127:
URL: https://github.com/apache/lucene/pull/127


   


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r639030715



##########
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:
       Yep. I'm actually optimizing already for `LongRangeFacetCounts` / `DoubleRangeFacetCounts`. If you have a look in the `RangeFacetCounts#count` method (lines 187-203), I'm taking advantage of the "single counting" paradigm whenever a "multi valued" source only contains a single value.




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


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

Posted by GitBox <gi...@apache.org>.
rmuir commented on pull request #127:
URL: https://github.com/apache/lucene/pull/127#issuecomment-840184139


   Sorry I fell behind here, real world intervened. Will try it out as soon as possible.


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r638254763



##########
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:
       That's correct. `rollupMultiValued` must get called after "counting" all values in a multi-valued hit. `rollupSingleValued` gets called once at the end of the segment. 




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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r638251217



##########
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:
       Thanks!




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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r627471783



##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
##########
@@ -79,76 +94,150 @@ public DoubleRangeFacetCounts(
       DoubleRange... ranges)
       throws IOException {
     super(field, ranges, fastMatchQuery);
-    count(valueSource, hits.getMatchingDocs());
+    // use the provided valueSource if non-null, otherwise use the doc values associated with the
+    // field
+    if (valueSource != null) {
+      count(valueSource, hits.getMatchingDocs());
+    } else {
+      count(field, hits.getMatchingDocs());
+    }
   }
 
+  /** Counts from the provided valueSource. */
   private void count(DoubleValuesSource valueSource, List<MatchingDocs> matchingDocs)
       throws IOException {
 
-    DoubleRange[] ranges = (DoubleRange[]) this.ranges;
-
-    LongRange[] longRanges = new LongRange[ranges.length];
-    for (int i = 0; i < ranges.length; i++) {
-      DoubleRange range = ranges[i];
-      longRanges[i] =
-          new LongRange(
-              range.label,
-              NumericUtils.doubleToSortableLong(range.min),
-              true,
-              NumericUtils.doubleToSortableLong(range.max),
-              true);
-    }
+    LongRange[] longRanges = createLongRanges();
 
-    LongRangeCounter counter = new LongRangeCounter(longRanges);
+    LongRangeCounter counter = new LongRangeCounter(longRanges, counts, false);
 
     int missingCount = 0;
     for (MatchingDocs hits : matchingDocs) {
       DoubleValues fv = valueSource.getValues(hits.context, null);
-
       totCount += hits.totalHits;
-      final DocIdSetIterator fastMatchDocs;
+
+      final DocIdSetIterator it;
       if (fastMatchQuery != null) {
-        final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(hits.context);
-        final IndexSearcher searcher = new IndexSearcher(topLevelContext);
-        searcher.setQueryCache(null);
-        final Weight fastMatchWeight =
-            searcher.createWeight(
-                searcher.rewrite(fastMatchQuery), ScoreMode.COMPLETE_NO_SCORES, 1);
-        Scorer s = fastMatchWeight.scorer(hits.context);
-        if (s == null) {
+        DocIdSetIterator fastMatchDocs = createFastMatchDisi(hits.context);
+        if (fastMatchDocs == null) {
           continue;
+        } else {
+          it =
+              ConjunctionDISI.intersectIterators(
+                  Arrays.asList(hits.bits.iterator(), fastMatchDocs));
         }
-        fastMatchDocs = s.iterator();
       } else {
-        fastMatchDocs = null;
+        it = hits.bits.iterator();
       }
 
-      DocIdSetIterator docs = hits.bits.iterator();
-
-      for (int doc = docs.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; ) {
-        if (fastMatchDocs != null) {
-          int fastMatchDoc = fastMatchDocs.docID();
-          if (fastMatchDoc < doc) {
-            fastMatchDoc = fastMatchDocs.advance(doc);
-          }
-
-          if (doc != fastMatchDoc) {
-            doc = docs.advance(fastMatchDoc);
-            continue;
-          }
-        }
+      for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; ) {
         // Skip missing docs:
         if (fv.advanceExact(doc)) {
           counter.add(NumericUtils.doubleToSortableLong(fv.doubleValue()));
         } else {
           missingCount++;
         }
 
-        doc = docs.nextDoc();
+        doc = it.nextDoc();
       }
     }
 
-    missingCount += counter.fillCounts(counts);
+    missingCount += counter.finish();
     totCount -= missingCount;
   }
+
+  /** Counts from the provided field. */
+  private void count(String field, List<MatchingDocs> matchingDocs) throws IOException {
+
+    LongRange[] longRanges = createLongRanges();
+
+    LongRangeCounter counter = null;
+
+    int missingCount = 0;
+
+    for (MatchingDocs hits : matchingDocs) {
+
+      final DocIdSetIterator it;
+      if (fastMatchQuery != null) {
+        DocIdSetIterator fastMatchDocs = createFastMatchDisi(hits.context);
+        if (fastMatchDocs == null) {
+          continue;
+        } else {
+          it =
+              ConjunctionDISI.intersectIterators(
+                  Arrays.asList(hits.bits.iterator(), fastMatchDocs));
+        }
+      } else {
+        it = hits.bits.iterator();
+      }
+
+      SortedNumericDocValues multiValues = DocValues.getSortedNumeric(hits.context.reader(), field);
+      NumericDocValues singleValues = DocValues.unwrapSingleton(multiValues);
+
+      if (singleValues != null) {
+
+        if (counter == null) {
+          counter = new LongRangeCounter(longRanges, counts, false);
+        }
+        assert counter.isMultiValued == false;

Review comment:
       @rmuir do you know if this is actually a valid assertion? Is it possible that different index segments could behave differently (e.g., some provide single-valued DVs while others provide multi-valued)? You mentioned that Lucene may optimize cases where the user specifies multi-valued but only ever provides single values. Could that come into play here (e.g., one index shard optimizes to single-valued while others are multi-valued)?
   
   If this is something I need to worry about, I'll just load the DVs up-front and use a multi-valued counter if any of them are multi-valued.




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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r627662566



##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
##########
@@ -79,76 +94,150 @@ public DoubleRangeFacetCounts(
       DoubleRange... ranges)
       throws IOException {
     super(field, ranges, fastMatchQuery);
-    count(valueSource, hits.getMatchingDocs());
+    // use the provided valueSource if non-null, otherwise use the doc values associated with the
+    // field
+    if (valueSource != null) {
+      count(valueSource, hits.getMatchingDocs());
+    } else {
+      count(field, hits.getMatchingDocs());
+    }
   }
 
+  /** Counts from the provided valueSource. */
   private void count(DoubleValuesSource valueSource, List<MatchingDocs> matchingDocs)
       throws IOException {
 
-    DoubleRange[] ranges = (DoubleRange[]) this.ranges;
-
-    LongRange[] longRanges = new LongRange[ranges.length];
-    for (int i = 0; i < ranges.length; i++) {
-      DoubleRange range = ranges[i];
-      longRanges[i] =
-          new LongRange(
-              range.label,
-              NumericUtils.doubleToSortableLong(range.min),
-              true,
-              NumericUtils.doubleToSortableLong(range.max),
-              true);
-    }
+    LongRange[] longRanges = createLongRanges();
 
-    LongRangeCounter counter = new LongRangeCounter(longRanges);
+    LongRangeCounter counter = new LongRangeCounter(longRanges, counts, false);
 
     int missingCount = 0;
     for (MatchingDocs hits : matchingDocs) {
       DoubleValues fv = valueSource.getValues(hits.context, null);
-
       totCount += hits.totalHits;
-      final DocIdSetIterator fastMatchDocs;
+
+      final DocIdSetIterator it;
       if (fastMatchQuery != null) {
-        final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(hits.context);
-        final IndexSearcher searcher = new IndexSearcher(topLevelContext);
-        searcher.setQueryCache(null);
-        final Weight fastMatchWeight =
-            searcher.createWeight(
-                searcher.rewrite(fastMatchQuery), ScoreMode.COMPLETE_NO_SCORES, 1);
-        Scorer s = fastMatchWeight.scorer(hits.context);
-        if (s == null) {
+        DocIdSetIterator fastMatchDocs = createFastMatchDisi(hits.context);
+        if (fastMatchDocs == null) {
           continue;
+        } else {
+          it =
+              ConjunctionDISI.intersectIterators(
+                  Arrays.asList(hits.bits.iterator(), fastMatchDocs));
         }
-        fastMatchDocs = s.iterator();
       } else {
-        fastMatchDocs = null;
+        it = hits.bits.iterator();
       }
 
-      DocIdSetIterator docs = hits.bits.iterator();
-
-      for (int doc = docs.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; ) {
-        if (fastMatchDocs != null) {
-          int fastMatchDoc = fastMatchDocs.docID();
-          if (fastMatchDoc < doc) {
-            fastMatchDoc = fastMatchDocs.advance(doc);
-          }
-
-          if (doc != fastMatchDoc) {
-            doc = docs.advance(fastMatchDoc);
-            continue;
-          }
-        }
+      for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; ) {
         // Skip missing docs:
         if (fv.advanceExact(doc)) {
           counter.add(NumericUtils.doubleToSortableLong(fv.doubleValue()));
         } else {
           missingCount++;
         }
 
-        doc = docs.nextDoc();
+        doc = it.nextDoc();
       }
     }
 
-    missingCount += counter.fillCounts(counts);
+    missingCount += counter.finish();
     totCount -= missingCount;
   }
+
+  /** Counts from the provided field. */
+  private void count(String field, List<MatchingDocs> matchingDocs) throws IOException {
+
+    LongRange[] longRanges = createLongRanges();
+
+    LongRangeCounter counter = null;
+
+    int missingCount = 0;
+
+    for (MatchingDocs hits : matchingDocs) {
+
+      final DocIdSetIterator it;
+      if (fastMatchQuery != null) {
+        DocIdSetIterator fastMatchDocs = createFastMatchDisi(hits.context);
+        if (fastMatchDocs == null) {
+          continue;
+        } else {
+          it =
+              ConjunctionDISI.intersectIterators(
+                  Arrays.asList(hits.bits.iterator(), fastMatchDocs));
+        }
+      } else {
+        it = hits.bits.iterator();
+      }
+
+      SortedNumericDocValues multiValues = DocValues.getSortedNumeric(hits.context.reader(), field);
+      NumericDocValues singleValues = DocValues.unwrapSingleton(multiValues);
+
+      if (singleValues != null) {
+
+        if (counter == null) {
+          counter = new LongRangeCounter(longRanges, counts, false);
+        }
+        assert counter.isMultiValued == false;

Review comment:
       OK, new version published that should work with mixed segment cases (along with unit tests). I also extracted some common logic into the parent RangeFacetCounts class to clean things up a bit.




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


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

Posted by GitBox <gi...@apache.org>.
rmuir commented on pull request #127:
URL: https://github.com/apache/lucene/pull/127#issuecomment-834312458


   in an overlapping case, we can't do the optimization. my suggestion was for a (i think typical?) case where ranges are non-overlapping (facets i see such as price ranges). in that case, it seems like we can avoid the overhead and just don't count a range if it was the same one we visited last time. 
   
   And I think adding that "if" wouldn't slow down single-valued case either, so we could still just have "two cases" but only do the bitset stuff when ranges overlap, which we can detect up front?


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r639038412



##########
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:
       I'll go ahead and add some javadoc to both of these methods since it's a bit confusing.




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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on pull request #127:
URL: https://github.com/apache/lucene/pull/127#issuecomment-834786325


   @rmuir just posted a new revision that optimizes non-overlapping range cases with its own implementation. It's a pretty big change from the previous version, but I'm curious to see what you think. It should be functionally correct (unit tests seem to agree at least) but I haven't done any bechmarking yet. I'm sure the code could be cleaned up a little bit as well, but I went ahead and put the revision up as-is to see if you think this makes more sense directionally than what I'd done previously. It still requires the startDoc + add + endDoc dance for multi-valued docs, but is maybe a bit more optimal. Thanks!


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r627620585



##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
##########
@@ -79,76 +94,150 @@ public DoubleRangeFacetCounts(
       DoubleRange... ranges)
       throws IOException {
     super(field, ranges, fastMatchQuery);
-    count(valueSource, hits.getMatchingDocs());
+    // use the provided valueSource if non-null, otherwise use the doc values associated with the
+    // field
+    if (valueSource != null) {
+      count(valueSource, hits.getMatchingDocs());
+    } else {
+      count(field, hits.getMatchingDocs());
+    }
   }
 
+  /** Counts from the provided valueSource. */
   private void count(DoubleValuesSource valueSource, List<MatchingDocs> matchingDocs)
       throws IOException {
 
-    DoubleRange[] ranges = (DoubleRange[]) this.ranges;
-
-    LongRange[] longRanges = new LongRange[ranges.length];
-    for (int i = 0; i < ranges.length; i++) {
-      DoubleRange range = ranges[i];
-      longRanges[i] =
-          new LongRange(
-              range.label,
-              NumericUtils.doubleToSortableLong(range.min),
-              true,
-              NumericUtils.doubleToSortableLong(range.max),
-              true);
-    }
+    LongRange[] longRanges = createLongRanges();
 
-    LongRangeCounter counter = new LongRangeCounter(longRanges);
+    LongRangeCounter counter = new LongRangeCounter(longRanges, counts, false);
 
     int missingCount = 0;
     for (MatchingDocs hits : matchingDocs) {
       DoubleValues fv = valueSource.getValues(hits.context, null);
-
       totCount += hits.totalHits;
-      final DocIdSetIterator fastMatchDocs;
+
+      final DocIdSetIterator it;
       if (fastMatchQuery != null) {
-        final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(hits.context);
-        final IndexSearcher searcher = new IndexSearcher(topLevelContext);
-        searcher.setQueryCache(null);
-        final Weight fastMatchWeight =
-            searcher.createWeight(
-                searcher.rewrite(fastMatchQuery), ScoreMode.COMPLETE_NO_SCORES, 1);
-        Scorer s = fastMatchWeight.scorer(hits.context);
-        if (s == null) {
+        DocIdSetIterator fastMatchDocs = createFastMatchDisi(hits.context);
+        if (fastMatchDocs == null) {
           continue;
+        } else {
+          it =
+              ConjunctionDISI.intersectIterators(
+                  Arrays.asList(hits.bits.iterator(), fastMatchDocs));
         }
-        fastMatchDocs = s.iterator();
       } else {
-        fastMatchDocs = null;
+        it = hits.bits.iterator();
       }
 
-      DocIdSetIterator docs = hits.bits.iterator();
-
-      for (int doc = docs.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; ) {
-        if (fastMatchDocs != null) {
-          int fastMatchDoc = fastMatchDocs.docID();
-          if (fastMatchDoc < doc) {
-            fastMatchDoc = fastMatchDocs.advance(doc);
-          }
-
-          if (doc != fastMatchDoc) {
-            doc = docs.advance(fastMatchDoc);
-            continue;
-          }
-        }
+      for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; ) {
         // Skip missing docs:
         if (fv.advanceExact(doc)) {
           counter.add(NumericUtils.doubleToSortableLong(fv.doubleValue()));
         } else {
           missingCount++;
         }
 
-        doc = docs.nextDoc();
+        doc = it.nextDoc();
       }
     }
 
-    missingCount += counter.fillCounts(counts);
+    missingCount += counter.finish();
     totCount -= missingCount;
   }
+
+  /** Counts from the provided field. */
+  private void count(String field, List<MatchingDocs> matchingDocs) throws IOException {
+
+    LongRange[] longRanges = createLongRanges();
+
+    LongRangeCounter counter = null;
+
+    int missingCount = 0;
+
+    for (MatchingDocs hits : matchingDocs) {
+
+      final DocIdSetIterator it;
+      if (fastMatchQuery != null) {
+        DocIdSetIterator fastMatchDocs = createFastMatchDisi(hits.context);
+        if (fastMatchDocs == null) {
+          continue;
+        } else {
+          it =
+              ConjunctionDISI.intersectIterators(
+                  Arrays.asList(hits.bits.iterator(), fastMatchDocs));
+        }
+      } else {
+        it = hits.bits.iterator();
+      }
+
+      SortedNumericDocValues multiValues = DocValues.getSortedNumeric(hits.context.reader(), field);
+      NumericDocValues singleValues = DocValues.unwrapSingleton(multiValues);
+
+      if (singleValues != null) {
+
+        if (counter == null) {
+          counter = new LongRangeCounter(longRanges, counts, false);
+        }
+        assert counter.isMultiValued == false;

Review comment:
       Thanks @rmuir! I reproduced this in a unit test and will fix it here shortly.




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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on pull request #127:
URL: https://github.com/apache/lucene/pull/127#issuecomment-834040647


   It occurred to me that we don't need to be so strict with setting up the seg-tree counter to only be single- or multi-valued. In fact, it's a bit more optimal to use the single-valued algorithm for docs that only have one value, even if docs can have multiple values in general (i.e., if a doc only has a single value, the caller only needs to call `add()`, and should only need to do `startDoc()` -> `add()` -> `endDoc()` when a doc actually has more than one value). I just uploaded a new revision with this tweak.


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


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

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r638845672



##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeCounter.java
##########
@@ -148,170 +91,142 @@ 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 elementary interval we counted for this doc.
+    // If we haven't, there's no sense doing anything else:
+    if (multiValuedDocLastSeenElementaryInterval != -1
+        && v <= boundaries[multiValuedDocLastSeenElementaryInterval]) {
+      return;
     }
-    if (node.outputs != null) {
-      for (int rangeIndex : node.outputs) {
-        counts[rangeIndex] += count;
+
+    // Also check if we've already counted the last elementary interval. If so, there's nothing
+    // else to count for this doc:
+    final int nextCandidateElementaryInterval = multiValuedDocLastSeenElementaryInterval + 1;
+    if (nextCandidateElementaryInterval == boundaries.length) {
+      return;
+    }
+
+    // Binary search in the range of the next candidate interval up to the last interval:
+    int lo = nextCandidateElementaryInterval;
+    int hi = boundaries.length - 1;
+    while (true) {
+      int mid = (lo + hi) >>> 1;
+      if (v <= boundaries[mid]) {
+        if (mid == nextCandidateElementaryInterval) {
+          processMultiValuedHit(mid);
+          multiValuedDocLastSeenElementaryInterval = mid;
+          return;
+        } else {
+          hi = mid - 1;
+        }
+      } else if (v > boundaries[mid + 1]) {
+        lo = mid + 1;
+      } else {
+        int idx = mid + 1;
+        processMultiValuedHit(idx);
+        multiValuedDocLastSeenElementaryInterval = 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 intervals (max inclusive value per interval) */
+  protected abstract long[] boundaries();
+
+  /** Process a single-value "hit" against an elementary interval. */
+  protected abstract void processSingleValuedHit(int elementaryIntervalNum);
+
+  /** Process a multi-value "hit" against an elementary interval. */
+  protected abstract void processMultiValuedHit(int elementaryIntervalNum);
+
+  /** 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 previousMax = sortedRanges[0].max;
+    for (int i = 1; i < sortedRanges.length; i++) {
+      // Ranges overlap if the next min is <= the previous max (note that LongRange models
+      // closed ranges, so equal limit points are considered overlapping):

Review comment:
       Thanks!




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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on a change in pull request #127:
URL: https://github.com/apache/lucene/pull/127#discussion_r626951160



##########
File path: lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeCounter.java
##########
@@ -17,26 +17,153 @@
 package org.apache.lucene.facet.range;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 /**
- * 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.
+ * Segment tree for counting numeric ranges. Works for both single- and multi-valued cases (assuming
+ * you use it correctly).
+ *
+ * <p>Usage notes: For counting against a single value field/source, callers should call add() for
+ * each value and then call finish() after all documents have been processed. The call to finish()
+ * will inform the caller how many documents didn't match against any ranges. After finish() has
+ * been called, the caller-provided count buffer (passed into the ctor) will be populated with
+ * accurate range counts.
+ *
+ * <p>For counting against a multi-valued field, callers should call startDoc() at the beginning of
+ * processing each doc, followed by add() for each value, and then endDoc() at the end of the doc.
+ * The call to endDoc() will inform the caller if that document matched against any ranges. It is
+ * not necessary to call finish() for multi-valued cases. After each call to endDoc(), the
+ * caller-provided count buffer (passed into the ctor) will be populated with accurate range counts.
  */
 final class LongRangeCounter {
 
-  final LongRangeNode root;
-  final long[] boundaries;
-  final int[] leafCounts;
+  /** segment tree root node */
+  private final LongRangeNode root;
+  /** elementary segment boundaries used for efficient counting (bsearch to find interval) */
+  private final long[] boundaries;
+  /** accumulated counts for all of the ranges */
+  private final int[] countBuffer;
+  /** whether-or-not we're counting docs that could be multi-valued */
+  final boolean isMultiValued;
+
+  // Needed only for counting single-valued docs:
+  /** counts seen in each elementary interval leaf */
+  private final 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 final boolean[] multiValuedDocLeafHits;
+  /** whether-or-not a range has seen at least one match for a single doc */
+  private final boolean[] multiValuedDocRangeHits;
 
   // Used during rollup
   private int leafUpto;
   private int missingCount;
 
-  public LongRangeCounter(LongRange[] ranges) {
+  LongRangeCounter(LongRange[] ranges, int[] countBuffer, boolean isMultiValued) {
+    // Whether-or-not we're processing docs that could be multi-valued:
+    this.isMultiValued = isMultiValued;
+
+    // We'll populate the user-provided count buffer with range counts:
+    this.countBuffer = 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]);
+    }
+
+    // Set boundaries (ends of each elementary interval):
+    boundaries = new long[elementaryIntervals.size()];
+    for (int i = 0; i < boundaries.length; i++) {
+      boundaries[i] = elementaryIntervals.get(i).end;
+    }
+
+    // Setup to count:
+    if (isMultiValued == false) {
+      // Setup to count single-valued docs only:
+      singleValuedLeafCounts = new int[boundaries.length];
+      multiValuedDocLeafHits = null;
+      multiValuedDocRangeHits = null;
+    } else {
+      // Setup to count multi-valued docs:
+      singleValuedLeafCounts = null;
+      multiValuedDocLeafHits = new boolean[boundaries.length];
+      multiValuedDocRangeHits = new boolean[ranges.length];
+    }
+  }
+
+  /**
+   * Start processing a new doc. It's unnecessary to call this for single-value cases (but it
+   * doesn't cause problems if you do).
+   */
+  void startDoc() {
+    if (isMultiValued) {
+      Arrays.fill(multiValuedDocLeafHits, false);
+    }
+  }
+
+  /**
+   * Finish processing a new doc. Returns whether-or-not the document contributed a count to at
+   * least one range. It's unnecessary to call this for single-value cases, and the return value in
+   * such cases will always be {@code true} (but calling it doesn't cause any problems).
+   */
+  boolean endDoc() {
+    // Necessary to rollup after each doc for multi-valued case:
+    if (isMultiValued) {
+      leafUpto = 0;
+      Arrays.fill(multiValuedDocRangeHits, false);
+      rollupMultiValued(root);
+      boolean docContributedToAtLeastOneRange = false;
+      for (int i = 0; i < multiValuedDocRangeHits.length; i++) {

Review comment:
       Interesting. I'm not sure I can speak with any authority on "typical" use cases, but it does seem like the range counting implementation in general is optimized to scale to a large number of ranges (e.g., paying the segment-tree setup costs to reduce runtime-complexity of matching ranges), so I lean towards doing the same. As for sparseness, I suspect that _will_ be a typical case. It's not unreasonable to expect any given (multi-valued) field would only match a small number of the ranges.
   
   Anyway, I like the `BitSet` suggestion and will go ahead and update this change to use it. It'd be really interesting to run a micro-bechmark to see the impact. I'll see if I can carve out some time for that in the next couple days. We could also explore that in a follow-up Jira if you don't think that's blocking. I'd be curious to dig into that either way.




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


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

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #127:
URL: https://github.com/apache/lucene/pull/127#issuecomment-847236610


   >  but are we also doing that per-hit rollup for the single valued case (for code simplicity)?
   
   Aha!  It looks like we still only rollup "in the end" (of each segment) for the single-valued case.


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


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

Posted by GitBox <gi...@apache.org>.
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