mirror of https://github.com/apache/lucene.git
SOLR-11173: implement Points support in TermsComponent via PointMerger
This commit is contained in:
parent
4a9d5630bc
commit
a4374e840d
|
@ -884,6 +884,8 @@ public class AssertingLeafReader extends FilterLeafReader {
|
|||
assertStats(maxDoc);
|
||||
}
|
||||
|
||||
public PointValues getWrapped() { return in; }
|
||||
|
||||
private void assertStats(int maxDoc) {
|
||||
assert in.size() > 0;
|
||||
assert in.getDocCount() > 0;
|
||||
|
|
|
@ -373,6 +373,8 @@ New Features
|
|||
* SOLR-10939: Add support for PointsFields to {!join} query. Joined fields should
|
||||
also have docValues enabled. (yonik)
|
||||
|
||||
* SOLR-11173 TermsComponent support for Points fields. (yonik)
|
||||
|
||||
Bug Fixes
|
||||
----------------------
|
||||
* SOLR-9262: Connection and read timeouts are being ignored by UpdateShardHandler after SOLR-4509.
|
||||
|
|
|
@ -30,10 +30,12 @@ import org.apache.lucene.index.Term;
|
|||
import org.apache.lucene.index.TermContext;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.CharsRefBuilder;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.mutable.MutableValue;
|
||||
import org.apache.solr.client.solrj.response.TermsResponse;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||
|
@ -45,7 +47,9 @@ import org.apache.solr.common.util.SimpleOrderedMap;
|
|||
import org.apache.solr.common.util.StrUtils;
|
||||
import org.apache.solr.request.SimpleFacets.CountPair;
|
||||
import org.apache.solr.schema.FieldType;
|
||||
import org.apache.solr.schema.SchemaField;
|
||||
import org.apache.solr.schema.StrField;
|
||||
import org.apache.solr.search.PointMerger;
|
||||
import org.apache.solr.search.SolrIndexSearcher;
|
||||
import org.apache.solr.util.BoundedTreeSet;
|
||||
|
||||
|
@ -108,16 +112,6 @@ public class TermsComponent extends SearchComponent {
|
|||
rb.rsp.add("terms", termsResult);
|
||||
|
||||
if (fields == null || fields.length==0) return;
|
||||
|
||||
for (String field : fields) {
|
||||
FieldType fieldType = rb.req.getSchema().getFieldTypeNoEx(field);
|
||||
if (null != fieldType) {
|
||||
if (fieldType.isPointField()) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
"The terms component does not support Points-based field " + field);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
boolean termStats = params.getBool(TermsParams.TERMS_STATS, false);
|
||||
|
||||
|
@ -134,10 +128,8 @@ public class TermsComponent extends SearchComponent {
|
|||
return;
|
||||
}
|
||||
|
||||
int limit = params.getInt(TermsParams.TERMS_LIMIT, 10);
|
||||
if (limit < 0) {
|
||||
limit = Integer.MAX_VALUE;
|
||||
}
|
||||
int _limit = params.getInt(TermsParams.TERMS_LIMIT, 10);
|
||||
final int limit = _limit < 0 ? Integer.MAX_VALUE : _limit;
|
||||
|
||||
String lowerStr = params.get(TermsParams.TERMS_LOWER);
|
||||
String upperStr = params.get(TermsParams.TERMS_UPPER);
|
||||
|
@ -146,10 +138,9 @@ public class TermsComponent extends SearchComponent {
|
|||
boolean sort = !TermsParams.TERMS_SORT_INDEX.equals(
|
||||
params.get(TermsParams.TERMS_SORT, TermsParams.TERMS_SORT_COUNT));
|
||||
int freqmin = params.getInt(TermsParams.TERMS_MINCOUNT, 1);
|
||||
int freqmax = params.getInt(TermsParams.TERMS_MAXCOUNT, UNLIMITED_MAX_COUNT);
|
||||
if (freqmax<0) {
|
||||
freqmax = Integer.MAX_VALUE;
|
||||
}
|
||||
int _freqmax = params.getInt(TermsParams.TERMS_MAXCOUNT, UNLIMITED_MAX_COUNT);
|
||||
final int freqmax = _freqmax < 0 ? Integer.MAX_VALUE : _freqmax;
|
||||
|
||||
String prefix = params.get(TermsParams.TERMS_PREFIX_STR);
|
||||
String regexp = params.get(TermsParams.TERMS_REGEXP_STR);
|
||||
Pattern pattern = regexp != null ? Pattern.compile(regexp, resolveRegexpFlags(params)) : null;
|
||||
|
@ -161,13 +152,76 @@ public class TermsComponent extends SearchComponent {
|
|||
|
||||
for (String field : fields) {
|
||||
NamedList<Integer> fieldTerms = new NamedList<>();
|
||||
termsResult.add(field, fieldTerms);
|
||||
|
||||
Terms terms = indexReader.terms(field);
|
||||
if (terms == null) {
|
||||
// field does not exist
|
||||
// field does not exist in terms index. Check points.
|
||||
SchemaField sf = rb.req.getSchema().getFieldOrNull(field);
|
||||
if (sf != null && sf.getType().isPointField()) {
|
||||
if (lowerStr!=null || upperStr!=null || prefix!=null || regexp!=null) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
String.format(Locale.ROOT, "The terms component does not support Points-based fields with sorting or with parameters %s,%s,%s,%s ", TermsParams.TERMS_LOWER, TermsParams.TERMS_UPPER, TermsParams.TERMS_PREFIX_STR, TermsParams.TERMS_REGEXP_STR));
|
||||
}
|
||||
|
||||
if (sort) {
|
||||
PointMerger.ValueIterator valueIterator = new PointMerger.ValueIterator(sf, rb.req.getSearcher().getRawReader().leaves());
|
||||
MutableValue mv = valueIterator.getMutableValue();
|
||||
BoundedTreeSet<CountPair<MutableValue, Integer>> queue = (sort ? new BoundedTreeSet<>(limit) : null);
|
||||
|
||||
for (; ; ) {
|
||||
long count = valueIterator.getNextCount();
|
||||
if (count < 0) break;
|
||||
if (count < freqmin || count > freqmax) continue;
|
||||
if (queue.size() < limit || queue.last().val < count || (queue.last().val == count && queue.last().key.compareTo(mv) < 0)) {
|
||||
queue.add(new CountPair<>(mv.duplicate(), (int) count));
|
||||
}
|
||||
}
|
||||
|
||||
for (CountPair<MutableValue, Integer> item : queue) {
|
||||
fieldTerms.add(item.key.toString(), item.val);
|
||||
}
|
||||
termsResult.add(field, fieldTerms);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (!sort) {
|
||||
/***
|
||||
// streaming solution that is deferred until writing the response
|
||||
// TODO: we can't use the streaming solution until XML writer supports PushWriter!
|
||||
termsResult.add(field, (MapWriter) ew -> {
|
||||
PointMerger.ValueIterator valueIterator = new PointMerger.ValueIterator(sf, rb.req.getSearcher().getRawReader().leaves());
|
||||
MutableValue mv = valueIterator.getMutableValue();
|
||||
int num = 0;
|
||||
for(;;) {
|
||||
long count = valueIterator.getNextCount();
|
||||
if (count < 0) break;
|
||||
if (count < freqmin || count > freqmax) continue;
|
||||
if (++num > limit) break;
|
||||
ew.put(mv.toString(), (int)count); // match the numeric type of terms
|
||||
}
|
||||
});
|
||||
***/
|
||||
|
||||
PointMerger.ValueIterator valueIterator = new PointMerger.ValueIterator(sf, rb.req.getSearcher().getRawReader().leaves());
|
||||
MutableValue mv = valueIterator.getMutableValue();
|
||||
int num = 0;
|
||||
for(;;) {
|
||||
long count = valueIterator.getNextCount();
|
||||
if (count < 0) break;
|
||||
if (count < freqmin || count > freqmax) continue;
|
||||
if (++num > limit) break;
|
||||
fieldTerms.add(mv.toString(), (int)count); // match the numeric type of terms
|
||||
}
|
||||
|
||||
termsResult.add(field, fieldTerms);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
termsResult.add(field, fieldTerms); // add empty
|
||||
continue;
|
||||
}
|
||||
termsResult.add(field, fieldTerms);
|
||||
|
||||
FieldType ft = raw ? null : rb.req.getSchema().getFieldTypeNoEx(field);
|
||||
if (ft==null) ft = new StrField();
|
||||
|
@ -545,7 +599,19 @@ public class TermsComponent extends SearchComponent {
|
|||
|
||||
IndexReaderContext topReaderContext = indexSearcher.getTopReaderContext();
|
||||
for (String field : fields) {
|
||||
FieldType fieldType = indexSearcher.getSchema().getField(field).getType();
|
||||
SchemaField sf = indexSearcher.getSchema().getField(field);
|
||||
FieldType fieldType = sf.getType();
|
||||
|
||||
if (fieldType.isPointField()) {
|
||||
NamedList<Object> termsMap = new SimpleOrderedMap<>();
|
||||
for (String term : splitTerms) {
|
||||
Query q = fieldType.getFieldQuery(null, sf, term);
|
||||
int count = indexSearcher.getDocSet(q).size();
|
||||
termsMap.add(term, count);
|
||||
}
|
||||
result.add(field, termsMap);
|
||||
continue;
|
||||
}
|
||||
|
||||
// Since splitTerms is already sorted, this array will also be sorted
|
||||
Term[] terms = new Term[splitTerms.length];
|
||||
|
|
|
@ -0,0 +1,454 @@
|
|||
/*
|
||||
* 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.solr.search;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.document.DoublePoint;
|
||||
import org.apache.lucene.document.FloatPoint;
|
||||
import org.apache.lucene.document.IntPoint;
|
||||
import org.apache.lucene.document.LongPoint;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
import org.apache.lucene.util.mutable.MutableValue;
|
||||
import org.apache.lucene.util.mutable.MutableValueDate;
|
||||
import org.apache.lucene.util.mutable.MutableValueDouble;
|
||||
import org.apache.lucene.util.mutable.MutableValueFloat;
|
||||
import org.apache.lucene.util.mutable.MutableValueInt;
|
||||
import org.apache.lucene.util.mutable.MutableValueLong;
|
||||
import org.apache.solr.schema.SchemaField;
|
||||
|
||||
/**
|
||||
* Merge multiple numeric point fields (segments) together.
|
||||
*
|
||||
* @lucene.internal
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class PointMerger {
|
||||
public static int TOTAL_BUFFER_SIZE = 1000000; // target number of elements to cache across all segments
|
||||
public static int MIN_SEG_BUFFER_SIZE = 100; // minimum buffer size on any segment (to limit unnecessary exception throws)
|
||||
|
||||
public static class ValueIterator {
|
||||
PQueue queue;
|
||||
MutableValue topVal;
|
||||
|
||||
public ValueIterator(SchemaField field, List<LeafReaderContext> readers) throws IOException {
|
||||
this(field, readers, TOTAL_BUFFER_SIZE, MIN_SEG_BUFFER_SIZE);
|
||||
}
|
||||
|
||||
public ValueIterator(SchemaField field, List<LeafReaderContext> readers, int totalBufferSize, int minSegBufferSize) throws IOException {
|
||||
assert field.getType().isPointField();
|
||||
queue = new PQueue(readers.size());
|
||||
long ndocs = readers.get(readers.size()-1).docBase + readers.get(readers.size()-1).reader().maxDoc();
|
||||
for (LeafReaderContext ctx : readers) {
|
||||
PointValues pv = ctx.reader().getPointValues(field.getName());
|
||||
if (pv == null) continue;
|
||||
BaseSeg seg = null;
|
||||
// int capacity = 2;
|
||||
int capacity = (int)((long)totalBufferSize * ctx.reader().maxDoc() / ndocs);
|
||||
capacity = Math.max(capacity, minSegBufferSize);
|
||||
|
||||
switch (field.getType().getNumberType()) {
|
||||
case INTEGER:
|
||||
seg = new IntSeg(pv, capacity);
|
||||
break;
|
||||
case LONG:
|
||||
seg = new LongSeg(pv, capacity);
|
||||
break;
|
||||
case FLOAT:
|
||||
seg = new FloatSeg(pv, capacity);
|
||||
break;
|
||||
case DOUBLE:
|
||||
seg = new DoubleSeg(pv, capacity);
|
||||
break;
|
||||
}
|
||||
int count = seg.setNextValue();
|
||||
if (count >= 0) {
|
||||
queue.add(seg);
|
||||
}
|
||||
}
|
||||
if (queue.size() > 0) topVal = queue.top().getMutableValue().duplicate();
|
||||
}
|
||||
|
||||
// gets the mutable value that is updated after every call to getNextCount().
|
||||
// getMutableValue only needs to be called a single time since the instance is reused for every call to getNextCount().
|
||||
public MutableValue getMutableValue() {
|
||||
return topVal;
|
||||
}
|
||||
|
||||
public long getNextCount() throws IOException {
|
||||
if (queue.size() == 0) return -1;
|
||||
|
||||
BaseSeg seg = queue.top();
|
||||
topVal.copy(seg.getMutableValue());
|
||||
long count = 0;
|
||||
|
||||
do {
|
||||
count += seg.getCurrentCount();
|
||||
int nextCount = seg.setNextValue();
|
||||
if (nextCount < 0) {
|
||||
queue.pop();
|
||||
if (queue.size() == 0) break;
|
||||
} else {
|
||||
queue.updateTop();
|
||||
}
|
||||
seg = queue.top();
|
||||
} while (seg.getMutableValue().equalsSameType(topVal));
|
||||
|
||||
return count;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
static class PQueue extends PriorityQueue<BaseSeg> {
|
||||
public PQueue(int maxSize) {
|
||||
super(maxSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean lessThan(BaseSeg a, BaseSeg b) {
|
||||
return BaseSeg.lessThan(a,b);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
abstract static class BaseSeg implements PointValues.IntersectVisitor {
|
||||
final PointValues points;
|
||||
final int[] count;
|
||||
int pos = -1; // index of the last valid entry
|
||||
int readPos = -1; // last position read from
|
||||
|
||||
MutableValue currentValue; // subclass constructor will fill this in
|
||||
int currentCount;
|
||||
|
||||
BaseSeg(PointValues points, int capacity) {
|
||||
this.points = points;
|
||||
this.count = new int[capacity];
|
||||
}
|
||||
|
||||
public static boolean lessThan(BaseSeg a, BaseSeg b) {
|
||||
return a.currentValue.compareTo(b.currentValue) < 0;
|
||||
}
|
||||
|
||||
public MutableValue getMutableValue() {
|
||||
return currentValue;
|
||||
}
|
||||
|
||||
// returns -1 count if there are no more values
|
||||
public int getCurrentCount() {
|
||||
return currentCount;
|
||||
}
|
||||
|
||||
// sets the next value and returns getCurrentCount()
|
||||
public int setNextValue() throws IOException {
|
||||
return 0;
|
||||
};
|
||||
|
||||
|
||||
void refill() throws IOException {
|
||||
assert readPos >= pos;
|
||||
readPos = -1;
|
||||
pos = -1;
|
||||
try {
|
||||
points.intersect(this);
|
||||
} catch (BreakException e) {
|
||||
// nothing to do
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
static class IntSeg extends BaseSeg {
|
||||
final int[] values;
|
||||
int last = Integer.MIN_VALUE;
|
||||
final MutableValueInt mval;
|
||||
|
||||
IntSeg(PointValues points, int capacity) {
|
||||
super(points, capacity);
|
||||
this.values = new int[capacity];
|
||||
this.currentValue = this.mval = new MutableValueInt();
|
||||
}
|
||||
|
||||
public int setNextValue() throws IOException {
|
||||
if (readPos >= pos) {
|
||||
if (last != Integer.MAX_VALUE) {
|
||||
++last;
|
||||
refill();
|
||||
}
|
||||
if (readPos >= pos) {
|
||||
last = Integer.MAX_VALUE;
|
||||
currentCount = -1;
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
|
||||
++readPos;
|
||||
mval.value = values[readPos];
|
||||
currentCount = count[readPos];
|
||||
return currentCount;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {
|
||||
// TODO: handle filter or deleted documents?
|
||||
int v = IntPoint.decodeDimension(packedValue, 0);
|
||||
if (v < last) return;
|
||||
|
||||
if (v == last && pos >= 0) {
|
||||
count[pos]++;
|
||||
} else {
|
||||
if (pos+1 < values.length) {
|
||||
last = v;
|
||||
++pos;
|
||||
values[pos] = v;
|
||||
count[pos] = 1;
|
||||
} else {
|
||||
// a new value we don't have room for
|
||||
throw breakException;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
int v = IntPoint.decodeDimension(maxPackedValue, 0);
|
||||
if (v >= last) {
|
||||
return PointValues.Relation.CELL_CROSSES_QUERY;
|
||||
} else {
|
||||
return PointValues.Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static class LongSeg extends BaseSeg {
|
||||
final long[] values;
|
||||
long last = Long.MIN_VALUE;
|
||||
MutableValueLong mval;
|
||||
|
||||
LongSeg(PointValues points, int capacity) {
|
||||
super(points, capacity);
|
||||
this.values = new long[capacity];
|
||||
this.currentValue = this.mval = new MutableValueLong();
|
||||
}
|
||||
|
||||
public int setNextValue() throws IOException {
|
||||
if (readPos >= pos) {
|
||||
if (last != Long.MAX_VALUE) {
|
||||
++last;
|
||||
refill();
|
||||
}
|
||||
if (readPos >= pos) {
|
||||
last = Long.MAX_VALUE;
|
||||
currentCount = -1;
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
|
||||
++readPos;
|
||||
mval.value = values[readPos];
|
||||
currentCount = count[readPos];
|
||||
return currentCount;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {
|
||||
// TODO: handle filter or deleted documents?
|
||||
long v = LongPoint.decodeDimension(packedValue, 0);
|
||||
if (v < last) return;
|
||||
|
||||
if (v == last && pos >= 0) {
|
||||
count[pos]++;
|
||||
} else {
|
||||
if (pos+1 < values.length) {
|
||||
last = v;
|
||||
++pos;
|
||||
values[pos] = v;
|
||||
count[pos] = 1;
|
||||
} else {
|
||||
// a new value we don't have room for
|
||||
throw breakException;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
long v = LongPoint.decodeDimension(maxPackedValue, 0);
|
||||
if (v >= last) {
|
||||
return PointValues.Relation.CELL_CROSSES_QUERY;
|
||||
} else {
|
||||
return PointValues.Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static class FloatSeg extends BaseSeg {
|
||||
final float[] values;
|
||||
float last = -Float.MAX_VALUE;
|
||||
final MutableValueFloat mval;
|
||||
|
||||
FloatSeg(PointValues points, int capacity) {
|
||||
super(points, capacity);
|
||||
this.values = new float[capacity];
|
||||
this.currentValue = this.mval = new MutableValueFloat();
|
||||
}
|
||||
|
||||
public int setNextValue() throws IOException {
|
||||
if (readPos >= pos) {
|
||||
if (last != Float.MAX_VALUE) {
|
||||
last = Math.nextUp(last);
|
||||
refill();
|
||||
}
|
||||
if (readPos >= pos) {
|
||||
last = Float.MAX_VALUE;
|
||||
currentCount = -1;
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
|
||||
++readPos;
|
||||
mval.value = values[readPos];
|
||||
currentCount = count[readPos];
|
||||
return currentCount;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {
|
||||
// TODO: handle filter or deleted documents?
|
||||
float v = FloatPoint.decodeDimension(packedValue, 0);
|
||||
if (v < last) return;
|
||||
|
||||
if (v == last && pos >= 0) {
|
||||
count[pos]++;
|
||||
} else {
|
||||
if (pos+1 < values.length) {
|
||||
last = v;
|
||||
++pos;
|
||||
values[pos] = v;
|
||||
count[pos] = 1;
|
||||
} else {
|
||||
// a new value we don't have room for
|
||||
throw breakException;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
float v = FloatPoint.decodeDimension(maxPackedValue, 0);
|
||||
if (v >= last) {
|
||||
return PointValues.Relation.CELL_CROSSES_QUERY;
|
||||
} else {
|
||||
return PointValues.Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static class DoubleSeg extends BaseSeg {
|
||||
final double[] values;
|
||||
double last = -Double.MAX_VALUE;
|
||||
final MutableValueDouble mval;
|
||||
|
||||
DoubleSeg(PointValues points, int capacity) {
|
||||
super(points, capacity);
|
||||
this.values = new double[capacity];
|
||||
this.currentValue = this.mval = new MutableValueDouble();
|
||||
}
|
||||
|
||||
public int setNextValue() throws IOException {
|
||||
if (readPos >= pos) {
|
||||
if (last != Double.MAX_VALUE) {
|
||||
last = Math.nextUp(last);
|
||||
refill();
|
||||
}
|
||||
if (readPos >= pos) {
|
||||
last = Double.MAX_VALUE;
|
||||
currentCount = -1;
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
|
||||
++readPos;
|
||||
mval.value = values[readPos];
|
||||
currentCount = count[readPos];
|
||||
return currentCount;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {
|
||||
// TODO: handle filter or deleted documents?
|
||||
double v = DoublePoint.decodeDimension(packedValue, 0);
|
||||
if (v < last) return;
|
||||
|
||||
if (v == last && pos >= 0) {
|
||||
count[pos]++;
|
||||
} else {
|
||||
if (pos+1 < values.length) {
|
||||
last = v;
|
||||
++pos;
|
||||
values[pos] = v;
|
||||
count[pos] = 1;
|
||||
} else {
|
||||
// a new value we don't have room for
|
||||
throw breakException;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
double v = DoublePoint.decodeDimension(maxPackedValue, 0);
|
||||
if (v >= last) {
|
||||
return PointValues.Relation.CELL_CROSSES_QUERY;
|
||||
} else {
|
||||
return PointValues.Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static class DateSeg extends LongSeg {
|
||||
DateSeg(PointValues points, int capacity) {
|
||||
super(points, capacity);
|
||||
this.currentValue = this.mval = new MutableValueDate();
|
||||
}
|
||||
}
|
||||
|
||||
static class BreakException extends RuntimeException {
|
||||
@Override
|
||||
public synchronized Throwable fillInStackTrace() {
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
static BreakException breakException = new BreakException();
|
||||
|
||||
}
|
|
@ -633,7 +633,11 @@
|
|||
<dynamicField name="*_b" type="boolean" indexed="true" stored="true"/>
|
||||
<dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
|
||||
|
||||
<dynamicField name="*_pi" type="pint" indexed="true" stored="true" docValues="false" multiValued="false"/>
|
||||
<dynamicField name="*_pi" type="pint" indexed="true" multiValued="false"/>
|
||||
<dynamicField name="*_pl" type="plong" indexed="true" multiValued="false"/>
|
||||
<dynamicField name="*_pf" type="pfloat" indexed="true" multiValued="false"/>
|
||||
<dynamicField name="*_pd" type="pdouble" indexed="true" multiValued="false"/>
|
||||
<dynamicField name="*_pdt" type="pdate" indexed="true" multiValued="false"/>
|
||||
|
||||
<!-- some trie-coded dynamic fields for faster range queries -->
|
||||
<dynamicField name="*_ti" type="tint" indexed="true" stored="true"/>
|
||||
|
|
|
@ -15,22 +15,24 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.solr.handler.component;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.lucene.util.mutable.MutableValueDouble;
|
||||
import org.apache.lucene.util.mutable.MutableValueFloat;
|
||||
import org.apache.lucene.util.mutable.MutableValueInt;
|
||||
import org.apache.lucene.util.mutable.MutableValueLong;
|
||||
import org.apache.solr.SolrTestCaseJ4;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||
import org.apache.solr.common.params.TermsParams;
|
||||
import org.apache.solr.request.SolrQueryRequest;
|
||||
import org.apache.solr.schema.SchemaField;
|
||||
import org.apache.solr.search.PointMerger;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
/**
|
||||
*
|
||||
*
|
||||
**/
|
||||
// TermsComponent not currently supported for PointFields
|
||||
@SolrTestCaseJ4.SuppressPointFields(bugUrl="https://issues.apache.org/jira/browse/SOLR-11173")
|
||||
public class TermsComponentTest extends SolrTestCaseJ4 {
|
||||
|
||||
@BeforeClass
|
||||
|
@ -283,10 +285,12 @@ public class TermsComponentTest extends SolrTestCaseJ4 {
|
|||
,"//int[@name='1'][.='2']"
|
||||
);
|
||||
|
||||
/* terms.raw only applies to indexed fields
|
||||
assertQ(req("indent","true", "qt","/terms", "terms","true",
|
||||
"terms.fl","foo_i", "terms.raw","true")
|
||||
,"not(//int[@name='1'][.='2'])"
|
||||
);
|
||||
*/
|
||||
|
||||
// check something at the end of the index
|
||||
assertQ(req("indent","true", "qt","/terms", "terms","true",
|
||||
|
@ -381,28 +385,121 @@ public class TermsComponentTest extends SolrTestCaseJ4 {
|
|||
|
||||
@Test
|
||||
public void testPointField() throws Exception {
|
||||
assertU(adoc("id", "10000", "foo_pi", "1"));
|
||||
assertU(commit());
|
||||
|
||||
try {
|
||||
final SolrQueryRequest req = req(
|
||||
"qt", "/terms",
|
||||
"terms", "true",
|
||||
"terms.fl", "foo_pi");
|
||||
Exception e = expectThrows(SolrException.class, () -> h.query(req));
|
||||
assertEquals(SolrException.ErrorCode.BAD_REQUEST.code, ((SolrException) e).code());
|
||||
assertTrue(e.getMessage().contains("The terms component does not support Points-based field foo_pi"));
|
||||
int nvals = 10000; int maxval = 1000000;
|
||||
// int nvals = 5; int maxval = 2;
|
||||
final int vals[] = new int[nvals];
|
||||
for (int i=0; i<nvals; i++) {
|
||||
vals[i] = random().nextInt(maxval);
|
||||
String v = Integer.toString(vals[i]);
|
||||
assertU(adoc("id", Integer.toString(100000+i), "foo_pi",v, "foo_pl",v, "foo_pf",v, "foo_pd",v) );
|
||||
if (random().nextInt(1000) == 0) assertU(commit()); // make multiple segments
|
||||
}
|
||||
|
||||
assertU(commit());
|
||||
// assertU(optimize());
|
||||
|
||||
Arrays.sort(vals);
|
||||
|
||||
// find the first two values and account for dups
|
||||
int val1 = vals[0];
|
||||
int val2 = vals[1];
|
||||
for (int i=2; i<vals.length; i++) {
|
||||
if (val2 != val1) break;
|
||||
val2 = vals[i];
|
||||
}
|
||||
|
||||
SolrQueryRequest req = req(
|
||||
"qt", "/terms",
|
||||
"terms", "true",
|
||||
"terms.fl", "foo_pi");
|
||||
;
|
||||
try {
|
||||
SchemaField sf = req.getSchema().getField("foo_pi");
|
||||
|
||||
/**
|
||||
LeafReader r = req.getSearcher().getIndexReader().leaves().get(0).reader();
|
||||
PointValues pv = r.getPointValues("foo_pi");
|
||||
System.out.println("pv=" + pv);
|
||||
if (pv instanceof AssertingLeafReader.AssertingPointValues) {
|
||||
pv = ((AssertingLeafReader.AssertingPointValues) pv).getWrapped();
|
||||
}
|
||||
System.out.println("pv=" + pv);
|
||||
BKDReader bkdr = (BKDReader)pv;
|
||||
|
||||
for (int i=0; i<Math.min(10,nvals); i++) { System.out.println("INDEXED VAL=" + vals[i]); }
|
||||
**/
|
||||
|
||||
|
||||
//
|
||||
// iterate all values
|
||||
//
|
||||
int totBuff = random().nextInt(50)+1;
|
||||
int minSegBuff = random().nextInt(10)+1;
|
||||
PointMerger.ValueIterator iter = new PointMerger.ValueIterator(req.getSchema().getField("foo_pi"), req.getSearcher().getIndexReader().leaves(), totBuff, minSegBuff);
|
||||
MutableValueInt v = (MutableValueInt)iter.getMutableValue();
|
||||
int i=0;
|
||||
for (;;) {
|
||||
long count = iter.getNextCount();
|
||||
if (count < 0) break;
|
||||
assertEquals( vals[i], v.value );
|
||||
i += count;
|
||||
// if (i < 10) System.out.println("COUNT=" + count + " OBJ="+v.toObject());
|
||||
}
|
||||
assert(i==nvals);
|
||||
|
||||
totBuff = random().nextInt(50)+1;
|
||||
minSegBuff = random().nextInt(10)+1;
|
||||
iter = new PointMerger.ValueIterator(req.getSchema().getField("foo_pl"), req.getSearcher().getIndexReader().leaves());
|
||||
MutableValueLong lv = (MutableValueLong)iter.getMutableValue();
|
||||
i=0;
|
||||
for (;;) {
|
||||
long count = iter.getNextCount();
|
||||
if (count < 0) break;
|
||||
assertEquals( vals[i], lv.value );
|
||||
i += count;
|
||||
// if (i < 10) System.out.println("COUNT=" + count + " OBJ="+v.toObject());
|
||||
}
|
||||
assert(i==nvals);
|
||||
|
||||
totBuff = random().nextInt(50)+1;
|
||||
minSegBuff = random().nextInt(10)+1;
|
||||
iter = new PointMerger.ValueIterator(req.getSchema().getField("foo_pf"), req.getSearcher().getIndexReader().leaves());
|
||||
MutableValueFloat fv = (MutableValueFloat)iter.getMutableValue();
|
||||
i=0;
|
||||
for (;;) {
|
||||
long count = iter.getNextCount();
|
||||
if (count < 0) break;
|
||||
assertEquals( vals[i], fv.value, 0);
|
||||
i += count;
|
||||
// if (i < 10) System.out.println("COUNT=" + count + " OBJ="+v.toObject());
|
||||
}
|
||||
assert(i==nvals);
|
||||
|
||||
totBuff = random().nextInt(50)+1;
|
||||
minSegBuff = random().nextInt(10)+1;
|
||||
iter = new PointMerger.ValueIterator(req.getSchema().getField("foo_pd"), req.getSearcher().getIndexReader().leaves());
|
||||
MutableValueDouble dv = (MutableValueDouble)iter.getMutableValue();
|
||||
i=0;
|
||||
for (;;) {
|
||||
long count = iter.getNextCount();
|
||||
if (count < 0) break;
|
||||
assertEquals( vals[i], dv.value, 0);
|
||||
i += count;
|
||||
// if (i < 10) System.out.println("COUNT=" + count + " OBJ="+v.toObject());
|
||||
}
|
||||
assert(i==nvals);
|
||||
|
||||
assertQ(req("indent","true", "qt","/terms", "terms","true",
|
||||
"terms.fl","foo_pi", "terms.sort","index", "terms.limit","2")
|
||||
,"count(//lst[@name='foo_pi']/*)=2"
|
||||
,"//lst[@name='foo_pi']/int[1][@name='" +val1+ "']"
|
||||
,"//lst[@name='foo_pi']/int[2][@name='" +val2+ "']"
|
||||
);
|
||||
|
||||
|
||||
final SolrQueryRequest req2 = req(
|
||||
"qt", "/terms",
|
||||
"terms", "true",
|
||||
"terms.fl", "foo_pi",
|
||||
"terms.list", "1");
|
||||
e = expectThrows(SolrException.class, () -> h.query(req2));
|
||||
assertEquals(SolrException.ErrorCode.BAD_REQUEST.code, ((SolrException) e).code());
|
||||
assertTrue(e.getMessage().contains("The terms component does not support Points-based field foo_pi"));
|
||||
} finally {
|
||||
assertU(delI("10000"));
|
||||
req.close();
|
||||
assertU(delQ("foo_pi:[* TO *]"));
|
||||
assertU(commit());
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue