From 37df5776b3655f4e07e827bd9a99870dd07bd918 Mon Sep 17 00:00:00 2001 From: Greg Miller Date: Thu, 6 May 2021 10:06:43 -0700 Subject: [PATCH 1/5] LUCENE-9950: New facet counting implementation for general string doc value fields --- lucene/CHANGES.txt | 3 + .../facet/StringDocValuesReaderState.java | 72 ++++ .../lucene/facet/StringValueFacetCounts.java | 371 ++++++++++++++++++ .../facet/TestStringValueFacetCounts.java | 344 ++++++++++++++++ 4 files changed, 790 insertions(+) create mode 100644 lucene/facet/src/java/org/apache/lucene/facet/StringDocValuesReaderState.java create mode 100644 lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java create mode 100644 lucene/facet/src/test/org/apache/lucene/facet/TestStringValueFacetCounts.java diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index cd6898398751..38224296e70e 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -19,6 +19,9 @@ New Features * LUCENE-9507: Custom order for leaves in IndexReader and IndexWriter (Mayya Sharipova, Mike McCandless, Jim Ferenczi) +* LUCENE-9950: New facet counting implementation for general string doc value fields + (SortedSetDocValues / SortedDocValues) not created through FacetsConfig (Greg Miller) + System Requirements * LUCENE-8738: Move to Java 11 as minimum Java version. diff --git a/lucene/facet/src/java/org/apache/lucene/facet/StringDocValuesReaderState.java b/lucene/facet/src/java/org/apache/lucene/facet/StringDocValuesReaderState.java new file mode 100644 index 000000000000..8be2ed679163 --- /dev/null +++ b/lucene/facet/src/java/org/apache/lucene/facet/StringDocValuesReaderState.java @@ -0,0 +1,72 @@ +/* + * 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; + +import java.io.IOException; +import java.util.List; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.OrdinalMap; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.util.packed.PackedInts; + +/** + * Stores an {@link OrdinalMap} created for a specific {@link IndexReader} ({@code reader}) + {@code + * field}. Enables re-use of the {@code ordinalMap} once created since creation is costly. + * + *

Note: It's important that callers confirm the ordinal map is still valid for their cases. + * Specifically, callers should confirm that the reader used to create the map ({@code reader}) + * matches their use-case. + */ +class StringDocValuesReaderState { + + final IndexReader reader; + final String field; + final OrdinalMap ordinalMap; + + StringDocValuesReaderState(IndexReader reader, String field) throws IOException { + this.reader = reader; + this.field = field; + ordinalMap = buildOrdinalMap(reader, field); + } + + private static OrdinalMap buildOrdinalMap(IndexReader reader, String field) throws IOException { + List leaves = reader.leaves(); + int leafCount = leaves.size(); + + if (leafCount <= 1) { + return null; + } + + SortedSetDocValues[] docValues = new SortedSetDocValues[leafCount]; + for (int i = 0; i < leafCount; i++) { + LeafReaderContext context = reader.leaves().get(i); + docValues[i] = DocValues.getSortedSet(context.reader(), field); + } + + IndexReader.CacheHelper cacheHelper = reader.getReaderCacheHelper(); + IndexReader.CacheKey owner = cacheHelper == null ? null : cacheHelper.getKey(); + + return OrdinalMap.build(owner, docValues, PackedInts.DEFAULT); + } + + @Override + public String toString() { + return "StringDocValuesReaderState(field=" + field + " reader=" + reader + ")"; + } +} diff --git a/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java new file mode 100644 index 000000000000..a5c96f77bceb --- /dev/null +++ b/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java @@ -0,0 +1,371 @@ +/* + * 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; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.MultiDocValues; +import org.apache.lucene.index.OrdinalMap; +import org.apache.lucene.index.ReaderUtil; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.search.ConjunctionDISI; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.LongValues; + +/** + * Compute facet counts from a previously indexed {@link SortedSetDocValues} or {@link + * org.apache.lucene.index.SortedDocValues} field. This approach will execute facet counting against + * the string values found in the specified field, with no assumptions on their format. Unlike + * {@link org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts}, no assumption is made + * about a "dimension" path component being indexed. Because of this, the field itself is + * effectively treated as the "dimension", and counts for all unique string values are produced. + * This approach is meant to compliment {@link LongValueFacetCounts} in that they both provide facet + * counting on a doc value field with no assumptions of content. + * + *

This implementation is useful if you want to dynamically count against any string doc value + * field without relying on {@link FacetField} and {@link FacetsConfig}. The disadvantage is that a + * separate field is required for each "dimension". If you want to pack multiple dimensions into the + * same doc values field, you probably want one of {@link + * org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts} or {@link + * org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts}. + * + *

Note that there is an added cost on every {@link IndexReader} open to create a new {@link + * StringDocValuesReaderState}. Also note that this class should be instantiated and used from a + * single thread, because it holds a thread-private instance of {@link SortedSetDocValues}. + * + * @lucene.experimental + */ +// TODO: Add a concurrent version much like ConcurrentSortedSetDocValuesFacetCounts? +public class StringValueFacetCounts extends Facets { + + private final IndexReader reader; + private final String field; + private final OrdinalMap ordinalMap; + private final SortedSetDocValues docValues; + + private final int[] counts; + + private int totalDocCount = 0; + + /** + * Returns all facet counts for the field, same result as searching on {@link MatchAllDocsQuery} + * but faster. + */ + public StringValueFacetCounts(StringDocValuesReaderState state) throws IOException { + this(state, null); + } + + /** Counts facets across the provided hits. */ + public StringValueFacetCounts(StringDocValuesReaderState state, FacetsCollector facetsCollector) + throws IOException { + reader = state.reader; + field = state.field; + ordinalMap = state.ordinalMap; + docValues = getDocValues(); + + // Since we accumulate counts in an array, we need to ensure the number of unique ordinals + // doesn't overflow an integer: + if (docValues.getValueCount() > Integer.MAX_VALUE) { + throw new IllegalArgumentException( + "can only handle valueCount < Integer.MAX_VALUE; got " + docValues.getValueCount()); + } + + counts = new int[(int) docValues.getValueCount()]; + + if (facetsCollector == null) { + countAll(); + } else { + count(facetsCollector); + } + } + + @Override + public FacetResult getTopChildren(int topN, String dim, String... path) throws IOException { + if (topN <= 0) { + throw new IllegalArgumentException("topN must be > 0 (got: " + topN + ")"); + } + if (dim.equals(field) == false) { + throw new IllegalArgumentException( + "invalid dim \"" + dim + "\"; should be \"" + field + "\""); + } + if (path.length != 0) { + throw new IllegalArgumentException("path.length should be 0"); + } + + topN = Math.min(topN, counts.length); + TopOrdAndIntQueue q = null; + TopOrdAndIntQueue.OrdAndValue reuse = null; + int bottomCount = 0; + int childCount = 0; // total number of labels with non-zero count + + for (int i = 0; i < counts.length; i++) { + int count = counts[i]; + if (count != 0) { + childCount++; + if (count > bottomCount) { + if (reuse == null) { + reuse = new TopOrdAndIntQueue.OrdAndValue(); + } + reuse.ord = i; + reuse.value = count; + if (q == null) { + // Lazy init for sparse case: + q = new TopOrdAndIntQueue(topN); + } + reuse = q.insertWithOverflow(reuse); + if (q.size() == topN) { + bottomCount = q.top().value; + } + } + } + } + + int resultCount = q == null ? 0 : q.size(); + LabelAndValue[] labelValues = new LabelAndValue[resultCount]; + for (int i = labelValues.length - 1; i >= 0; i--) { + TopOrdAndIntQueue.OrdAndValue ordAndValue = q.pop(); + final BytesRef term = docValues.lookupOrd(ordAndValue.ord); + labelValues[i] = new LabelAndValue(term.utf8ToString(), ordAndValue.value); + } + + return new FacetResult(field, new String[0], totalDocCount, labelValues, childCount); + } + + @Override + public Number getSpecificValue(String dim, String... path) throws IOException { + if (dim.equals(field) == false) { + throw new IllegalArgumentException( + "invalid dim \"" + dim + "\"; should be \"" + field + "\""); + } + if (path.length != 1) { + throw new IllegalArgumentException("path must be length=1"); + } + int ord = (int) docValues.lookupTerm(new BytesRef(path[0])); + if (ord < 0) { + return -1; + } + + return counts[ord]; + } + + @Override + public List getAllDims(int topN) throws IOException { + return Collections.singletonList(getTopChildren(topN, field)); + } + + private SortedSetDocValues getDocValues() throws IOException { + + List leaves = reader.leaves(); + int leafCount = leaves.size(); + + if (leafCount == 0) { + return DocValues.emptySortedSet(); + } + + if (leafCount == 1) { + return DocValues.getSortedSet(leaves.get(0).reader(), field); + } + + // A good bit of this logic is forked from MultiDocValues so we can re-use an ordinal map + SortedSetDocValues[] docValues = new SortedSetDocValues[leafCount]; + int[] starts = new int[leafCount + 1]; + long cost = 0; + for (int i = 0; i < leafCount; i++) { + LeafReaderContext context = leaves.get(i); + SortedSetDocValues dv = DocValues.getSortedSet(context.reader(), field); + docValues[i] = dv; + starts[i] = context.docBase; + cost += dv.cost(); + } + starts[leafCount] = reader.maxDoc(); + + return new MultiDocValues.MultiSortedSetDocValues(docValues, starts, ordinalMap, cost); + } + + private void count(FacetsCollector facetsCollector) throws IOException { + + List matchingDocs = facetsCollector.getMatchingDocs(); + + if (matchingDocs.isEmpty()) { + return; + } + + if (matchingDocs.size() == 1) { + + FacetsCollector.MatchingDocs hits = matchingDocs.get(0); + + // Validate state before doing anything else: + validateState(hits.context); + + // Assuming the state is valid, ordinalMap should be null since we have one segment: + assert ordinalMap == null; + + countOneSegment(docValues, hits.context.ord, hits); + } else { + for (int i = 0; i < matchingDocs.size(); i++) { + + FacetsCollector.MatchingDocs hits = matchingDocs.get(i); + + // Validate state before doing anything else: + validateState(hits.context); + + // Assuming the state is valid, ordinalMap should be non-null and docValues should be + // a MultiSortedSetDocValues since we have more than one segment: + assert ordinalMap != null; + assert docValues instanceof MultiDocValues.MultiSortedSetDocValues; + + MultiDocValues.MultiSortedSetDocValues multiValues = + (MultiDocValues.MultiSortedSetDocValues) docValues; + + countOneSegment(multiValues.values[i], hits.context.ord, hits); + } + } + } + + private void countAll() throws IOException { + + List leaves = reader.leaves(); + int numLeaves = leaves.size(); + if (numLeaves == 0) { + return; + } + + if (numLeaves == 1) { + // Since we have a single segment, ordinalMap should be null: + assert ordinalMap == null; + + LeafReaderContext context = leaves.get(0); + countOneSegment(docValues, context.ord, null); + } else { + // Since we have more than one segment, we should have a non-null ordinalMap and docValues + // should be a MultiSortedSetDocValues instance: + assert ordinalMap != null; + assert docValues instanceof MultiDocValues.MultiSortedSetDocValues; + + MultiDocValues.MultiSortedSetDocValues multiValues = + (MultiDocValues.MultiSortedSetDocValues) docValues; + + for (int i = 0; i < numLeaves; i++) { + LeafReaderContext context = leaves.get(i); + countOneSegment(multiValues.values[i], context.ord, null); + } + } + } + + private void countOneSegment( + SortedSetDocValues segValues, int segmentOrd, FacetsCollector.MatchingDocs hits) + throws IOException { + + // Intersect hits with doc values unless we're "counting all," in which case we'll iterate + // all doc values for this segment: + DocIdSetIterator it; + if (hits == null) { + it = segValues; + } else { + it = ConjunctionDISI.intersectIterators(Arrays.asList(hits.bits.iterator(), segValues)); + } + + // TODO: yet another option is to count all segs + // first, only in seg-ord space, and then do a + // merge-sort-PQ in the end to only "resolve to + // global" those seg ords that can compete, if we know + // we just want top K? ie, this is the same algo + // that'd be used for merging facets across shards + // (distributed faceting). but this has much higher + // temp ram req'ts (sum of number of ords across all + // segs) + if (ordinalMap == null) { + // If there's no ordinal map we don't need to map segment ordinals to globals, so counting + // is very straight-forward: + for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) { + int term = (int) segValues.nextOrd(); + boolean countedDocInTotal = false; + while (term != SortedSetDocValues.NO_MORE_ORDS) { + counts[term]++; + if (countedDocInTotal == false) { + totalDocCount++; + } + countedDocInTotal = true; + term = (int) segValues.nextOrd(); + } + } + } else { + // We need to map segment ordinals to globals. We have two different approaches to this + // depending on how many hits we have to count relative to how many unique doc val ordinals + // there are in this segment: + final LongValues ordMap = ordinalMap.getGlobalOrds(segmentOrd); + int numSegOrds = (int) segValues.getValueCount(); + + if (hits != null && hits.totalHits < numSegOrds / 10) { + // Remap every ord to global ord as we iterate: + for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) { + int term = (int) segValues.nextOrd(); + boolean countedDocInTotal = false; + while (term != SortedSetDocValues.NO_MORE_ORDS) { + counts[term]++; + if (countedDocInTotal == false) { + totalDocCount++; + } + countedDocInTotal = true; + term = (int) segValues.nextOrd(); + } + } + } else { + // First count in seg-ord space: + final int[] segCounts = new int[numSegOrds]; + for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) { + int term = (int) segValues.nextOrd(); + boolean countedDocInTotal = false; + while (term != SortedSetDocValues.NO_MORE_ORDS) { + segCounts[term]++; + if (countedDocInTotal == false) { + totalDocCount++; + } + countedDocInTotal = true; + term = (int) segValues.nextOrd(); + } + } + + // Then, migrate to global ords: + for (int ord = 0; ord < numSegOrds; ord++) { + int count = segCounts[ord]; + if (count != 0) { + counts[(int) ordMap.get(ord)] += count; + } + } + } + } + } + + private void validateState(LeafReaderContext context) { + // LUCENE-5090: make sure the provided reader context "matches" + // the top-level reader passed to the + // SortedSetDocValuesReaderState, else cryptic + // AIOOBE can happen: + if (ReaderUtil.getTopLevelContext(context).reader() != reader) { + throw new IllegalStateException( + "the SortedSetDocValuesReaderState provided to this class does not match the reader being searched; you must create a new SortedSetDocValuesReaderState every time you open a new IndexReader"); + } + } +} diff --git a/lucene/facet/src/test/org/apache/lucene/facet/TestStringValueFacetCounts.java b/lucene/facet/src/test/org/apache/lucene/facet/TestStringValueFacetCounts.java new file mode 100644 index 000000000000..b4073ae604dc --- /dev/null +++ b/lucene/facet/src/test/org/apache/lucene/facet/TestStringValueFacetCounts.java @@ -0,0 +1,344 @@ +/* + * 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; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.SortedDocValuesField; +import org.apache.lucene.document.SortedSetDocValuesField; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.LuceneTestCase; + +public class TestStringValueFacetCounts extends FacetTestCase { + + public void testBasicSingleValued() throws Exception { + + Directory dir = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random(), dir); + + Document doc = new Document(); + doc.add(new SortedSetDocValuesField("field", new BytesRef("foo"))); + writer.addDocument(doc); + + doc = new Document(); + doc.add(new SortedSetDocValuesField("field", new BytesRef("foo"))); + writer.addDocument(doc); + + doc = new Document(); + doc.add(new SortedSetDocValuesField("field", new BytesRef("bar"))); + writer.addDocument(doc); + + doc = new Document(); + doc.add(new SortedSetDocValuesField("field", new BytesRef("bar"))); + writer.addDocument(doc); + + doc = new Document(); + doc.add(new SortedSetDocValuesField("field", new BytesRef("baz"))); + writer.addDocument(doc); + + Map expectedCounts = Map.of("foo", 2, "bar", 2, "baz", 1); + int expectedTotalDocCount = 5; + + IndexSearcher searcher = newSearcher(writer.getReader()); + writer.close(); + + checkFacetResult(expectedCounts, expectedTotalDocCount, searcher, 10, 2, 1, 0); + + IOUtils.close(searcher.getIndexReader(), dir); + } + + public void testBasicingleValuedUsingSortedDoc() throws Exception { + + Directory dir = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random(), dir); + + Document doc = new Document(); + doc.add(new SortedDocValuesField("field", new BytesRef("foo"))); + writer.addDocument(doc); + + doc = new Document(); + doc.add(new SortedDocValuesField("field", new BytesRef("foo"))); + writer.addDocument(doc); + + doc = new Document(); + doc.add(new SortedDocValuesField("field", new BytesRef("bar"))); + writer.addDocument(doc); + + doc = new Document(); + doc.add(new SortedDocValuesField("field", new BytesRef("bar"))); + writer.addDocument(doc); + + doc = new Document(); + doc.add(new SortedDocValuesField("field", new BytesRef("baz"))); + writer.addDocument(doc); + + Map expectedCounts = Map.of("foo", 2, "bar", 2, "baz", 1); + int expectedTotalDocCount = 5; + + IndexSearcher searcher = newSearcher(writer.getReader()); + writer.close(); + + checkFacetResult(expectedCounts, expectedTotalDocCount, searcher, 10, 2, 1, 0); + + IOUtils.close(searcher.getIndexReader(), dir); + } + + public void testBasicMultiValued() throws Exception { + + Directory dir = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random(), dir); + + Document doc = new Document(); + doc.add(new SortedSetDocValuesField("field", new BytesRef("foo"))); + doc.add(new SortedSetDocValuesField("field", new BytesRef("bar"))); + writer.addDocument(doc); + + doc = new Document(); + doc.add(new SortedSetDocValuesField("field", new BytesRef("foo"))); + doc.add(new SortedSetDocValuesField("field", new BytesRef("baz"))); + writer.addDocument(doc); + + doc = new Document(); + doc.add(new SortedSetDocValuesField("field", new BytesRef("baz"))); + writer.addDocument(doc); + + Map expectedCounts = Map.of("foo", 2, "bar", 1, "baz", 2); + int expectedTotalDocCount = 3; + + IndexSearcher searcher = newSearcher(writer.getReader()); + writer.close(); + + checkFacetResult(expectedCounts, expectedTotalDocCount, searcher, 10, 2, 1, 0); + + IOUtils.close(searcher.getIndexReader(), dir); + } + + public void testMissingSegment() throws Exception { + + Directory dir = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random(), dir); + + Document doc = new Document(); + doc.add(new SortedSetDocValuesField("field", new BytesRef("foo"))); + doc.add(new SortedSetDocValuesField("field", new BytesRef("bar"))); + writer.addDocument(doc); + writer.commit(); + + // segment with no values + doc = new Document(); + writer.addDocument(doc); + writer.commit(); + + doc = new Document(); + doc.add(new SortedSetDocValuesField("field", new BytesRef("baz"))); + writer.addDocument(doc); + writer.commit(); + + Map expectedCounts = Map.of("foo", 1, "bar", 1, "baz", 1); + int expectedTotalDocCount = 2; + + IndexSearcher searcher = newSearcher(writer.getReader()); + writer.close(); + + checkFacetResult(expectedCounts, expectedTotalDocCount, searcher, 10, 2, 1, 0); + + IOUtils.close(searcher.getIndexReader(), dir); + } + + public void testStaleState() throws Exception { + + Directory dir = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random(), dir); + + Document doc = new Document(); + doc.add(new SortedSetDocValuesField("field", new BytesRef("foo"))); + writer.addDocument(doc); + + IndexReader reader = writer.getReader(); + StringDocValuesReaderState state = new StringDocValuesReaderState(reader, "field"); + + doc = new Document(); + doc.add(new SortedSetDocValuesField("field", new BytesRef("bar"))); + writer.addDocument(doc); + + IndexSearcher searcher = newSearcher(writer.getReader()); + writer.close(); + + FacetsCollector c = new FacetsCollector(); + searcher.search(new MatchAllDocsQuery(), c); + + // using a stale state + expectThrows(IllegalStateException.class, () -> new StringValueFacetCounts(state, c)); + + IOUtils.close(reader, searcher.getIndexReader(), dir); + } + + public void testRandom() throws Exception { + + Directory dir = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter(random(), dir); + + // Build up test data + String[] tokens = getRandomTokens(50); // 50 random values to pick from + int numDocs = atLeast(1000); + int expectedTotalDocCount = 0; + Map expected = new HashMap<>(); + for (int i = 0; i < numDocs; i++) { + Document doc = new Document(); + int valCount = random().nextInt(5); // each doc can have up to 5 values + Set docVals = new HashSet<>(); + for (int j = 0; j < valCount; j++) { + int tokenIdx = random().nextInt(tokens.length); + String val = tokens[tokenIdx]; + // values should only be counted once per document + if (docVals.contains(val) == false) { + expected.put(val, expected.getOrDefault(val, 0) + 1); + } + docVals.add(val); + doc.add(new SortedSetDocValuesField("field", new BytesRef(val))); + } + // only docs with at least one value in the field should be counted in the total + if (docVals.isEmpty() == false) { + expectedTotalDocCount++; + } + writer.addDocument(doc); + if (random().nextInt(10) == 0) { + writer.commit(); // sometimes commit + } + } + + IndexSearcher searcher = newSearcher(writer.getReader()); + writer.close(); + + // run iterations with random values of topN + int iterations = LuceneTestCase.TEST_NIGHTLY ? 10_000 : 50; + int[] topNs = new int[iterations]; + for (int i = 0; i < iterations; i++) { + topNs[i] = atLeast(1); + } + + checkFacetResult(expected, expectedTotalDocCount, searcher, topNs); + + IOUtils.close(searcher.getIndexReader(), dir); + } + + private void checkFacetResult( + Map expectedCounts, + int expectedTotalDocsWithValue, + IndexSearcher searcher, + int... topNs) + throws IOException { + + StringDocValuesReaderState state = + new StringDocValuesReaderState(searcher.getIndexReader(), "field"); + + FacetsCollector c = new FacetsCollector(); + searcher.search(new MatchAllDocsQuery(), c); + + for (int topN : topNs) { + + StringValueFacetCounts facets; + // should get the same result whether-or-not we provide a FacetsCollector since it's doing + // a MatchAllDocsQuery: + if (random().nextBoolean()) { + facets = new StringValueFacetCounts(state, c); + } else { + facets = new StringValueFacetCounts(state); + } + + // sort expected counts by count, value + List> expectedCountsSorted = + new ArrayList<>(expectedCounts.entrySet()); + expectedCountsSorted.sort( + (a, b) -> { + int cmp = b.getValue().compareTo(a.getValue()); // high-to-low + if (cmp == 0) { + cmp = a.getKey().compareTo(b.getKey()); // low-to-high + } + return cmp; + }); + + // number of labels we expect is the number with a non-zero count + int expectedLabelCount = + (int) expectedCountsSorted.stream().filter(e -> e.getValue() > 0).count(); + + // topN == 0 is intentionally unsupported + if (topN == 0) { + assertThrows(IllegalArgumentException.class, () -> facets.getTopChildren(topN, "field")); + return; + } + + FacetResult facetResult = facets.getTopChildren(topN, "field"); + + // also sort expected labels by count, value (these will be sorted by count, ord -- but since + // we have no insight into the ordinals assigned to the values, we resort) + Arrays.sort( + facetResult.labelValues, + (a, b) -> { + int cmp = Long.compare(b.value.longValue(), a.value.longValue()); // high-to-low + if (cmp == 0) { + cmp = a.label.compareTo(b.label); // low-to-high + } + return cmp; + }); + + assertEquals("field", facetResult.dim); + assertEquals(0, facetResult.path.length); + assertEquals(expectedTotalDocsWithValue, facetResult.value); + assertEquals(expectedLabelCount, facetResult.childCount); + + for (int i = 0; i < Math.min(topN, expectedCountsSorted.size()); i++) { + String expectedKey = expectedCountsSorted.get(i).getKey(); + int expectedValue = expectedCountsSorted.get(i).getValue(); + assertEquals(expectedKey, facetResult.labelValues[i].label); + assertEquals(expectedValue, facetResult.labelValues[i].value); + // make sure getSpecificValue reports the same count + assertEquals(expectedValue, facets.getSpecificValue("field", expectedKey)); + } + + // getAllDims should return a singleton list with the same results as getTopChildren + List allDims = facets.getAllDims(topN); + assertEquals(1, allDims.size()); + assertEquals(facetResult, allDims.get(0)); + + // execute a "drill down" query on one of the values at random and make sure the total hits + // match the expected count provided by faceting + if (expectedCountsSorted.isEmpty() == false) { + DrillDownQuery q = new DrillDownQuery(new FacetsConfig()); + int randomTestValIdx = random().nextInt(Math.min(expectedCountsSorted.size(), topN)); + q.add("field", expectedCountsSorted.get(randomTestValIdx).getKey()); + searcher.search(q, 1); + assertEquals( + expectedCountsSorted.get(randomTestValIdx).getValue(), + facetResult.labelValues[randomTestValIdx].value); + } + } + } +} From 7781f0dd9a847aca3c1c9869612b8ca3a887bf95 Mon Sep 17 00:00:00 2001 From: Greg Miller Date: Fri, 14 May 2021 11:40:35 -0700 Subject: [PATCH 2/5] PR feedback --- .../lucene/facet/StringDocValuesReaderState.java | 10 ++++++++-- .../apache/lucene/facet/StringValueFacetCounts.java | 10 +++++++++- 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/lucene/facet/src/java/org/apache/lucene/facet/StringDocValuesReaderState.java b/lucene/facet/src/java/org/apache/lucene/facet/StringDocValuesReaderState.java index 8be2ed679163..4507bc82b21b 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/StringDocValuesReaderState.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/StringDocValuesReaderState.java @@ -33,13 +33,19 @@ * Specifically, callers should confirm that the reader used to create the map ({@code reader}) * matches their use-case. */ -class StringDocValuesReaderState { +public class StringDocValuesReaderState { final IndexReader reader; final String field; final OrdinalMap ordinalMap; - StringDocValuesReaderState(IndexReader reader, String field) throws IOException { + /** + * Construct state specific to a reader + field. This builds an {@link OrdinalMap} that can be + * reused for mapping segment-specific ordinals to global ordinals for the given field. Keep in + * mind that the state is only valid for the specified {@link IndexReader}, so opening new readers + * (e.g., to pickup NRT updates) requires constructing a new state instance. + */ + public StringDocValuesReaderState(IndexReader reader, String field) throws IOException { this.reader = reader; this.field = field; ordinalMap = buildOrdinalMap(reader, field); diff --git a/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java index a5c96f77bceb..fa64f5b4d09a 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java @@ -54,6 +54,10 @@ * StringDocValuesReaderState}. Also note that this class should be instantiated and used from a * single thread, because it holds a thread-private instance of {@link SortedSetDocValues}. * + *

