LUCENE-9950: New facet counting implementation for general string doc value fields (#133)

Co-authored-by: Greg Miller <gmiller@amazon.com>
This commit is contained in:
Greg Miller 2021-05-18 07:28:00 -07:00 committed by GitHub
parent ba9fee502b
commit ade50f0796
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 863 additions and 0 deletions

View File

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

View File

@ -0,0 +1,78 @@
/*
* 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.
*
* <p>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.
*/
public class StringDocValuesReaderState {
final IndexReader reader;
final String field;
final OrdinalMap ordinalMap;
/**
* 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);
}
private static OrdinalMap buildOrdinalMap(IndexReader reader, String field) throws IOException {
List<LeafReaderContext> 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 + ")";
}
}

View File

@ -0,0 +1,438 @@
/*
* 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 com.carrotsearch.hppc.IntIntScatterMap;
import com.carrotsearch.hppc.cursors.IntIntCursor;
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 complement {@link LongValueFacetCounts} in that they both provide facet
* counting on a doc value field with no assumptions of content.
*
* <p>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}.
*
* <p>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[] denseCounts;
private final IntIntScatterMap sparseCounts;
private final int cardinality;
private int totalDocCount;
/**
* 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();
long valueCount = docValues.getValueCount();
if (valueCount > Integer.MAX_VALUE) {
throw new IllegalArgumentException(
"can only handle valueCount < Integer.MAX_VALUE; got " + valueCount);
}
cardinality = (int) valueCount;
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];
}
}
count(facetsCollector);
} else {
// Since we're counting all ordinals, count densely:
sparseCounts = null;
denseCounts = new int[cardinality];
countAll();
}
}
@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, cardinality);
TopOrdAndIntQueue q = null;
TopOrdAndIntQueue.OrdAndValue reuse = null;
int bottomCount = 0;
int childCount = 0; // total number of labels with non-zero count
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 = cursor.key;
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;
}
}
}
} 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();
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 sparseCounts != null ? sparseCounts.get(ord) : denseCounts[ord];
}
@Override
public List<FacetResult> getAllDims(int topN) throws IOException {
return Collections.singletonList(getTopChildren(topN, field));
}
private SortedSetDocValues getDocValues() throws IOException {
List<LeafReaderContext> 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<FacetsCollector.MatchingDocs> 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 {
// 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);
// 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<LeafReaderContext> 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) {
increment(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 segmentCardinality = (int) segValues.getValueCount();
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) {
increment(term);
if (countedDocInTotal == false) {
totalDocCount++;
}
countedDocInTotal = true;
term = (int) segValues.nextOrd();
}
}
} else {
// 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;
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 < segmentCardinality; ord++) {
int count = segCounts[ord];
if (count != 0) {
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
// 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");
}
}
}

View File

@ -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<String, Integer> 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 testBasicSingleValuedUsingSortedDoc() 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<String, Integer> 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<String, Integer> 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<String, Integer> 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<String, Integer> 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<String> 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<String, Integer> 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<Map.Entry<String, Integer>> 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<FacetResult> 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);
}
}
}
}