LUCENE-7927: add LongValueFacetCounts to compute facet count for individual numeric doc values

This commit is contained in:
Mike McCandless 2017-08-29 11:57:33 -04:00
parent 1cc3d8050c
commit de5c68b5f8
6 changed files with 1092 additions and 6 deletions

View File

@ -20,6 +20,9 @@ New Features
* LUCENE-7621: Add CoveringQuery, a query whose required number of matching
clauses can be defined per document. (Adrien Grand)
* LUCENE-7927: Add LongValueFacetCounts, to compute facet counts for individual
numeric values (Mike McCandless)
Optimizations
* LUCENE-7905: Optimize how OrdinalMap (used by

View File

@ -0,0 +1,499 @@
/*
* 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.Collections;
import java.util.List;
import org.apache.lucene.facet.FacetResult;
import org.apache.lucene.facet.Facets;
import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
import org.apache.lucene.facet.FacetsCollector;
import org.apache.lucene.facet.LabelAndValue;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.search.ConjunctionDISI;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.LongValues;
import org.apache.lucene.search.LongValuesSource;
import org.apache.lucene.util.InPlaceMergeSorter;
import org.apache.lucene.util.PriorityQueue;
/** {@link Facets} implementation that computes counts for
* all uniqute long values, more efficiently counting small values (0-1023) using an int array,
* and switching to a <code>HashMap</code> for values above 1023.
* Retrieve all facet counts, in value order, with {@link #getAllChildrenSortByValue},
* or get the topN values sorted by count with {@link #getTopChildrenSortByCount}.
*
* @lucene.experimental */
public class LongValueFacetCounts extends Facets {
/** Used for all values that are < 1K. */
private final int[] counts = new int[1024];
/** Used for all values that are >= 1K. */
private final HashTable hashCounts = new HashTable();
private final String field;
/** Total number of values counted, which is the subset of hits that had a value for this field. */
private int totCount;
/** Create {@code LongValueFacetCounts}, using either single-valued {@link
* NumericDocValues} or multi-valued {@link SortedNumericDocValues} from the
* specified field. */
public LongValueFacetCounts(String field, FacetsCollector hits, boolean multiValued) throws IOException {
this(field, null, hits, multiValued);
}
/** Create {@code LongValueFacetCounts}, using the provided
* {@link org.apache.lucene.queries.function.ValueSource}. If hits is
* null then all facets are counted. */
public LongValueFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits) throws IOException {
this(field, valueSource, hits, false);
}
/** Create {@code LongValueFacetCounts}, using the provided
* {@link org.apache.lucene.queries.function.ValueSource}.
* random access (implement {@link org.apache.lucene.search.DocIdSet#bits}). */
public LongValueFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits,
boolean multiValued) throws IOException {
this.field = field;
if (valueSource == null) {
if (multiValued) {
countMultiValued(field, hits.getMatchingDocs());
} else {
count(field, hits.getMatchingDocs());
}
} else {
// value source is always single valued
if (multiValued) {
throw new IllegalArgumentException("can only compute multi-valued facets directly from doc values (when valueSource is null)");
}
count(valueSource, hits.getMatchingDocs());
}
}
/** Counts all facet values for this reader. This produces the same result as computing
* facets on a {@link org.apache.lucene.search.MatchAllDocsQuery}, but is more efficient. */
public LongValueFacetCounts(String field, IndexReader reader, boolean multiValued) throws IOException {
this.field = field;
if (multiValued) {
countAllMultiValued(reader, field);
} else {
countAll(reader, field);
}
}
/** Counts all facet values for the provided {@link LongValuesSource}. This produces the same result as computing
* facets on a {@link org.apache.lucene.search.MatchAllDocsQuery}, but is more efficient. */
public LongValueFacetCounts(String field, LongValuesSource valueSource, IndexReader reader) throws IOException {
this.field = field;
countAll(valueSource, field, reader);
}
private void count(LongValuesSource valueSource, List<MatchingDocs> matchingDocs) throws IOException {
for (MatchingDocs hits : matchingDocs) {
LongValues fv = valueSource.getValues(hits.context, null);
// NOTE: this is not as efficient as working directly with the doc values APIs in the sparse case
// because we are doing a linear scan across all hits, but this API is more flexible since a
// LongValuesSource can compute interesting values at query time
DocIdSetIterator docs = hits.bits.iterator();
for (int doc = docs.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS;) {
// Skip missing docs:
if (fv.advanceExact(doc)) {
increment(fv.longValue());
totCount++;
}
doc = docs.nextDoc();
}
}
}
private void count(String field, List<MatchingDocs> matchingDocs) throws IOException {
for (MatchingDocs hits : matchingDocs) {
NumericDocValues fv = hits.context.reader().getNumericDocValues(field);
if (fv == null) {
continue;
}
countOneSegment(fv, hits);
}
}
private void countOneSegment(NumericDocValues values, MatchingDocs hits) throws IOException {
DocIdSetIterator it = ConjunctionDISI.intersectIterators(
Arrays.asList(hits.bits.iterator(), values));
for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
increment(values.longValue());
totCount++;
}
}
/** Counts directly from SortedNumericDocValues. */
private void countMultiValued(String field, List<MatchingDocs> matchingDocs) throws IOException {
for (MatchingDocs hits : matchingDocs) {
SortedNumericDocValues values = hits.context.reader().getSortedNumericDocValues(field);
if (values == null) {
// this field has no doc values for this segment
continue;
}
NumericDocValues singleValues = DocValues.unwrapSingleton(values);
if (singleValues != null) {
countOneSegment(singleValues, hits);
} else {
DocIdSetIterator it = ConjunctionDISI.intersectIterators(
Arrays.asList(hits.bits.iterator(), values));
for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
int limit = values.docValueCount();
totCount += limit;
for (int i = 0; i < limit; i++) {
increment(values.nextValue());
}
}
}
}
}
/** Optimized version that directly counts all doc values. */
private void countAll(IndexReader reader, String field) throws IOException {
for (LeafReaderContext context : reader.leaves()) {
NumericDocValues values = context.reader().getNumericDocValues(field);
if (values == null) {
// this field has no doc values for this segment
continue;
}
countAllOneSegment(values);
}
}
private void countAllOneSegment(NumericDocValues values) throws IOException {
int doc;
while ((doc = values.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
totCount++;
increment(values.longValue());
}
}
private void countAll(LongValuesSource valueSource, String field, IndexReader reader) throws IOException {
for (LeafReaderContext context : reader.leaves()) {
LongValues fv = valueSource.getValues(context, null);
int maxDoc = context.reader().maxDoc();
for (int doc = 0; doc < maxDoc; doc++) {
// Skip missing docs:
if (fv.advanceExact(doc)) {
increment(fv.longValue());
totCount++;
}
}
}
}
private void countAllMultiValued(IndexReader reader, String field) throws IOException {
for (LeafReaderContext context : reader.leaves()) {
SortedNumericDocValues values = context.reader().getSortedNumericDocValues(field);
if (values == null) {
// this field has no doc values for this segment
continue;
}
NumericDocValues singleValues = DocValues.unwrapSingleton(values);
if (singleValues != null) {
countAllOneSegment(singleValues);
} else {
int doc;
while ((doc = values.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
int limit = values.docValueCount();
totCount += limit;
for (int i = 0; i < limit; i++) {
increment(values.nextValue());
}
}
}
}
}
private void increment(long value) {
/*
if (value >= 0 && value < counts.length) {
counts[(int) value]++;
} else {
hashCounts.add(value, 1);
}
*/
hashCounts.add(value, 1);
}
@Override
public FacetResult getTopChildren(int topN, String dim, String... path) {
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");
}
return getTopChildrenSortByCount(topN);
}
/** Reusable hash entry to hold long facet value and int count. */
private static class Entry {
int count;
long value;
}
/** Returns the specified top number of facets, sorted by count. */
public FacetResult getTopChildrenSortByCount(int topN) {
PriorityQueue<Entry> pq = new PriorityQueue<Entry>(Math.min(topN, counts.length + hashCounts.size)) {
@Override
protected boolean lessThan(Entry a, Entry b) {
// sort by count descending, breaking ties by value ascending:
return a.count < b.count || (a.count == b.count && a.value > b.value);
}
};
int childCount = 0;
Entry e = null;
for (int i = 0; i < counts.length; i++) {
if (counts[i] != 0) {
childCount++;
if (e == null) {
e = new Entry();
}
e.value = i;
e.count = counts[i];
e = pq.insertWithOverflow(e);
}
}
if (hashCounts.size != 0) {
childCount += hashCounts.size;
for (int i = 0; i < hashCounts.values.length; i++) {
int count = hashCounts.counts[i];
if (count != 0) {
if (e == null) {
e = new Entry();
}
e.value = hashCounts.values[i];
e.count = count;
e = pq.insertWithOverflow(e);
}
}
}
LabelAndValue[] results = new LabelAndValue[pq.size()];
while (pq.size() != 0) {
Entry entry = pq.pop();
results[pq.size()] = new LabelAndValue(Long.toString(entry.value), entry.count);
}
return new FacetResult(field, new String[0], totCount, results, childCount);
}
/** Returns all unique values seen, sorted by value. */
public FacetResult getAllChildrenSortByValue() {
List<LabelAndValue> labelValues = new ArrayList<>();
// compact & sort hash table's arrays by value
int upto = 0;
for (int i = 0; i < hashCounts.values.length; i++) {
if (hashCounts.counts[i] != 0) {
hashCounts.counts[upto] = hashCounts.counts[i];
hashCounts.values[upto] = hashCounts.values[i];
upto++;
}
}
// zero fill all remaining counts so if we are called again we don't mistake these as real values
Arrays.fill(hashCounts.counts, upto, hashCounts.counts.length, 0);
assert upto == hashCounts.size : "upto=" + upto + " hashCounts.size=" + hashCounts.size;
new InPlaceMergeSorter() {
@Override
public int compare(int i, int j) {
return Long.compare(hashCounts.values[i], hashCounts.values[j]);
}
@Override
public void swap(int i, int j) {
int x = hashCounts.counts[i];
hashCounts.counts[i] = hashCounts.counts[j];
hashCounts.counts[j] = x;
long y = hashCounts.values[j];
hashCounts.values[j] = hashCounts.values[i];
hashCounts.values[i] = y;
}
}.sort(0, upto);
boolean countsAdded = false;
for (int i = 0; i < upto; i++) {
/*
if (countsAdded == false && hashCounts.values[i] >= counts.length) {
countsAdded = true;
appendCounts(labelValues);
}
*/
labelValues.add(new LabelAndValue(Long.toString(hashCounts.values[i]),
hashCounts.counts[i]));
}
/*
if (countsAdded == false) {
appendCounts(labelValues);
}
*/
return new FacetResult(field, new String[0], totCount, labelValues.toArray(new LabelAndValue[0]), labelValues.size());
}
private void appendCounts(List<LabelAndValue> labelValues) {
for (int i = 0; i < counts.length; i++) {
if (counts[i] != 0) {
labelValues.add(new LabelAndValue(Long.toString(i), counts[i]));
}
}
}
@Override
public Number getSpecificValue(String dim, String... path) throws IOException {
// TODO: should we impl this?
throw new UnsupportedOperationException();
}
@Override
public List<FacetResult> getAllDims(int topN) throws IOException {
return Collections.singletonList(getTopChildren(topN, field));
}
@Override
public String toString() {
StringBuilder b = new StringBuilder();
b.append("LongValueFacetCounts totCount=");
b.append(totCount);
b.append(":\n");
for (int i = 0; i < counts.length; i++) {
if (counts[i] != 0) {
b.append(" ");
b.append(i);
b.append(" -> count=");
b.append(counts[i]);
b.append('\n');
}
}
if (hashCounts.size != 0) {
for (int i = 0; i < hashCounts.values.length; i++) {
if (hashCounts.counts[i] != 0) {
b.append(" ");
b.append(hashCounts.values[i]);
b.append(" -> count=");
b.append(hashCounts.counts[i]);
b.append('\n');
}
}
}
return b.toString();
}
/** Native typed hash table. */
static class HashTable {
static final float LOAD_FACTOR = 0.7f;
long[] values; // values identifying a value
int[] counts;
int mask;
int size;
int threshold;
HashTable() {
int capacity = 64; // must be a power of 2
values = new long[capacity];
counts = new int[capacity];
mask = capacity - 1;
size = 0;
threshold = (int) (capacity * LOAD_FACTOR);
}
private int hash(long v) {
int h = (int) (v ^ (v >>> 32));
h = (31 * h) & mask; // * 31 to try to use the whole table, even if values are dense
return h;
}
void add(long value, int inc) {
if (size >= threshold) {
rehash();
}
final int h = hash(value);
for (int slot = h;; slot = (slot + 1) & mask) {
if (counts[slot] == 0) {
values[slot] = value;
++size;
} else if (values[slot] != value) {
continue;
}
counts[slot] += inc;
break;
}
}
private void rehash() {
final long[] oldValues = values;
final int[] oldCounts = counts;
final int newCapacity = values.length * 2;
values = new long[newCapacity];
counts = new int[newCapacity];
mask = newCapacity - 1;
threshold = (int) (LOAD_FACTOR * newCapacity);
size = 0;
for (int i = 0; i < oldValues.length; ++i) {
if (oldCounts[i] > 0) {
add(oldValues[i], oldCounts[i]);
}
}
}
}
}