Also note that counting does not use a sparse data structure, so heap memory cost scales with + * the number of unique ordinals for the field being counting. For high-cardinality fields, this + * could be costly. + * * @lucene.experimental */ // TODO: Add a concurrent version much like ConcurrentSortedSetDocValuesFacetCounts? @@ -64,9 +68,13 @@ public class StringValueFacetCounts extends Facets { private final OrdinalMap ordinalMap; private final SortedSetDocValues docValues; + // TODO: There's an optimization opportunity here to use a sparse counting structure in some + // cases, + // much like what IntTaxonomyFacetCounts does. + /** Dense counting array indexed by ordinal. */ private final int[] counts; - private int totalDocCount = 0; + private int totalDocCount; /** * Returns all facet counts for the field, same result as searching on {@link MatchAllDocsQuery} From c710043545faa46b9df93343c0c7cafc88e676e4 Mon Sep 17 00:00:00 2001 From: Greg Miller Date: Fri, 14 May 2021 12:13:10 -0700 Subject: [PATCH 3/5] spelling misses --- .../java/org/apache/lucene/facet/StringValueFacetCounts.java | 2 +- .../org/apache/lucene/facet/TestStringValueFacetCounts.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java index fa64f5b4d09a..46da54055a38 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java @@ -40,7 +40,7 @@ * {@link org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetCounts}, no assumption is made * about a "dimension" path component being indexed. Because of this, the field itself is * effectively treated as the "dimension", and counts for all unique string values are produced. - * This approach is meant to compliment {@link LongValueFacetCounts} in that they both provide facet + * This approach is meant to complement {@link LongValueFacetCounts} in that they both provide facet * counting on a doc value field with no assumptions of content. * *

