diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index a9545776616..97cdf2fcd95 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -231,6 +231,9 @@ Improvements * LUCENE-9850: Switch to PFOR encoding for doc IDs (instead of FOR). (Greg Miller) +* LUCENE-9946: Support for multi-value fields in LongRangeFacetCounts and + DoubleRangeFacetCounts. (Greg Miller) + * LUCENE-9929: Add NorwegianNormalizationFilter, which does the same as ScandinavianNormalizationFilter except it does not fold oo->ø and ao->å. (janhoy, Robert Muir, Adrien Grand) diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java index ae017b4d1e9..225318383d4 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java @@ -22,23 +22,19 @@ import org.apache.lucene.document.FloatDocValuesField; import org.apache.lucene.facet.Facets; import org.apache.lucene.facet.FacetsCollector; import org.apache.lucene.facet.FacetsCollector.MatchingDocs; -import org.apache.lucene.index.IndexReaderContext; -import org.apache.lucene.index.ReaderUtil; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.DoubleValues; import org.apache.lucene.search.DoubleValuesSource; -import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; -import org.apache.lucene.search.ScoreMode; -import org.apache.lucene.search.Scorer; -import org.apache.lucene.search.Weight; import org.apache.lucene.util.NumericUtils; /** - * {@link Facets} implementation that computes counts for dynamic double ranges from a provided - * {@link DoubleValuesSource}. Use this for dimensions that change in real-time (e.g. a relative - * time based dimension like "Past day", "Past 2 days", etc.) or that change for each request (e.g. - * distance from the user's location, "< 1 km", "< 2 km", etc.). + * {@link Facets} implementation that computes counts for dynamic double ranges. Use this for + * dimensions that change in real-time (e.g. a relative time based dimension like "Past day", "Past + * 2 days", etc.) or that change for each request (e.g. distance from the user's location, "< 1 + * km", "< 2 km", etc.). * *

If you have indexed your field using {@link FloatDocValuesField}, then you should use a * DoubleValuesSource generated from {@link DoubleValuesSource#fromFloatField(String)}. @@ -48,18 +44,29 @@ import org.apache.lucene.util.NumericUtils; public class DoubleRangeFacetCounts extends RangeFacetCounts { /** - * Create {@code RangeFacetCounts}, using {@link DoubleValues} from the specified field. + * Create {@code RangeFacetCounts}, using double value from the specified field. The field may be + * single-valued ({@link NumericDocValues}) or multi-valued ({@link SortedNumericDocValues}), and + * will be interpreted as containing double values. * *

N.B This assumes that the field was indexed with {@link * org.apache.lucene.document.DoubleDocValuesField}. For float-valued fields, use {@link * #DoubleRangeFacetCounts(String, DoubleValuesSource, FacetsCollector, DoubleRange...)} + * + *

TODO: Extend multi-valued support to fields that have been indexed as float values */ public DoubleRangeFacetCounts(String field, FacetsCollector hits, DoubleRange... ranges) throws IOException { - this(field, DoubleValuesSource.fromDoubleField(field), hits, ranges); + this(field, null, hits, ranges); } - /** Create {@code RangeFacetCounts} using the provided {@link DoubleValuesSource} */ + /** + * Create {@code RangeFacetCounts}, using the provided {@link DoubleValuesSource} if non-null. If + * {@code valueSource} is null, doc values from the provided {@code field} will be used. + * + *

N.B If relying on the provided {@code field}, see javadoc notes associated with {@link + * #DoubleRangeFacetCounts(String, FacetsCollector, DoubleRange...)} for assumptions on how the + * field is indexed. + */ public DoubleRangeFacetCounts( String field, DoubleValuesSource valueSource, FacetsCollector hits, DoubleRange... ranges) throws IOException { @@ -67,9 +74,14 @@ public class DoubleRangeFacetCounts extends RangeFacetCounts { } /** - * Create {@code RangeFacetCounts}, using the provided {@link DoubleValuesSource}, and using the - * provided Query as a fastmatch: only documents matching the query are checked for the matching - * ranges. + * Create {@code RangeFacetCounts}, using the provided {@link DoubleValuesSource} if non-null. If + * {@code valueSource} is null, doc values from the provided {@code field} will be used. Use the + * provided {@code Query} as a fastmatch: only documents matching the query are checked for the + * matching ranges. + * + *

N.B If relying on the provided {@code field}, see javadoc notes associated with {@link + * #DoubleRangeFacetCounts(String, FacetsCollector, DoubleRange...)} for assumptions on how the + * field is indexed. */ public DoubleRangeFacetCounts( String field, @@ -79,14 +91,53 @@ public class DoubleRangeFacetCounts extends RangeFacetCounts { 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) throws IOException { - DoubleRange[] ranges = (DoubleRange[]) this.ranges; + LongRange[] longRanges = getLongRanges(); + LongRangeCounter counter = LongRangeCounter.create(longRanges, counts); + + int missingCount = 0; + for (MatchingDocs hits : matchingDocs) { + DoubleValues fv = valueSource.getValues(hits.context, null); + totCount += hits.totalHits; + + final DocIdSetIterator it = createIterator(hits); + if (it == null) { + continue; + } + + for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; ) { + // Skip missing docs: + if (fv.advanceExact(doc)) { + counter.addSingleValued(NumericUtils.doubleToSortableLong(fv.doubleValue())); + } else { + missingCount++; + } + + doc = it.nextDoc(); + } + } + + missingCount += counter.finish(); + totCount -= missingCount; + } + + /** Create long ranges from the double ranges. */ + @Override + protected LongRange[] getLongRanges() { + DoubleRange[] ranges = (DoubleRange[]) this.ranges; LongRange[] longRanges = new LongRange[ranges.length]; for (int i = 0; i < ranges.length; i++) { DoubleRange range = ranges[i]; @@ -99,56 +150,11 @@ public class DoubleRangeFacetCounts extends RangeFacetCounts { true); } - LongRangeCounter counter = new LongRangeCounter(longRanges); + return longRanges; + } - int missingCount = 0; - for (MatchingDocs hits : matchingDocs) { - DoubleValues fv = valueSource.getValues(hits.context, null); - - totCount += hits.totalHits; - final DocIdSetIterator fastMatchDocs; - 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) { - continue; - } - fastMatchDocs = s.iterator(); - } else { - fastMatchDocs = null; - } - - 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; - } - } - // Skip missing docs: - if (fv.advanceExact(doc)) { - counter.add(NumericUtils.doubleToSortableLong(fv.doubleValue())); - } else { - missingCount++; - } - - doc = docs.nextDoc(); - } - } - - missingCount += counter.fillCounts(counts); - totCount -= missingCount; + @Override + protected long mapDocValue(long l) { + return NumericUtils.sortableDoubleBits(l); } } diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/ExclusiveLongRangeCounter.java b/lucene/facet/src/java/org/apache/lucene/facet/range/ExclusiveLongRangeCounter.java new file mode 100644 index 00000000000..37051df427d --- /dev/null +++ b/lucene/facet/src/java/org/apache/lucene/facet/range/ExclusiveLongRangeCounter.java @@ -0,0 +1,170 @@ +/* + * 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 ExclusiveLongRangeCounter extends LongRangeCounter { + + /** elementary interval boundaries used for efficient counting (bsearch to find interval) */ + private final long[] boundaries; + /** original range number each elementary interval corresponds to (index into countBuffer) */ + private final int[] rangeNums; + /** number of counted documents that haven't matched any requested ranges */ + private int missingCount; + /** whether-or-not the multi-valued doc currently being counted has matched any ranges */ + private boolean multiValuedDocMatchedRange; + + ExclusiveLongRangeCounter(LongRange[] ranges, int[] countBuffer) { + super(countBuffer); + + // Create a copy of the requested ranges, sorted by min, and keeping track of the original + // position: + LongRangeAndPos[] sortedRanges = new LongRangeAndPos[ranges.length]; + for (int i = 0; i < ranges.length; i++) { + sortedRanges[i] = new LongRangeAndPos(ranges[i], i); + } + Arrays.sort(sortedRanges, Comparator.comparingLong(r -> r.range.min)); + + // Create elementary intervals, which include requested ranges and "gaps" in-between: + List 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) { + LongRangeAndPos 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 elementaryIntervalNum) { + int rangeNum = rangeNums[elementaryIntervalNum]; + if (rangeNum != -1) { + // The elementary interval we matched against corresponds to a requested + // range, so increment it: + increment(rangeNum); + } else { + // The matched elementary interval is a "gap" range, so the doc isn't + // present in any requested ranges: + missingCount++; + } + } + + @Override + protected void processMultiValuedHit(int elementaryIntervalNum) { + int rangeNum = rangeNums[elementaryIntervalNum]; + if (rangeNum != -1) { + // The elementary interval 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 buildElementaryIntervals(LongRangeAndPos[] sortedRanges) { + List elementaryIntervals = new ArrayList<>(); + long prev = Long.MIN_VALUE; + for (LongRangeAndPos 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 its original position */ + private static final class LongRangeAndPos { + final LongRange range; + final int pos; + + LongRangeAndPos(LongRange range, int pos) { + this.range = range; + this.pos = pos; + } + } +} diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeCounter.java b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeCounter.java index ffdf6536d8c..4bbbacb7f3e 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeCounter.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeCounter.java @@ -16,122 +16,65 @@ */ 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). + * + *

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 { +abstract class LongRangeCounter { - final LongRangeNode root; - final long[] boundaries; - final int[] leafCounts; + /** accumulated counts for all of the ranges */ + private final int[] countBuffer; - // Used during rollup - private int leafUpto; - private int missingCount; + /** + * for multi-value docs, we keep track of the last elementary interval we've counted so we can use + * that as a lower-bound when counting subsequent values. this takes advantage of the fact that + * values within a given doc are sorted. + */ + protected int multiValuedDocLastSeenElementaryInterval; - 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 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 endsList = new ArrayList<>(endsMap.keySet()); - Collections.sort(endsList); - - // Build elementaryIntervals (a 1D Venn diagram): - List 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; + static LongRangeCounter create(LongRange[] ranges, int[] countBuffer) { + if (hasOverlappingRanges(ranges)) { + return new OverlappingLongRangeCounter(ranges, countBuffer); } else { - prev = v; + return new ExclusiveLongRangeCounter(ranges, countBuffer); } - - 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++; - } - - // 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; - } - - leafCounts = new int[boundaries.length]; - - // System.out.println("ranges: " + Arrays.toString(ranges)); - // System.out.println("intervals: " + elementaryIntervals); - // System.out.println("boundaries: " + Arrays.toString(boundaries)); - // System.out.println("root:\n" + root); } - public void add(long v) { - // System.out.println("add v=" + v); + protected LongRangeCounter(int[] countBuffer) { + // We'll populate the user-provided count buffer with range counts: + this.countBuffer = countBuffer; + } + + /** Start processing a new doc. It's unnecessary to call this for single-value cases. */ + void startMultiValuedDoc() { + multiValuedDocLastSeenElementaryInterval = -1; + } + + /** + * 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. + */ + abstract boolean endMultiValuedDoc(); + + /** Count a single valued doc */ + void addSingleValued(long v) { // NOTE: this works too, but it's ~6% slower on a simple // test with a high-freq TermQuery w/ range faceting on @@ -148,15 +91,15 @@ final class LongRangeCounter { // 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; @@ -164,68 +107,118 @@ final class LongRangeCounter { } 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; + } + } + } + + /** Count a multi-valued doc value */ + void addMultiValued(long v) { + + 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; + } + + // 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; } } } /** - * Fills counts corresponding to the original input ranges, returning the missing count (how many - * hits didn't match any ranges). + * 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()). */ - public int fillCounts(int[] counts) { - // System.out.println(" rollup"); - missingCount = 0; - leafUpto = 0; - rollup(root, counts, false); - return missingCount; + 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 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; + /** Increment the specified range by the specified count. */ + protected final void increment(int rangeNum, int count) { + countBuffer[rangeNum] += count; + } + + /** 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; + } + + // 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): + if (sortedRanges[i].min <= previousMax) { + return true; } + previousMax = sortedRanges[i].max; } - if (node.outputs != null) { - for (int rangeIndex : node.outputs) { - counts[rangeIndex] += count; - } - } - // System.out.println(" rollup node=" + node.start + " to " + node.end + ": count=" + count); - return count; + + return false; } - private static LongRangeNode split(int start, int end, List 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); - } - } + protected static final class InclusiveRange { + final long start; + final long end; - private static final class InclusiveRange { - public final long start; - public final long end; - - public InclusiveRange(long start, long end) { + InclusiveRange(long start, long end) { assert end >= start; this.start = start; this.end = end; @@ -236,82 +229,4 @@ final class LongRangeCounter { return start + " to " + end; } } - - /** Holds one node of the segment tree. */ - public static final class LongRangeNode { - final LongRangeNode left; - final LongRangeNode right; - - // Our range, inclusive: - final long start; - final long end; - - // If we are a leaf, the index into elementary ranges that - // we point to: - final int leafIndex; - - // Which range indices to output when a query goes - // through this node: - List outputs; - - public LongRangeNode( - long start, long end, LongRangeNode left, LongRangeNode right, int leafIndex) { - this.start = start; - this.end = end; - this.left = left; - this.right = right; - this.leafIndex = leafIndex; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - toString(sb, 0); - return sb.toString(); - } - - static void indent(StringBuilder sb, int depth) { - for (int i = 0; i < depth; i++) { - sb.append(" "); - } - } - - /** Recursively assigns range outputs to each node. */ - void addOutputs(int index, LongRange range) { - if (start >= range.min && end <= range.max) { - // Our range is fully included in the incoming - // range; add to our output list: - if (outputs == null) { - outputs = new ArrayList<>(); - } - outputs.add(index); - } else if (left != null) { - assert right != null; - // Recurse: - left.addOutputs(index, range); - right.addOutputs(index, range); - } - } - - void toString(StringBuilder sb, int depth) { - indent(sb, depth); - if (left == null) { - assert right == null; - sb.append("leaf: ").append(start).append(" to ").append(end); - } else { - sb.append("node: ").append(start).append(" to ").append(end); - } - if (outputs != null) { - sb.append(" outputs="); - sb.append(outputs); - } - sb.append('\n'); - - if (left != null) { - assert right != null; - left.toString(sb, depth + 1); - right.toString(sb, depth + 1); - } - } - } } diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java index 20cecc4ff79..50c9dd635cb 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java @@ -21,37 +21,37 @@ import java.util.List; import org.apache.lucene.facet.Facets; import org.apache.lucene.facet.FacetsCollector; import org.apache.lucene.facet.FacetsCollector.MatchingDocs; -import org.apache.lucene.index.IndexReaderContext; -import org.apache.lucene.index.ReaderUtil; -import org.apache.lucene.search.DocIdSet; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.LongValues; import org.apache.lucene.search.LongValuesSource; import org.apache.lucene.search.Query; -import org.apache.lucene.search.ScoreMode; -import org.apache.lucene.search.Scorer; -import org.apache.lucene.search.Weight; /** - * {@link Facets} implementation that computes counts for dynamic long ranges from a provided {@link - * LongValuesSource}. Use this for dimensions that change in real-time (e.g. a relative time based - * dimension like "Past day", "Past 2 days", etc.) or that change for each request (e.g. distance - * from the user's location, "< 1 km", "< 2 km", etc.). + * {@link Facets} implementation that computes counts for dynamic long ranges. Use this for + * dimensions that change in real-time (e.g. a relative time based dimension like "Past day", "Past + * 2 days", etc.) or that change for each request (e.g. distance from the user's location, "< 1 + * km", "< 2 km", etc.). * * @lucene.experimental */ public class LongRangeFacetCounts extends RangeFacetCounts { /** - * Create {@code LongRangeFacetCounts}, using {@link LongValuesSource} from the specified field. + * Create {@code LongRangeFacetCounts} using long values from the specified field. The field may + * be single-valued ({@link NumericDocValues}) or multi-valued ({@link SortedNumericDocValues}), + * and will be interpreted as containing long values. */ public LongRangeFacetCounts(String field, FacetsCollector hits, LongRange... ranges) throws IOException { - this(field, LongValuesSource.fromLongField(field), hits, ranges); + this(field, null, hits, ranges); } - /** Create {@code LongRangeFacetCounts}, using the provided {@link LongValuesSource}. */ + /** + * Create {@code LongRangeFacetCounts}, using the provided {@link LongValuesSource} if non-null. + * If {@code valueSource} is null, doc values from the provided {@code field} will be used. + */ public LongRangeFacetCounts( String field, LongValuesSource valueSource, FacetsCollector hits, LongRange... ranges) throws IOException { @@ -59,10 +59,11 @@ public class LongRangeFacetCounts extends RangeFacetCounts { } /** - * Create {@code LongRangeFacetCounts}, using the provided {@link LongValuesSource}, and using the - * provided Filter as a fastmatch: only documents passing the filter are checked for the matching - * ranges, which is helpful when the provided {@link LongValuesSource} is costly per-document, - * such as a geo distance. The filter must be random access (implement {@link DocIdSet#bits}). + * Create {@code LongRangeFacetCounts}, using the provided {@link LongValuesSource} if non-null. + * If {@code valueSource} is null, doc values from the provided {@code field} will be used. Use + * the provided {@code Query} as a fastmatch: only documents passing the filter are checked for + * the matching ranges, which is helpful when the provided {@link LongValuesSource} is costly + * per-document, such as a geo distance. */ public LongRangeFacetCounts( String field, @@ -72,67 +73,58 @@ public class LongRangeFacetCounts extends RangeFacetCounts { LongRange... 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. + * + *

TODO: Seems like we could extract this into RangeFacetCounts and make the logic common + * between this class and DoubleRangeFacetCounts somehow. The blocker right now is that this + * implementation expects LongValueSource and DoubleRangeFacetCounts expects DoubleValueSource. + */ private void count(LongValuesSource valueSource, List matchingDocs) throws IOException { - LongRange[] ranges = (LongRange[]) this.ranges; + LongRange[] ranges = getLongRanges(); - LongRangeCounter counter = new LongRangeCounter(ranges); + LongRangeCounter counter = LongRangeCounter.create(ranges, counts); int missingCount = 0; + for (MatchingDocs hits : matchingDocs) { LongValues fv = valueSource.getValues(hits.context, null); - totCount += hits.totalHits; - final DocIdSetIterator fastMatchDocs; - 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) { - continue; - } - fastMatchDocs = s.iterator(); - } else { - fastMatchDocs = null; + + final DocIdSetIterator it = createIterator(hits); + if (it == null) { + continue; } - 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(fv.longValue()); + counter.addSingleValued(fv.longValue()); } else { missingCount++; } - doc = docs.nextDoc(); + doc = it.nextDoc(); } } - int x = counter.fillCounts(counts); - - missingCount += x; - - // System.out.println("totCount " + totCount + " x " + x + " missingCount " + missingCount); + missingCount += counter.finish(); totCount -= missingCount; } + + @Override + protected LongRange[] getLongRanges() { + return (LongRange[]) this.ranges; + } } diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/OverlappingLongRangeCounter.java b/lucene/facet/src/java/org/apache/lucene/facet/range/OverlappingLongRangeCounter.java new file mode 100644 index 00000000000..3c9ae7ebd7c --- /dev/null +++ b/lucene/facet/src/java/org/apache/lucene/facet/range/OverlappingLongRangeCounter.java @@ -0,0 +1,383 @@ +/* + * 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 + * intervals, 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 interval boundaries used for efficient counting (bsearch to find interval) */ + private final long[] boundaries; + /** + * whether-or-not there are elementary interval counts that still need to be rolled up at the end + */ + private boolean hasUnflushedCounts; + + // Needed only for counting single-valued docs: + /** counts seen in each elementary interval */ + private int[] singleValuedElementaryIntervalCounts; + + // 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 multiValuedDocElementaryIntervalHits; + /** whether-or-not a requested range has seen at least one match for a single doc */ + private FixedBitSet multiValuedDocRangeHits; + + // Used during rollup + private int elementaryIntervalUpto; + /** number of counted documents that haven't matched any requested ranges */ + private int missingCount; + + OverlappingLongRangeCounter(LongRange[] ranges, int[] countBuffer) { + super(countBuffer); + + // Build elementary intervals: + List 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 intervals we see of a multi-valued doc: + if (multiValuedDocElementaryIntervalHits == null) { + multiValuedDocElementaryIntervalHits = new FixedBitSet(boundaries.length); + } else { + multiValuedDocElementaryIntervalHits.clear(0, multiValuedDocElementaryIntervalHits.length()); + } + } + + @Override + boolean endMultiValuedDoc() { + assert multiValuedDocElementaryIntervalHits != 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()); + } + elementaryIntervalUpto = 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; + elementaryIntervalUpto = 0; + rollupSingleValued(root, false); + + return missingCount; + } else { + return 0; + } + } + + @Override + protected long[] boundaries() { + return boundaries; + } + + @Override + protected void processSingleValuedHit(int elementaryIntervalNum) { + // Lazy init: + if (singleValuedElementaryIntervalCounts == null) { + singleValuedElementaryIntervalCounts = new int[boundaries.length]; + } + + singleValuedElementaryIntervalCounts[elementaryIntervalNum]++; + hasUnflushedCounts = true; + } + + @Override + protected void processMultiValuedHit(int elementaryIntervalNum) { + assert multiValuedDocElementaryIntervalHits != null : "must call startDoc() first"; + multiValuedDocElementaryIntervalHits.set(elementaryIntervalNum); + } + + private static LongRangeNode split(int start, int end, List 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); + } + } + + /** + * Rolls up all the single-valued doc counts. Note that this is done once at the end of processing + * all documents (as part of {@link #finish()}. This is done in bulk at the end for efficiency + * purposes (vs. after ever document). This works only for cases where documents have a + * single-value. Multi-valued docs need to get rolled up after each document to ensure there's no + * double-counting (see {@link #rollupMultiValued(LongRangeNode)}) + */ + 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 = singleValuedElementaryIntervalCounts[elementaryIntervalUpto]; + elementaryIntervalUpto++; + if (sawOutputs == false) { + // 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; + } + + /** + * Rolls up all the multi-valued doc counts. Note that this is done at the end of each document + * (as part of {@link #endMultiValuedDoc()}). All of the counts contributed by a single document + * get rolled up into the appropriate ranges in this step. It must be done after each document so + * that counts don't get double-counted, and so we know whether-or-not an individual doc actually + * contributed to any of the user-requested ranges. + */ + private boolean rollupMultiValued(LongRangeNode node) { + boolean containedHit; + if (node.left != null) { + containedHit = rollupMultiValued(node.left); + containedHit |= rollupMultiValued(node.right); + } else { + // Leaf: + containedHit = multiValuedDocElementaryIntervalHits.get(elementaryIntervalUpto); + elementaryIntervalUpto++; + } + if (containedHit && node.outputs != null) { + for (int rangeIndex : node.outputs) { + multiValuedDocRangeHits.set(rangeIndex); + } + } + + return containedHit; + } + + private static List buildElementaryIntervals(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 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 | 1); + } + cur = endsMap.get(range.max); + if (cur == null) { + endsMap.put(range.max, 2); + } else { + endsMap.put(range.max, cur | 2); + } + } + + List endsList = new ArrayList<>(endsMap.keySet()); + Collections.sort(endsList); + + // Build elementaryIntervals (a 1D Venn diagram): + List elementaryIntervals = new ArrayList<>(); + int upto = 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 (upto < endsList.size()) { + v = endsList.get(upto); + int flags = endsMap.get(v); + 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; + } + upto++; + } + + return elementaryIntervals; + } + + /** Holds one node of the segment tree. */ + public static final class LongRangeNode { + final LongRangeNode left; + final LongRangeNode right; + + // Our range, inclusive: + final long start; + final long end; + + // If we are a leaf, the index into elementary ranges that we point to: + final int elementaryIntervalIndex; + + // Which range indices to output when a query goes + // through this node: + List outputs; + + public LongRangeNode( + long start, + long end, + LongRangeNode left, + LongRangeNode right, + int elementaryIntervalIndex) { + this.start = start; + this.end = end; + this.left = left; + this.right = right; + this.elementaryIntervalIndex = elementaryIntervalIndex; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + toString(sb, 0); + return sb.toString(); + } + + static void indent(StringBuilder sb, int depth) { + for (int i = 0; i < depth; i++) { + sb.append(" "); + } + } + + /** Recursively assigns range outputs to each node. */ + void addOutputs(int index, LongRange range) { + if (start >= range.min && end <= range.max) { + // Our range is fully included in the incoming + // range; add to our output list: + if (outputs == null) { + outputs = new ArrayList<>(); + } + outputs.add(index); + } else if (left != null) { + assert right != null; + // Recurse: + left.addOutputs(index, range); + right.addOutputs(index, range); + } + } + + void toString(StringBuilder sb, int depth) { + indent(sb, depth); + if (left == null) { + assert right == null; + sb.append("leaf: ").append(start).append(" to ").append(end); + } else { + sb.append("node: ").append(start).append(" to ").append(end); + } + if (outputs != null) { + sb.append(" outputs="); + sb.append(outputs); + } + sb.append('\n'); + + if (left != null) { + assert right != null; + left.toString(sb, depth + 1); + right.toString(sb, depth + 1); + } + } + } +} diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetCounts.java index 05b8175d90d..29461f01cf0 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetCounts.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetCounts.java @@ -17,12 +17,25 @@ package org.apache.lucene.facet.range; import java.io.IOException; +import java.util.Arrays; import java.util.Collections; import java.util.List; import org.apache.lucene.facet.FacetResult; import org.apache.lucene.facet.Facets; +import org.apache.lucene.facet.FacetsCollector; import org.apache.lucene.facet.LabelAndValue; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.IndexReaderContext; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.ReaderUtil; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.search.ConjunctionUtils; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreMode; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.Weight; /** * Base class for range faceting. @@ -57,6 +70,147 @@ abstract class RangeFacetCounts extends Facets { counts = new int[ranges.length]; } + /** + * Create a {@link org.apache.lucene.search.DocIdSetIterator} from the provided {@code hits} that + * relies on {@code fastMatchQuery} if available for first-pass filtering. A null response + * indicates no documents will match. + */ + protected DocIdSetIterator createIterator(FacetsCollector.MatchingDocs hits) throws IOException { + + 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); + final Scorer s = fastMatchWeight.scorer(hits.context); + if (s == null) { + return null; // no hits from the fastMatchQuery; return null + } else { + DocIdSetIterator fastMatchDocs = s.iterator(); + return ConjunctionUtils.intersectIterators( + Arrays.asList(hits.bits.iterator(), fastMatchDocs)); + } + + } else { + return hits.bits.iterator(); + } + } + + protected abstract LongRange[] getLongRanges(); + + protected long mapDocValue(long l) { + return l; + } + + /** Counts from the provided field. */ + protected void count(String field, List matchingDocs) + throws IOException { + + // load doc values for all segments up front and keep track of whether-or-not we found any that + // were actually multi-valued. this allows us to optimize the case where all segments contain + // single-values. + SortedNumericDocValues[] multiValuedDocVals = new SortedNumericDocValues[matchingDocs.size()]; + NumericDocValues[] singleValuedDocVals = null; + boolean foundMultiValued = false; + + for (int i = 0; i < matchingDocs.size(); i++) { + + FacetsCollector.MatchingDocs hits = matchingDocs.get(i); + + SortedNumericDocValues multiValues = DocValues.getSortedNumeric(hits.context.reader(), field); + multiValuedDocVals[i] = multiValues; + + // only bother trying to unwrap a singleton if we haven't yet seen any true multi-valued cases + if (foundMultiValued == false) { + NumericDocValues singleValues = DocValues.unwrapSingleton(multiValues); + if (singleValues != null) { + if (singleValuedDocVals == null) { + singleValuedDocVals = new NumericDocValues[matchingDocs.size()]; + } + singleValuedDocVals[i] = singleValues; + } else { + foundMultiValued = true; + } + } + } + + // we only need to keep around one or the other at this point + if (foundMultiValued) { + singleValuedDocVals = null; + } else { + multiValuedDocVals = null; + } + + LongRangeCounter counter = LongRangeCounter.create(getLongRanges(), counts); + + int missingCount = 0; + + // if we didn't find any multi-valued cases, we can run a more optimal counting algorithm + if (foundMultiValued == false) { + + for (int i = 0; i < matchingDocs.size(); i++) { + + FacetsCollector.MatchingDocs hits = matchingDocs.get(i); + + final DocIdSetIterator it = createIterator(hits); + if (it == null) { + continue; + } + + assert singleValuedDocVals != null; + NumericDocValues singleValues = singleValuedDocVals[i]; + + totCount += hits.totalHits; + for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; ) { + if (singleValues.advanceExact(doc)) { + counter.addSingleValued(mapDocValue(singleValues.longValue())); + } else { + missingCount++; + } + + doc = it.nextDoc(); + } + } + } else { + + for (int i = 0; i < matchingDocs.size(); i++) { + + final DocIdSetIterator it = createIterator(matchingDocs.get(i)); + if (it == null) { + continue; + } + + SortedNumericDocValues multiValues = multiValuedDocVals[i]; + + for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; ) { + if (multiValues.advanceExact(doc)) { + int limit = multiValues.docValueCount(); + // optimize single-value case + if (limit == 1) { + counter.addSingleValued(mapDocValue(multiValues.nextValue())); + totCount++; + } else { + counter.startMultiValuedDoc(); + for (int j = 0; j < limit; j++) { + counter.addMultiValued(mapDocValue(multiValues.nextValue())); + } + if (counter.endMultiValuedDoc()) { + totCount++; + } + } + } + + doc = it.nextDoc(); + } + } + } + + missingCount += counter.finish(); + totCount -= missingCount; + } + @Override public FacetResult getTopChildren(int topN, String dim, String... path) { if (dim.equals(field) == false) { diff --git a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java index 5fad75c1eac..e7959101ce7 100644 --- a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java +++ b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java @@ -16,6 +16,7 @@ */ package org.apache.lucene.facet.range; +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import java.io.IOException; import java.util.HashMap; import java.util.List; @@ -26,6 +27,7 @@ import org.apache.lucene.document.DoubleDocValuesField; import org.apache.lucene.document.DoublePoint; import org.apache.lucene.document.LongPoint; import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.SortedNumericDocValuesField; import org.apache.lucene.facet.DrillDownQuery; import org.apache.lucene.facet.DrillSideways; import org.apache.lucene.facet.DrillSideways.DrillSidewaysResult; @@ -104,6 +106,103 @@ public class TestRangeFacetCounts extends FacetTestCase { d.close(); } + public void testBasicLongMultiValued() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + Document doc = new Document(); + // just index the same value twice each time and make sure we don't double count + SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("field", 0L); + SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("field", 0L); + doc.add(field1); + doc.add(field2); + for (long l = 0; l < 100; l++) { + field1.setLongValue(l); + field2.setLongValue(l); + w.addDocument(doc); + } + + // Also add Long.MAX_VALUE + field1.setLongValue(Long.MAX_VALUE); + field2.setLongValue(Long.MAX_VALUE); + w.addDocument(doc); + + IndexReader r = w.getReader(); + w.close(); + + FacetsCollector fc = new FacetsCollector(); + IndexSearcher s = newSearcher(r); + s.search(new MatchAllDocsQuery(), fc); + + Facets facets = + new LongRangeFacetCounts( + "field", + fc, + new LongRange("less than 10", 0L, true, 10L, false), + new LongRange("less than or equal to 10", 0L, true, 10L, true), + new LongRange("over 90", 90L, false, 100L, false), + new LongRange("90 or above", 90L, true, 100L, false), + new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, true)); + + FacetResult result = facets.getTopChildren(10, "field"); + assertEquals( + "dim=field path=[] value=22 childCount=5\n less than 10 (10)\n less than or equal to 10 (11)\n over 90 (9)\n 90 or above (10)\n over 1000 (1)\n", + result.toString()); + + r.close(); + d.close(); + } + + public void testBasicLongMultiValuedMixedSegmentTypes() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("field", 0L); + SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("field", 0L); + // write docs as two segments (50 in each). the first segment will contain a mix of single- and + // multi-value cases, while the second segment will be all single values. + for (int l = 0; l < 100; l++) { + field1.setLongValue(l); + field2.setLongValue(l); + Document doc = new Document(); + doc.add(field1); + if (l == 0) { + doc.add(field2); + } else if (l < 50) { + if (random().nextBoolean()) { + doc.add(field2); + } + } + w.addDocument(doc); + if (l == 50) { + w.commit(); + } + } + + IndexReader r = w.getReader(); + w.close(); + + FacetsCollector fc = new FacetsCollector(); + IndexSearcher s = newSearcher(r); + s.search(new MatchAllDocsQuery(), fc); + + Facets facets = + new LongRangeFacetCounts( + "field", + fc, + new LongRange("less than 10", 0L, true, 10L, false), + new LongRange("less than or equal to 10", 0L, true, 10L, true), + new LongRange("over 90", 90L, false, 100L, false), + new LongRange("90 or above", 90L, true, 100L, false), + new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, true)); + + FacetResult result = facets.getTopChildren(10, "field"); + assertEquals( + "dim=field path=[] value=21 childCount=5\n less than 10 (10)\n less than or equal to 10 (11)\n over 90 (9)\n 90 or above (10)\n over 1000 (0)\n", + result.toString()); + + r.close(); + d.close(); + } + public void testLongGetAllDims() throws Exception { Directory d = newDirectory(); RandomIndexWriter w = new RandomIndexWriter(random(), d); @@ -248,6 +347,63 @@ public class TestRangeFacetCounts extends FacetTestCase { d.close(); } + public void testEmptyRangesSingleValued() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + Document doc = new Document(); + NumericDocValuesField field = new NumericDocValuesField("field", 0L); + doc.add(field); + for (long l = 0; l < 100; l++) { + field.setLongValue(l); + w.addDocument(doc); + } + + IndexReader r = w.getReader(); + w.close(); + + FacetsCollector fc = new FacetsCollector(); + IndexSearcher s = newSearcher(r); + s.search(new MatchAllDocsQuery(), fc); + + Facets facets = new LongRangeFacetCounts("field", fc); + + FacetResult result = facets.getTopChildren(10, "field"); + assertEquals("dim=field path=[] value=0 childCount=0\n", result.toString()); + + r.close(); + d.close(); + } + + public void testEmptyRangesMultiValued() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + Document doc = new Document(); + SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("field", 0L); + SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("field", 0L); + doc.add(field1); + doc.add(field2); + for (long l = 0; l < 100; l++) { + field1.setLongValue(l); + field2.setLongValue(l); + w.addDocument(doc); + } + + IndexReader r = w.getReader(); + w.close(); + + FacetsCollector fc = new FacetsCollector(); + IndexSearcher s = newSearcher(r); + s.search(new MatchAllDocsQuery(), fc); + + Facets facets = new LongRangeFacetCounts("field", fc); + + FacetResult result = facets.getTopChildren(10, "field"); + assertEquals("dim=field path=[] value=0 childCount=0\n", result.toString()); + + r.close(); + d.close(); + } + /** * Tests single request that mixes Range and non-Range faceting, with DrillSideways and taxonomy. */ @@ -402,7 +558,96 @@ public class TestRangeFacetCounts extends FacetTestCase { IOUtils.close(r, d); } - public void testRandomLongs() throws Exception { + public void testBasicDoubleMultiValued() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + Document doc = new Document(); + // index the same value twice and make sure we don't double count + SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("field", 0); + SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("field", 0); + doc.add(field1); + doc.add(field2); + for (long l = 0; l < 100; l++) { + field1.setLongValue(NumericUtils.doubleToSortableLong(l)); + field2.setLongValue(NumericUtils.doubleToSortableLong(l)); + w.addDocument(doc); + } + + IndexReader r = w.getReader(); + + FacetsCollector fc = new FacetsCollector(); + + IndexSearcher s = newSearcher(r); + s.search(new MatchAllDocsQuery(), fc); + Facets facets = + new DoubleRangeFacetCounts( + "field", + fc, + new DoubleRange("less than 10", 0.0, true, 10.0, false), + new DoubleRange("less than or equal to 10", 0.0, true, 10.0, true), + new DoubleRange("over 90", 90.0, false, 100.0, false), + new DoubleRange("90 or above", 90.0, true, 100.0, false), + new DoubleRange("over 1000", 1000.0, false, Double.POSITIVE_INFINITY, false)); + + assertEquals( + "dim=field path=[] value=21 childCount=5\n less than 10 (10)\n less than or equal to 10 (11)\n over 90 (9)\n 90 or above (10)\n over 1000 (0)\n", + facets.getTopChildren(10, "field").toString()); + w.close(); + IOUtils.close(r, d); + } + + public void testBasicDoubleMultiValuedMixedSegmentTypes() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("field", 0L); + SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("field", 0L); + // write docs as two segments (50 in each). the first segment will contain a mix of single- and + // multi-value cases, while the second segment will be all single values. + for (int l = 0; l < 100; l++) { + field1.setLongValue(NumericUtils.doubleToSortableLong(l)); + field2.setLongValue(NumericUtils.doubleToSortableLong(l)); + Document doc = new Document(); + doc.add(field1); + if (l == 0) { + doc.add(field2); + } else if (l < 50) { + if (random().nextBoolean()) { + doc.add(field2); + } + } + w.addDocument(doc); + if (l == 50) { + w.commit(); + } + } + + IndexReader r = w.getReader(); + w.close(); + + FacetsCollector fc = new FacetsCollector(); + IndexSearcher s = newSearcher(r); + s.search(new MatchAllDocsQuery(), fc); + + Facets facets = + new DoubleRangeFacetCounts( + "field", + fc, + new DoubleRange("less than 10", 0.0, true, 10.0, false), + new DoubleRange("less than or equal to 10", 0.0, true, 10.0, true), + new DoubleRange("over 90", 90.0, false, 100.0, false), + new DoubleRange("90 or above", 90.0, true, 100.0, false), + new DoubleRange("over 1000", 1000.0, false, Double.POSITIVE_INFINITY, false)); + + FacetResult result = facets.getTopChildren(10, "field"); + assertEquals( + "dim=field path=[] value=21 childCount=5\n less than 10 (10)\n less than or equal to 10 (11)\n over 90 (9)\n 90 or above (10)\n over 1000 (0)\n", + result.toString()); + + r.close(); + d.close(); + } + + public void testRandomLongsSingleValued() throws Exception { Directory dir = newDirectory(); RandomIndexWriter w = new RandomIndexWriter(random(), dir); @@ -550,7 +795,166 @@ public class TestRangeFacetCounts extends FacetTestCase { IOUtils.close(r, dir); } - public void testRandomDoubles() throws Exception { + public void testRandomLongsMultiValued() throws Exception { + Directory dir = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), dir); + + int numDocs = atLeast(1000); + if (VERBOSE) { + System.out.println("TEST: numDocs=" + numDocs); + } + long[][] values = new long[numDocs][]; + long minValue = Long.MAX_VALUE; + long maxValue = Long.MIN_VALUE; + for (int i = 0; i < numDocs; i++) { + Document doc = new Document(); + int numVals = RandomNumbers.randomIntBetween(random(), 1, 50); + if (random().nextInt(10) == 0) { + numVals = 1; // make sure we have ample testing of single-value cases + } + values[i] = new long[numVals]; + for (int j = 0; j < numVals; j++) { + long v = random().nextLong(); + values[i][j] = v; + doc.add(new SortedNumericDocValuesField("field", v)); + doc.add(new LongPoint("field", v)); + minValue = Math.min(minValue, v); + maxValue = Math.max(maxValue, v); + } + w.addDocument(doc); + } + IndexReader r = w.getReader(); + + IndexSearcher s = newSearcher(r, false); + FacetsConfig config = new FacetsConfig(); + + int numIters = atLeast(10); + for (int iter = 0; iter < numIters; iter++) { + if (VERBOSE) { + System.out.println("TEST: iter=" + iter); + } + int numRange = TestUtil.nextInt(random(), 1, 100); + LongRange[] ranges = new LongRange[numRange]; + int[] expectedCounts = new int[numRange]; + long minAcceptedValue = Long.MAX_VALUE; + long maxAcceptedValue = Long.MIN_VALUE; + for (int rangeID = 0; rangeID < numRange; rangeID++) { + long min; + if (rangeID > 0 && random().nextInt(10) == 7) { + // Use an existing boundary: + LongRange prevRange = ranges[random().nextInt(rangeID)]; + if (random().nextBoolean()) { + min = prevRange.min; + } else { + min = prevRange.max; + } + } else { + min = random().nextLong(); + } + long max; + if (rangeID > 0 && random().nextInt(10) == 7) { + // Use an existing boundary: + LongRange prevRange = ranges[random().nextInt(rangeID)]; + if (random().nextBoolean()) { + max = prevRange.min; + } else { + max = prevRange.max; + } + } else { + max = random().nextLong(); + } + + if (min > max) { + long x = min; + min = max; + max = x; + } + boolean minIncl; + boolean maxIncl; + + // NOTE: max - min >= 0 is here to handle the common overflow case! + if (max - min >= 0 && max - min < 2) { + // If max == min or max == min+1, we always do inclusive, else we might pass an empty + // range and hit exc from LongRange's ctor: + minIncl = true; + maxIncl = true; + } else { + minIncl = random().nextBoolean(); + maxIncl = random().nextBoolean(); + } + ranges[rangeID] = new LongRange("r" + rangeID, min, minIncl, max, maxIncl); + if (VERBOSE) { + System.out.println(" range " + rangeID + ": " + ranges[rangeID]); + } + + // Do "slow but hopefully correct" computation of + // expected count: + for (int i = 0; i < numDocs; i++) { + for (int j = 0; j < values[i].length; j++) { + boolean accept = true; + if (minIncl) { + accept &= values[i][j] >= min; + } else { + accept &= values[i][j] > min; + } + if (maxIncl) { + accept &= values[i][j] <= max; + } else { + accept &= values[i][j] < max; + } + if (accept) { + expectedCounts[rangeID]++; + minAcceptedValue = Math.min(minAcceptedValue, values[i][j]); + maxAcceptedValue = Math.max(maxAcceptedValue, values[i][j]); + break; // ensure each doc can contribute at most 1 count to each range + } + } + } + } + + FacetsCollector sfc = new FacetsCollector(); + s.search(new MatchAllDocsQuery(), sfc); + Query fastMatchQuery; + if (random().nextBoolean()) { + if (random().nextBoolean()) { + fastMatchQuery = LongPoint.newRangeQuery("field", minValue, maxValue); + } else { + fastMatchQuery = LongPoint.newRangeQuery("field", minAcceptedValue, maxAcceptedValue); + } + } else { + fastMatchQuery = null; + } + Facets facets = new LongRangeFacetCounts("field", null, sfc, fastMatchQuery, ranges); + FacetResult result = facets.getTopChildren(10, "field"); + assertEquals(numRange, result.labelValues.length); + for (int rangeID = 0; rangeID < numRange; rangeID++) { + if (VERBOSE) { + System.out.println(" range " + rangeID + " expectedCount=" + expectedCounts[rangeID]); + } + LabelAndValue subNode = result.labelValues[rangeID]; + assertEquals("r" + rangeID, subNode.label); + assertEquals(expectedCounts[rangeID], subNode.value.intValue()); + + LongRange range = ranges[rangeID]; + + // Test drill-down: + DrillDownQuery ddq = new DrillDownQuery(config); + if (random().nextBoolean()) { + ddq.add("field", LongPoint.newRangeQuery("field", range.min, range.max)); + } else { + ddq.add( + "field", + SortedNumericDocValuesField.newSlowRangeQuery("field", range.min, range.max)); + } + assertEquals(expectedCounts[rangeID], s.count(ddq)); + } + } + + w.close(); + IOUtils.close(r, dir); + } + + public void testRandomDoublesSingleValued() throws Exception { Directory dir = newDirectory(); RandomIndexWriter w = new RandomIndexWriter(random(), dir); @@ -694,6 +1098,162 @@ public class TestRangeFacetCounts extends FacetTestCase { IOUtils.close(r, dir); } + public void testRandomDoublesMultiValued() throws Exception { + Directory dir = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), dir); + + int numDocs = atLeast(1000); + double[][] values = new double[numDocs][]; + double minValue = Double.POSITIVE_INFINITY; + double maxValue = Double.NEGATIVE_INFINITY; + for (int i = 0; i < numDocs; i++) { + Document doc = new Document(); + int numVals = RandomNumbers.randomIntBetween(random(), 1, 50); + if (random().nextInt(10) == 0) { + numVals = 1; // make sure we have ample testing of single-value cases + } + values[i] = new double[numVals]; + for (int j = 0; j < numVals; j++) { + double v = random().nextDouble(); + values[i][j] = v; + doc.add(new SortedNumericDocValuesField("field", Double.doubleToLongBits(v))); + doc.add(new DoublePoint("field", v)); + minValue = Math.min(minValue, v); + maxValue = Math.max(maxValue, v); + } + w.addDocument(doc); + } + IndexReader r = w.getReader(); + + IndexSearcher s = newSearcher(r, false); + FacetsConfig config = new FacetsConfig(); + + int numIters = atLeast(10); + for (int iter = 0; iter < numIters; iter++) { + if (VERBOSE) { + System.out.println("TEST: iter=" + iter); + } + int numRange = TestUtil.nextInt(random(), 1, 5); + DoubleRange[] ranges = new DoubleRange[numRange]; + int[] expectedCounts = new int[numRange]; + double minAcceptedValue = Double.POSITIVE_INFINITY; + double maxAcceptedValue = Double.NEGATIVE_INFINITY; + for (int rangeID = 0; rangeID < numRange; rangeID++) { + double min; + if (rangeID > 0 && random().nextInt(10) == 7) { + // Use an existing boundary: + DoubleRange prevRange = ranges[random().nextInt(rangeID)]; + if (random().nextBoolean()) { + min = prevRange.min; + } else { + min = prevRange.max; + } + } else { + min = random().nextDouble(); + } + double max; + if (rangeID > 0 && random().nextInt(10) == 7) { + // Use an existing boundary: + DoubleRange prevRange = ranges[random().nextInt(rangeID)]; + if (random().nextBoolean()) { + max = prevRange.min; + } else { + max = prevRange.max; + } + } else { + max = random().nextDouble(); + } + + if (min > max) { + double x = min; + min = max; + max = x; + } + + boolean minIncl; + boolean maxIncl; + + long minAsLong = NumericUtils.doubleToSortableLong(min); + long maxAsLong = NumericUtils.doubleToSortableLong(max); + // NOTE: maxAsLong - minAsLong >= 0 is here to handle the common overflow case! + if (maxAsLong - minAsLong >= 0 && maxAsLong - minAsLong < 2) { + minIncl = true; + maxIncl = true; + } else { + minIncl = random().nextBoolean(); + maxIncl = random().nextBoolean(); + } + ranges[rangeID] = new DoubleRange("r" + rangeID, min, minIncl, max, maxIncl); + + // Do "slow but hopefully correct" computation of + // expected count: + for (int i = 0; i < numDocs; i++) { + for (int j = 0; j < values[i].length; j++) { + boolean accept = true; + if (minIncl) { + accept &= values[i][j] >= min; + } else { + accept &= values[i][j] > min; + } + if (maxIncl) { + accept &= values[i][j] <= max; + } else { + accept &= values[i][j] < max; + } + if (accept) { + expectedCounts[rangeID]++; + minAcceptedValue = Math.min(minAcceptedValue, values[i][j]); + maxAcceptedValue = Math.max(maxAcceptedValue, values[i][j]); + break; // ensure each doc can contribute at most 1 count to each range + } + } + } + } + + FacetsCollector sfc = new FacetsCollector(); + s.search(new MatchAllDocsQuery(), sfc); + Query fastMatchFilter; + if (random().nextBoolean()) { + if (random().nextBoolean()) { + fastMatchFilter = DoublePoint.newRangeQuery("field", minValue, maxValue); + } else { + fastMatchFilter = DoublePoint.newRangeQuery("field", minAcceptedValue, maxAcceptedValue); + } + } else { + fastMatchFilter = null; + } + Facets facets = new DoubleRangeFacetCounts("field", null, sfc, fastMatchFilter, ranges); + FacetResult result = facets.getTopChildren(10, "field"); + assertEquals(numRange, result.labelValues.length); + for (int rangeID = 0; rangeID < numRange; rangeID++) { + if (VERBOSE) { + System.out.println(" range " + rangeID + " expectedCount=" + expectedCounts[rangeID]); + } + LabelAndValue subNode = result.labelValues[rangeID]; + assertEquals("r" + rangeID, subNode.label); + assertEquals(expectedCounts[rangeID], subNode.value.intValue()); + + DoubleRange range = ranges[rangeID]; + + // Test drill-down: + DrillDownQuery ddq = new DrillDownQuery(config); + if (random().nextBoolean()) { + ddq.add("field", DoublePoint.newRangeQuery("field", range.min, range.max)); + } else { + ddq.add( + "field", + SortedNumericDocValuesField.newSlowRangeQuery( + "field", Double.doubleToLongBits(range.min), Double.doubleToLongBits(range.max))); + } + + assertEquals(expectedCounts[rangeID], s.count(ddq)); + } + } + + w.close(); + IOUtils.close(r, dir); + } + // LUCENE-5178 public void testMissingValues() throws Exception { Directory d = newDirectory(); @@ -735,6 +1295,50 @@ public class TestRangeFacetCounts extends FacetTestCase { IOUtils.close(r, d); } + public void testMissingValuesMultiValued() throws Exception { + Directory d = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), d); + Document doc = new Document(); + // index the same field twice to test multi-valued logic + SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("field", 0L); + SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("field", 0L); + doc.add(field1); + doc.add(field2); + for (long l = 0; l < 100; l++) { + if (l % 5 == 0) { + // Every 5th doc is missing the value: + w.addDocument(new Document()); + continue; + } + field1.setLongValue(l); + field2.setLongValue(l); + w.addDocument(doc); + } + + IndexReader r = w.getReader(); + + FacetsCollector fc = new FacetsCollector(); + + IndexSearcher s = newSearcher(r); + s.search(new MatchAllDocsQuery(), fc); + Facets facets = + new LongRangeFacetCounts( + "field", + fc, + new LongRange("less than 10", 0L, true, 10L, false), + new LongRange("less than or equal to 10", 0L, true, 10L, true), + new LongRange("over 90", 90L, false, 100L, false), + new LongRange("90 or above", 90L, true, 100L, false), + new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, false)); + + assertEquals( + "dim=field path=[] value=16 childCount=5\n less than 10 (8)\n less than or equal to 10 (8)\n over 90 (8)\n 90 or above (8)\n over 1000 (0)\n", + facets.getTopChildren(10, "field").toString()); + + w.close(); + IOUtils.close(r, d); + } + private static class UsedQuery extends Query { private final AtomicBoolean used;