View File

@ -51,16 +51,18 @@ public class LongRangeFacetCounts extends RangeFacetCounts {
this(field, LongValuesSource.fromLongField(field), hits, ranges);
}
/** Create {@code RangeFacetCounts}, using the provided
/** Create {@code LongRangeFacetCounts}, using the provided
* {@link ValueSource}. */
public LongRangeFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits, LongRange... ranges) throws IOException {
this(field, valueSource, hits, null, ranges);
}
/** Create {@code RangeFacetCounts}, using the provided
/** Create {@code LongRangeFacetCounts}, using the provided
* {@link ValueSource}, and using the provided Filter as
* a fastmatch: only documents passing the filter are
* checked for the matching ranges. The filter must be
* 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}). */
public LongRangeFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits, Query fastMatchQuery, LongRange... ranges) throws IOException {
super(field, ranges, fastMatchQuery);
@ -121,7 +123,7 @@ public class LongRangeFacetCounts extends RangeFacetCounts {
missingCount += x;
//System.out.println("totCount " + totCount + " missingCount " + counter.missingCount);
//System.out.println("totCount " + totCount + " x " + x + " missingCount " + missingCount);
totCount -= missingCount;
}
}

View File

@ -78,7 +78,7 @@ abstract class RangeFacetCounts extends Facets {
@Override
public List<FacetResult> getAllDims(int topN) throws IOException {
return Collections.singletonList(getTopChildren(topN, null));
return Collections.singletonList(getTopChildren(topN, field));
}
@Override

View File

@ -0,0 +1,543 @@
/*
* 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.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.IntPoint;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.facet.FacetResult;
import org.apache.lucene.facet.Facets;
import org.apache.lucene.facet.FacetsCollector;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.LongValuesSource;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
/** Tests long value facets. */
public class TestLongValueFacetCounts extends LuceneTestCase {
public void testBasic() throws Exception {
Directory d = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), d);
for (long l = 0; l < 100; l++) {
Document doc = new Document();
doc.add(new NumericDocValuesField("field", l % 5));
w.addDocument(doc);
}
// Also add Long.MAX_VALUE
Document doc = new Document();
doc.add(new NumericDocValuesField("field", 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);
LongValueFacetCounts facets = new LongValueFacetCounts("field", fc, false);
FacetResult result = facets.getAllChildrenSortByValue();
assertEquals("dim=field path=[] value=101 childCount=6\n 0 (20)\n 1 (20)\n 2 (20)\n 3 (20)\n " +
"4 (20)\n 9223372036854775807 (1)\n",
result.toString());
r.close();
d.close();
}
public void testOnlyBigLongs() throws Exception {
Directory d = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), d);
for (long l = 0; l < 3; l++) {
Document doc = new Document();
doc.add(new NumericDocValuesField("field", Long.MAX_VALUE - l));
w.addDocument(doc);
}
IndexReader r = w.getReader();
w.close();
FacetsCollector fc = new FacetsCollector();
IndexSearcher s = newSearcher(r);
s.search(new MatchAllDocsQuery(), fc);
LongValueFacetCounts facets = new LongValueFacetCounts("field", fc, false);
FacetResult result = facets.getAllChildrenSortByValue();
assertEquals("dim=field path=[] value=3 childCount=3\n 9223372036854775805 (1)\n " +
"9223372036854775806 (1)\n 9223372036854775807 (1)\n",
result.toString());
r.close();
d.close();
}
public void testGetAllDims() throws Exception {
Directory d = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), d);
for (long l = 0; l < 100; l++) {
Document doc = new Document();
doc.add(new NumericDocValuesField("field", l % 5));
w.addDocument(doc);
}
// Also add Long.MAX_VALUE
Document doc = new Document();
doc.add(new NumericDocValuesField("field", 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 LongValueFacetCounts("field", fc, false);
List<FacetResult> result = facets.getAllDims(10);
assertEquals(1, result.size());
assertEquals("dim=field path=[] value=101 childCount=6\n 0 (20)\n 1 (20)\n 2 (20)\n " +
"3 (20)\n 4 (20)\n 9223372036854775807 (1)\n",
result.get(0).toString());
r.close();
d.close();
}
public void testRandom() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
int valueCount = atLeast(1000);
double missingChance = random().nextDouble();
long maxValue;
if (random().nextBoolean()) {
maxValue = random().nextLong() & Long.MAX_VALUE;
} else {
maxValue = random().nextInt(1000);
}
if (VERBOSE) {
System.out.println("TEST: valueCount=" + valueCount + " valueRange=-" + maxValue +
"-" + maxValue + " missingChance=" + missingChance);
}
Long[] values = new Long[valueCount];
int missingCount = 0;
for (int i = 0; i < valueCount; i++) {
Document doc = new Document();
doc.add(new IntPoint("id", i));
if (random().nextDouble() > missingChance) {
long value = TestUtil.nextLong(random(), -maxValue, maxValue);
doc.add(new NumericDocValuesField("field", value));
values[i] = value;
} else {
missingCount++;
}
w.addDocument(doc);
}
IndexReader r = w.getReader();
w.close();
IndexSearcher s = newSearcher(r);
int iters = atLeast(100);
for (int iter = 0; iter < iters; iter++) {
FacetsCollector fc = new FacetsCollector();
if (VERBOSE) {
System.out.println("\nTEST: iter=" + iter);
System.out.println(" test all docs");
}
// all docs
Map<Long, Integer> expected = new HashMap<>();
int expectedChildCount = 0;
for (int i = 0; i < valueCount; i++) {
if (values[i] != null) {
Integer curCount = expected.get(values[i]);
if (curCount == null) {
curCount = 0;
expectedChildCount++;
}
expected.put(values[i], curCount + 1);
}
}
List<Map.Entry<Long, Integer>> expectedCounts = new ArrayList<>(expected.entrySet());
// sort by value
Collections.sort(expectedCounts,
(a, b) -> (Long.compare(a.getKey(), b.getKey())));
LongValueFacetCounts facetCounts;
if (random().nextBoolean()) {
s.search(new MatchAllDocsQuery(), fc);
if (random().nextBoolean()) {
if (VERBOSE) {
System.out.println(" use value source");
}
facetCounts = new LongValueFacetCounts("field", LongValuesSource.fromLongField("field"), fc);
} else {
if (VERBOSE) {
System.out.println(" use doc values");
}
facetCounts = new LongValueFacetCounts("field", fc, false);
}
} else {
// optimized count all:
if (random().nextBoolean()) {
if (VERBOSE) {
System.out.println(" count all value source");
}
facetCounts = new LongValueFacetCounts("field", LongValuesSource.fromLongField("field"), r);
} else {
if (VERBOSE) {
System.out.println(" count all doc values");
}
facetCounts = new LongValueFacetCounts("field", r, false);
}
}
FacetResult actual = facetCounts.getAllChildrenSortByValue();
assertSame("all docs, sort facets by value", expectedCounts, expectedChildCount,
valueCount - missingCount, actual, Integer.MAX_VALUE);
// sort by count
Collections.sort(expectedCounts,
(a, b) -> {
int cmp = -Integer.compare(a.getValue(), b.getValue());
if (cmp == 0) {
// tie break by value
cmp = Long.compare(a.getKey(), b.getKey());
}
return cmp;
});
int topN;
if (random().nextBoolean()) {
topN = valueCount;
} else {
topN = random().nextInt(valueCount);
}
if (VERBOSE) {
System.out.println(" topN=" + topN);
}
actual = facetCounts.getTopChildrenSortByCount(topN);
assertSame("all docs, sort facets by count", expectedCounts, expectedChildCount, valueCount - missingCount, actual, topN);
// subset of docs
int minId = random().nextInt(valueCount);
int maxId = random().nextInt(valueCount);
if (minId > maxId) {
int tmp = minId;
minId = maxId;
maxId = tmp;
}
if (VERBOSE) {
System.out.println(" test id range " + minId + "-" + maxId);
}
fc = new FacetsCollector();
s.search(IntPoint.newRangeQuery("id", minId, maxId), fc);
if (random().nextBoolean()) {
if (VERBOSE) {
System.out.println(" use doc values");
}
facetCounts = new LongValueFacetCounts("field", fc, false);
} else {
if (VERBOSE) {
System.out.println(" use value source");
}
facetCounts = new LongValueFacetCounts("field", LongValuesSource.fromLongField("field"), fc);
}
expected = new HashMap<>();
expectedChildCount = 0;
int totCount = 0;
for (int i = minId; i <= maxId; i++) {
if (values[i] != null) {
totCount++;
Integer curCount = expected.get(values[i]);
if (curCount == null) {
expectedChildCount++;
curCount = 0;
}
expected.put(values[i], curCount + 1);
}
}
expectedCounts = new ArrayList<>(expected.entrySet());
// sort by value
Collections.sort(expectedCounts,
(a, b) -> (Long.compare(a.getKey(), b.getKey())));
actual = facetCounts.getAllChildrenSortByValue();
assertSame("id " + minId + "-" + maxId + ", sort facets by value", expectedCounts,
expectedChildCount, totCount, actual, Integer.MAX_VALUE);
// sort by count
Collections.sort(expectedCounts,
(a, b) -> {
int cmp = -Integer.compare(a.getValue(), b.getValue());
if (cmp == 0) {
// tie break by value
cmp = Long.compare(a.getKey(), b.getKey());
}
return cmp;
});
if (random().nextBoolean()) {
topN = valueCount;
} else {
topN = random().nextInt(valueCount);
}
actual = facetCounts.getTopChildrenSortByCount(topN);
assertSame("id " + minId + "-" + maxId + ", sort facets by count", expectedCounts, expectedChildCount, totCount, actual, topN);
}
r.close();
dir.close();
}
public void testRandomMultiValued() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
int valueCount = atLeast(1000);
double missingChance = random().nextDouble();
// sometimes exercise codec optimizations when a claimed multi valued field is in fact single valued:
boolean allSingleValued = rarely();
long maxValue;
if (random().nextBoolean()) {
maxValue = random().nextLong() & Long.MAX_VALUE;
} else {
maxValue = random().nextInt(1000);
}
if (VERBOSE) {
System.out.println("TEST: valueCount=" + valueCount + " valueRange=-" + maxValue +
"-" + maxValue + " missingChance=" + missingChance + " allSingleValued=" + allSingleValued);
}
long[][] values = new long[valueCount][];
int missingCount = 0;
for (int i = 0; i < valueCount; i++) {
Document doc = new Document();
doc.add(new IntPoint("id", i));
if (random().nextDouble() > missingChance) {
if (allSingleValued) {
values[i] = new long[1];
} else {
values[i] = new long[TestUtil.nextInt(random(), 1, 5)];
}
for (int j = 0; j < values[i].length; j++) {
long value = TestUtil.nextLong(random(), -maxValue, maxValue);
values[i][j] = value;
doc.add(new SortedNumericDocValuesField("field", value));
}
if (VERBOSE) {
System.out.println(" doc=" + i + " values=" + Arrays.toString(values[i]));
}
} else {
missingCount++;
if (VERBOSE) {
System.out.println(" doc=" + i + " missing values");
}
}
w.addDocument(doc);
}
IndexReader r = w.getReader();
w.close();
IndexSearcher s = newSearcher(r);
int iters = atLeast(100);
for (int iter = 0; iter < iters; iter++) {
FacetsCollector fc = new FacetsCollector();
if (VERBOSE) {
System.out.println("\nTEST: iter=" + iter);
System.out.println(" test all docs");
}
// all docs
Map<Long, Integer> expected = new HashMap<>();
int expectedChildCount = 0;
int expectedTotalCount = 0;
for (int i = 0; i < valueCount; i++) {
if (values[i] != null) {
for (long value : values[i]) {
Integer curCount = expected.get(value);
if (curCount == null) {
curCount = 0;
expectedChildCount++;
}
expected.put(value, curCount + 1);
expectedTotalCount++;
}
}
}
List<Map.Entry<Long, Integer>> expectedCounts = new ArrayList<>(expected.entrySet());
// sort by value
Collections.sort(expectedCounts,
(a, b) -> (Long.compare(a.getKey(), b.getKey())));
LongValueFacetCounts facetCounts;
if (random().nextBoolean()) {
s.search(new MatchAllDocsQuery(), fc);
if (VERBOSE) {
System.out.println(" use doc values");
}
facetCounts = new LongValueFacetCounts("field", fc, true);
} else {
// optimized count all:
if (VERBOSE) {
System.out.println(" count all doc values");
}
facetCounts = new LongValueFacetCounts("field", r, true);
}
FacetResult actual = facetCounts.getAllChildrenSortByValue();
assertSame("all docs, sort facets by value", expectedCounts, expectedChildCount,
expectedTotalCount, actual, Integer.MAX_VALUE);
// sort by count
Collections.sort(expectedCounts,
(a, b) -> {
int cmp = -Integer.compare(a.getValue(), b.getValue());
if (cmp == 0) {
// tie break by value
cmp = Long.compare(a.getKey(), b.getKey());
}
return cmp;
});
int topN;
if (random().nextBoolean()) {
topN = valueCount;
} else {
topN = random().nextInt(valueCount);
}
if (VERBOSE) {
System.out.println(" topN=" + topN);
}
actual = facetCounts.getTopChildrenSortByCount(topN);
assertSame("all docs, sort facets by count", expectedCounts, expectedChildCount, expectedTotalCount, actual, topN);
// subset of docs
int minId = random().nextInt(valueCount);
int maxId = random().nextInt(valueCount);
if (minId > maxId) {
int tmp = minId;
minId = maxId;
maxId = tmp;
}
if (VERBOSE) {
System.out.println(" test id range " + minId + "-" + maxId);
}
fc = new FacetsCollector();
s.search(IntPoint.newRangeQuery("id", minId, maxId), fc);
// cannot use value source here because we are multi valued
facetCounts = new LongValueFacetCounts("field", fc, true);
expected = new HashMap<>();
expectedChildCount = 0;
int totCount = 0;
for (int i = minId; i <= maxId; i++) {
if (values[i] != null) {
for (long value : values[i]) {
totCount++;
Integer curCount = expected.get(value);
if (curCount == null) {
expectedChildCount++;
curCount = 0;
}
expected.put(value, curCount + 1);
}
}
}
expectedCounts = new ArrayList<>(expected.entrySet());
// sort by value
Collections.sort(expectedCounts,
(a, b) -> (Long.compare(a.getKey(), b.getKey())));
actual = facetCounts.getAllChildrenSortByValue();
assertSame("id " + minId + "-" + maxId + ", sort facets by value", expectedCounts,
expectedChildCount, totCount, actual, Integer.MAX_VALUE);
// sort by count
Collections.sort(expectedCounts,
(a, b) -> {
int cmp = -Integer.compare(a.getValue(), b.getValue());
if (cmp == 0) {
// tie break by value
cmp = Long.compare(a.getKey(), b.getKey());
}
return cmp;
});
if (random().nextBoolean()) {
topN = valueCount;
} else {
topN = random().nextInt(valueCount);
}
actual = facetCounts.getTopChildrenSortByCount(topN);
assertSame("id " + minId + "-" + maxId + ", sort facets by count", expectedCounts, expectedChildCount, totCount, actual, topN);
}
r.close();
dir.close();
}
private static void assertSame(String desc, List<Map.Entry<Long, Integer>> expectedCounts,
int expectedChildCount, int expectedTotalCount, FacetResult actual, int topN) {
int expectedTopN = Math.min(topN, expectedCounts.size());
if (VERBOSE) {
System.out.println(" expected topN=" + expectedTopN);
for (int i = 0; i < expectedTopN; i++) {
System.out.println(" " + i + ": value=" + expectedCounts.get(i).getKey() + " count=" + expectedCounts.get(i).getValue());
}
System.out.println(" actual topN=" + actual.labelValues.length);
for (int i = 0; i < actual.labelValues.length; i++) {
System.out.println(" " + i + ": value=" + actual.labelValues[i].label + " count=" + actual.labelValues[i].value);
}
}
assertEquals(desc + ": topN", expectedTopN, actual.labelValues.length);
assertEquals(desc + ": childCount", expectedChildCount, actual.childCount);
assertEquals(desc + ": totCount", expectedTotalCount, actual.value.intValue());
assertTrue(actual.labelValues.length <= topN);
for (int i = 0; i < expectedTopN; i++) {
assertEquals(desc + ": label[" + i + "]", Long.toString(expectedCounts.get(i).getKey()), actual.labelValues[i].label);
assertEquals(desc + ": counts[" + i + "]", expectedCounts.get(i).getValue().intValue(), actual.labelValues[i].value.intValue());
}
}
}

View File

@ -18,6 +18,7 @@ package org.apache.lucene.facet.range;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
@ -28,8 +29,8 @@ import org.apache.lucene.document.DoublePoint;
import org.apache.lucene.document.LongPoint;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.facet.DrillDownQuery;
import org.apache.lucene.facet.DrillSideways;
import org.apache.lucene.facet.DrillSideways.DrillSidewaysResult;
import org.apache.lucene.facet.DrillSideways;
import org.apache.lucene.facet.FacetField;
import org.apache.lucene.facet.FacetResult;
import org.apache.lucene.facet.FacetTestCase;
@ -99,6 +100,44 @@ public class TestRangeFacetCounts extends FacetTestCase {
d.close();
}
public void testLongGetAllDims() 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);
}
// Also add Long.MAX_VALUE
field.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));
List<FacetResult> result = facets.getAllDims(10);
assertEquals(1, result.size());
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.get(0).toString());
r.close();
d.close();
}
public void testUselessRange() {
expectThrows(IllegalArgumentException.class, () -> {
new LongRange("useless", 7, true, 6, true);