This implementation is useful if you want to dynamically count against any string doc value diff --git a/lucene/facet/src/test/org/apache/lucene/facet/TestStringValueFacetCounts.java b/lucene/facet/src/test/org/apache/lucene/facet/TestStringValueFacetCounts.java index b4073ae604dc..e13eb1b908e2 100644 --- a/lucene/facet/src/test/org/apache/lucene/facet/TestStringValueFacetCounts.java +++ b/lucene/facet/src/test/org/apache/lucene/facet/TestStringValueFacetCounts.java @@ -74,7 +74,7 @@ public void testBasicSingleValued() throws Exception { IOUtils.close(searcher.getIndexReader(), dir); } - public void testBasicingleValuedUsingSortedDoc() throws Exception { + public void testBasicSingleValuedUsingSortedDoc() throws Exception { Directory dir = newDirectory(); RandomIndexWriter writer = new RandomIndexWriter(random(), dir); From 3f3229e54cf523c90db566a0c02452a17dd923e5 Mon Sep 17 00:00:00 2001 From: Greg Miller Date: Fri, 14 May 2021 16:55:08 -0700 Subject: [PATCH 4/5] add sparse counting in addition to dense counting --- .../lucene/facet/StringValueFacetCounts.java | 121 +++++++++++++----- 1 file changed, 89 insertions(+), 32 deletions(-) diff --git a/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java index 46da54055a38..2286b8494ab6 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java @@ -16,6 +16,8 @@ */ package org.apache.lucene.facet; +import com.carrotsearch.hppc.IntIntScatterMap; +import com.carrotsearch.hppc.cursors.IntIntCursor; import java.io.IOException; import java.util.Arrays; import java.util.Collections; @@ -54,10 +56,6 @@ * StringDocValuesReaderState}. Also note that this class should be instantiated and used from a * single thread, because it holds a thread-private instance of {@link SortedSetDocValues}. * - *

Also note that counting does not use a sparse data structure, so heap memory cost scales with - * the number of unique ordinals for the field being counting. For high-cardinality fields, this - * could be costly. - * * @lucene.experimental */ // TODO: Add a concurrent version much like ConcurrentSortedSetDocValuesFacetCounts? @@ -68,12 +66,10 @@ public class StringValueFacetCounts extends Facets { private final OrdinalMap ordinalMap; private final SortedSetDocValues docValues; - // TODO: There's an optimization opportunity here to use a sparse counting structure in some - // cases, - // much like what IntTaxonomyFacetCounts does. - /** Dense counting array indexed by ordinal. */ - private final int[] counts; + private final int[] denseCounts; + private final IntIntScatterMap sparseCounts; + private final int cardinality; private int totalDocCount; /** @@ -92,19 +88,43 @@ public StringValueFacetCounts(StringDocValuesReaderState state, FacetsCollector ordinalMap = state.ordinalMap; docValues = getDocValues(); - // Since we accumulate counts in an array, we need to ensure the number of unique ordinals - // doesn't overflow an integer: - if (docValues.getValueCount() > Integer.MAX_VALUE) { + long valueCount = docValues.getValueCount(); + if (valueCount > Integer.MAX_VALUE) { throw new IllegalArgumentException( - "can only handle valueCount < Integer.MAX_VALUE; got " + docValues.getValueCount()); + "can only handle valueCount < Integer.MAX_VALUE; got " + valueCount); } + cardinality = (int) valueCount; - counts = new int[(int) docValues.getValueCount()]; + if (facetsCollector != null) { + if (cardinality < 1024) { // count densely for low cardinality + sparseCounts = null; + denseCounts = new int[cardinality]; + } else { + int totalHits = 0; + int totalDocs = 0; + for (FacetsCollector.MatchingDocs matchingDocs : facetsCollector.getMatchingDocs()) { + totalHits += matchingDocs.totalHits; + totalDocs += matchingDocs.context.reader().maxDoc(); + } + + // If our result set is < 10% of the index, we collect sparsely (use hash map). This + // heuristic is borrowed from IntTaxonomyFacetCounts: + if (totalHits < totalDocs / 10) { + sparseCounts = new IntIntScatterMap(); + denseCounts = null; + } else { + sparseCounts = null; + denseCounts = new int[cardinality]; + } + } - if (facetsCollector == null) { - countAll(); - } else { count(facetsCollector); + } else { + // Since we're counting all ordinals, count densely: + sparseCounts = null; + denseCounts = new int[cardinality]; + + countAll(); } } @@ -121,21 +141,21 @@ public FacetResult getTopChildren(int topN, String dim, String... path) throws I throw new IllegalArgumentException("path.length should be 0"); } - topN = Math.min(topN, counts.length); + topN = Math.min(topN, cardinality); TopOrdAndIntQueue q = null; TopOrdAndIntQueue.OrdAndValue reuse = null; int bottomCount = 0; int childCount = 0; // total number of labels with non-zero count - for (int i = 0; i < counts.length; i++) { - int count = counts[i]; - if (count != 0) { - childCount++; + if (sparseCounts != null) { + for (IntIntCursor cursor : sparseCounts) { + childCount++; // every count in sparseValues should be non-zero + int count = cursor.value; if (count > bottomCount) { if (reuse == null) { reuse = new TopOrdAndIntQueue.OrdAndValue(); } - reuse.ord = i; + reuse.ord = cursor.key; reuse.value = count; if (q == null) { // Lazy init for sparse case: @@ -147,6 +167,28 @@ public FacetResult getTopChildren(int topN, String dim, String... path) throws I } } } + } else { + for (int i = 0; i < denseCounts.length; i++) { + int count = denseCounts[i]; + if (count != 0) { + childCount++; + if (count > bottomCount) { + if (reuse == null) { + reuse = new TopOrdAndIntQueue.OrdAndValue(); + } + reuse.ord = i; + reuse.value = count; + if (q == null) { + // Lazy init for sparse case: + q = new TopOrdAndIntQueue(topN); + } + reuse = q.insertWithOverflow(reuse); + if (q.size() == topN) { + bottomCount = q.top().value; + } + } + } + } } int resultCount = q == null ? 0 : q.size(); @@ -174,7 +216,7 @@ public Number getSpecificValue(String dim, String... path) throws IOException { return -1; } - return counts[ord]; + return sparseCounts != null ? sparseCounts.get(ord) : denseCounts[ord]; } @Override @@ -310,7 +352,7 @@ private void countOneSegment( int term = (int) segValues.nextOrd(); boolean countedDocInTotal = false; while (term != SortedSetDocValues.NO_MORE_ORDS) { - counts[term]++; + increment(term); if (countedDocInTotal == false) { totalDocCount++; } @@ -323,15 +365,15 @@ private void countOneSegment( // depending on how many hits we have to count relative to how many unique doc val ordinals // there are in this segment: final LongValues ordMap = ordinalMap.getGlobalOrds(segmentOrd); - int numSegOrds = (int) segValues.getValueCount(); + int segmentCardinality = (int) segValues.getValueCount(); - if (hits != null && hits.totalHits < numSegOrds / 10) { + if (hits != null && hits.totalHits < segmentCardinality / 10) { // Remap every ord to global ord as we iterate: for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) { int term = (int) segValues.nextOrd(); boolean countedDocInTotal = false; while (term != SortedSetDocValues.NO_MORE_ORDS) { - counts[term]++; + increment(term); if (countedDocInTotal == false) { totalDocCount++; } @@ -340,8 +382,11 @@ private void countOneSegment( } } } else { - // First count in seg-ord space: - final int[] segCounts = new int[numSegOrds]; + // First count in seg-ord space. + // At this point, we're either counting all ordinals or our heuristic suggests that + // we expect to visit a large percentage of the unique ordinals (lots of hits relative + // to the segment cardinality), so we count the segment densely: + final int[] segCounts = new int[segmentCardinality]; for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) { int term = (int) segValues.nextOrd(); boolean countedDocInTotal = false; @@ -356,16 +401,28 @@ private void countOneSegment( } // Then, migrate to global ords: - for (int ord = 0; ord < numSegOrds; ord++) { + for (int ord = 0; ord < segmentCardinality; ord++) { int count = segCounts[ord]; if (count != 0) { - counts[(int) ordMap.get(ord)] += count; + increment((int) ordMap.get(ord), count); } } } } } + private void increment(int ordinal) { + increment(ordinal, 1); + } + + private void increment(int ordinal, int amount) { + if (sparseCounts != null) { + sparseCounts.addTo(ordinal, amount); + } else { + denseCounts[ordinal] += amount; + } + } + private void validateState(LeafReaderContext context) { // LUCENE-5090: make sure the provided reader context "matches" // the top-level reader passed to the From b5b703d750c034957fc036cd791b5cd9c9f551c4 Mon Sep 17 00:00:00 2001 From: Greg Miller Date: Mon, 17 May 2021 11:06:47 -0700 Subject: [PATCH 5/5] only check state match for the first segment --- .../org/apache/lucene/facet/StringValueFacetCounts.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java index 2286b8494ab6..8b874bc5a2bf 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/StringValueFacetCounts.java @@ -273,13 +273,15 @@ private void count(FacetsCollector facetsCollector) throws IOException { countOneSegment(docValues, hits.context.ord, hits); } else { + + // Validate state before doing anything else. We only check the first segment since they + // should all ladder up to the same top-level reader: + validateState(matchingDocs.get(0).context); + for (int i = 0; i < matchingDocs.size(); i++) { FacetsCollector.MatchingDocs hits = matchingDocs.get(i); - // Validate state before doing anything else: - validateState(hits.context); - // Assuming the state is valid, ordinalMap should be non-null and docValues should be // a MultiSortedSetDocValues since we have more than one segment: assert ordinalMap != null;