SOLR-9987: Implement support for multi-valued DocValues in PointFields

CC SOLR-8396
This commit is contained in:
Tomas Fernandez Lobbe 2017-02-13 09:47:49 -08:00
parent f1c5cd5784
commit 7dcf9de41f
38 changed files with 1275 additions and 436 deletions

View File

@ -139,6 +139,8 @@ New Features
* SOLR-9903: Stop interrupting the update executor on shutdown, it can cause graceful shutdowns to put replicas into Leader
Initiated Recovery among other undesirable things. (Mark Miller)
* SOLR-9987: Add support for MultiValued DocValues in PointFields using SortedNumericDocValues (Tomás Fernández Löbbe)
Bug Fixes
----------------------

View File

@ -688,7 +688,12 @@ public class RealTimeGetComponent extends SearchComponent
if (sf != null && sf.multiValued()) {
List<Object> vals = new ArrayList<>();
vals.add( f );
if (f.fieldType().docValuesType() == DocValuesType.SORTED_NUMERIC) {
// SORTED_NUMERICS store sortable bits version of the value, need to retrieve the original
vals.add(sf.getType().toObject(f));
} else {
vals.add( f );
}
out.setField( f.name(), vals );
}
else{

View File

@ -0,0 +1,106 @@
/*
* 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.handler.component;
import java.io.IOException;
import java.util.Map;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.NumericUtils;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.schema.NumberType;
public class SortedNumericStatsValues implements StatsValues {
private final NumericStatsValues nsv;
private final String fieldName;
private final NumberType numberType;
private SortedNumericDocValues sndv;
public SortedNumericStatsValues(NumericStatsValues nsv, StatsField field) {
this.nsv = nsv;
this.fieldName = field.getSchemaField().getName();
this.numberType = field.getSchemaField().getType().getNumberType();
}
@Override
public void accumulate(NamedList stv) {
nsv.accumulate(stv);
}
@Override
public void accumulate(int docId) throws IOException {
if (!sndv.advanceExact(docId)) {
missing();
} else {
for (int i = 0 ; i < sndv.docValueCount(); i++) {
nsv.accumulate(toCorrectType(sndv.nextValue()), 1);
}
}
}
private Number toCorrectType(long value) {
switch (numberType) {
case INTEGER:
case LONG:
return value;
case FLOAT:
return NumericUtils.sortableIntToFloat((int)value);
case DOUBLE:
return NumericUtils.sortableLongToDouble(value);
default:
throw new AssertionError("Unsupported number type");
}
}
@Override
public void accumulate(BytesRef value, int count) {
nsv.accumulate(value, count);
}
@Override
public void missing() {
nsv.missing();
}
@Override
public void addMissing(int count) {
nsv.addMissing(count);
}
@Override
public void addFacet(String facetName, Map<String,StatsValues> facetValues) {
nsv.addFacet(facetName, facetValues);
}
@Override
public NamedList<?> getStatsValues() {
return nsv.getStatsValues();
}
@Override
public void setNextReader(LeafReaderContext ctx) throws IOException {
sndv = DocValues.getSortedNumeric(ctx.reader(), fieldName);
assert sndv != null;
}
}

View File

@ -416,7 +416,7 @@ public class StatsField {
return StatsValuesFactory.createStatsValues(this);
}
if (null != schemaField
if (null != schemaField && !schemaField.getType().isPointField()
&& (schemaField.multiValued() || schemaField.getType().multiValuedFieldCache())) {
// TODO: should this also be used for single-valued string fields? (should work fine)

View File

@ -66,7 +66,12 @@ public class StatsValuesFactory {
if (TrieDateField.class.isInstance(fieldType)) {
return new DateStatsValues(statsField);
} else if (TrieField.class.isInstance(fieldType) || PointField.class.isInstance(fieldType)) {
return new NumericStatsValues(statsField);
NumericStatsValues statsValue = new NumericStatsValues(statsField);
if (sf.multiValued()) {
return new SortedNumericStatsValues(statsValue, statsField);
}
return statsValue;
} else if (StrField.class.isInstance(fieldType)) {
return new StringStatsValues(statsField);
} else if (sf.getType().getClass().equals(EnumField.class)) {

View File

@ -30,6 +30,7 @@ import org.apache.lucene.index.FilterNumericDocValues;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
@ -174,8 +175,12 @@ public class IntervalFacets implements Iterable<FacetInterval> {
}
private void doCount() throws IOException {
if (schemaField.getType().getNumberType() != null && !schemaField.multiValued()) {
getCountNumeric();
if (schemaField.getType().getNumberType() != null && (!schemaField.multiValued() || schemaField.getType().isPointField())) {
if (schemaField.multiValued()) {
getCountMultiValuedNumeric();
} else {
getCountNumeric();
}
} else {
getCountString();
}
@ -241,6 +246,36 @@ public class IntervalFacets implements Iterable<FacetInterval> {
}
}
}
private void getCountMultiValuedNumeric() throws IOException {
final FieldType ft = schemaField.getType();
final String fieldName = schemaField.getName();
if (ft.getNumberType() == null) {
throw new IllegalStateException();
}
final List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
final Iterator<LeafReaderContext> ctxIt = leaves.iterator();
LeafReaderContext ctx = null;
SortedNumericDocValues longs = null;
for (DocIterator docsIt = docs.iterator(); docsIt.hasNext(); ) {
final int doc = docsIt.nextDoc();
if (ctx == null || doc >= ctx.docBase + ctx.reader().maxDoc()) {
do {
ctx = ctxIt.next();
} while (ctx == null || doc >= ctx.docBase + ctx.reader().maxDoc());
assert doc >= ctx.docBase;
longs = DocValues.getSortedNumeric(ctx.reader(), fieldName);
}
int valuesDocID = longs.docID();
if (valuesDocID < doc - ctx.docBase) {
valuesDocID = longs.advance(doc - ctx.docBase);
}
if (valuesDocID == doc - ctx.docBase) {
accumIntervalWithMultipleValues(longs);
}
}
}
private void getCountString() throws IOException {
Filter filter = docs.getTopFilter();
@ -276,6 +311,44 @@ public class IntervalFacets implements Iterable<FacetInterval> {
}
}
private void accumIntervalWithMultipleValues(SortedNumericDocValues longs) throws IOException {
// longs should be already positioned to the correct doc
assert longs.docID() != -1;
assert longs.docValueCount() > 0: "Should have at least one value for this document";
int currentInterval = 0;
for (int i = 0; i < longs.docValueCount(); i++) {
boolean evaluateNextInterval = true;
long value = longs.nextValue();
while (evaluateNextInterval && currentInterval < intervals.length) {
IntervalCompareResult result = intervals[currentInterval].includes(value);
switch (result) {
case INCLUDED:
/*
* Increment the current interval and move to the next one using
* the same value
*/
intervals[currentInterval].incCount();
currentInterval++;
break;
case LOWER_THAN_START:
/*
* None of the next intervals will match this value (all of them have
* higher start value). Move to the next value for this document.
*/
evaluateNextInterval = false;
break;
case GREATER_THAN_END:
/*
* Next interval may match this value
*/
currentInterval++;
break;
}
//Maybe return if currentInterval == intervals.length?
}
}
}
private void accumIntervalsMulti(SortedSetDocValues ssdv,
DocIdSetIterator disi, Bits bits) throws IOException {
// First update the ordinals in the intervals for this segment

View File

@ -32,12 +32,14 @@ import org.apache.lucene.index.FilterNumericDocValues;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.ReaderUtil;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRefBuilder;
import org.apache.lucene.util.NumericUtils;
import org.apache.lucene.util.PriorityQueue;
import org.apache.lucene.util.StringHelper;
import org.apache.solr.common.params.FacetParams;
@ -61,16 +63,18 @@ final class NumericFacets {
long[] bits; // bits identifying a value
int[] counts;
int[] docIDs;
int[] docIDs; //Will be null if HashTable is created with needsDocId=false
int mask;
int size;
int threshold;
HashTable() {
HashTable(boolean needsDocId) {
final int capacity = 64; // must be a power of 2
bits = new long[capacity];
counts = new int[capacity];
docIDs = new int[capacity];
if (needsDocId) {
docIDs = new int[capacity];
}
mask = capacity - 1;
size = 0;
threshold = (int) (capacity * LOAD_FACTOR);
@ -99,6 +103,23 @@ final class NumericFacets {
break;
}
}
void add(long value, int count) {
if (size >= threshold) {
rehash();
}
final int h = hash(value);
for (int slot = h; ; slot = (slot + 1) & mask) {
if (counts[slot] == 0) {
bits[slot] = value;
++size;
} else if (bits[slot] != value) {
continue;
}
counts[slot] += count;
break;
}
}
private void rehash() {
final long[] oldBits = bits;
@ -108,14 +129,24 @@ final class NumericFacets {
final int newCapacity = bits.length * 2;
bits = new long[newCapacity];
counts = new int[newCapacity];
docIDs = new int[newCapacity];
if (oldDocIDs!= null) {
docIDs = new int[newCapacity];
}
mask = newCapacity - 1;
threshold = (int) (LOAD_FACTOR * newCapacity);
size = 0;
for (int i = 0; i < oldBits.length; ++i) {
if (oldCounts[i] > 0) {
add(oldDocIDs[i], oldBits[i], oldCounts[i]);
if (oldDocIDs!= null) {
for (int i = 0; i < oldBits.length; ++i) {
if (oldCounts[i] > 0) {
add(oldDocIDs[i], oldBits[i], oldCounts[i]);
}
}
} else {
for (int i = 0; i < oldBits.length; ++i) {
if (oldCounts[i] > 0) {
add(oldBits[i], oldCounts[i]);
}
}
}
}
@ -129,7 +160,16 @@ final class NumericFacets {
}
public static NamedList<Integer> getCounts(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort) throws IOException {
final boolean zeros = mincount <= 0;
final SchemaField sf = searcher.getSchema().getField(fieldName);
if (sf.multiValued()) {
// TODO: evaluate using getCountsMultiValued for singleValued numerics with SingletonSortedNumericDocValues
return getCountsMultiValued(searcher, docs, fieldName, offset, limit, mincount, missing, sort);
}
return getCountsSingleValue(searcher, docs, fieldName, offset, limit, mincount, missing, sort);
}
private static NamedList<Integer> getCountsSingleValue(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort) throws IOException {
boolean zeros = mincount <= 0;
mincount = Math.max(mincount, 1);
final SchemaField sf = searcher.getSchema().getField(fieldName);
final FieldType ft = sf.getType();
@ -137,10 +177,11 @@ final class NumericFacets {
if (numericType == null) {
throw new IllegalStateException();
}
zeros = zeros && !ft.isPointField() && sf.indexed(); // We don't return zeros when using PointFields or when index=false
final List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
// 1. accumulate
final HashTable hashTable = new HashTable();
final HashTable hashTable = new HashTable(true);
final Iterator<LeafReaderContext> ctxIt = leaves.iterator();
LeafReaderContext ctx = null;
NumericDocValues longs = null;
@ -363,4 +404,118 @@ final class NumericFacets {
return result;
}
private static NamedList<Integer> getCountsMultiValued(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort) throws IOException {
// If facet.mincount=0 with PointFields the only option is to get the values from DocValues
// not currently supported. See SOLR-10033
mincount = Math.max(mincount, 1);
final SchemaField sf = searcher.getSchema().getField(fieldName);
final FieldType ft = sf.getType();
assert sf.multiValued();
final List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
// 1. accumulate
final HashTable hashTable = new HashTable(false);
final Iterator<LeafReaderContext> ctxIt = leaves.iterator();
LeafReaderContext ctx = null;
SortedNumericDocValues longs = null;
int missingCount = 0;
for (DocIterator docsIt = docs.iterator(); docsIt.hasNext(); ) {
final int doc = docsIt.nextDoc();
if (ctx == null || doc >= ctx.docBase + ctx.reader().maxDoc()) {
do {
ctx = ctxIt.next();
} while (ctx == null || doc >= ctx.docBase + ctx.reader().maxDoc());
assert doc >= ctx.docBase;
longs = DocValues.getSortedNumeric(ctx.reader(), fieldName);
}
int valuesDocID = longs.docID();
if (valuesDocID < doc - ctx.docBase) {
valuesDocID = longs.advance(doc - ctx.docBase);
}
if (valuesDocID == doc - ctx.docBase) {
long l = longs.nextValue(); // This document must have at least one value
hashTable.add(l, 1);
for (int i = 1; i < longs.docValueCount(); i++) {
long lnew = longs.nextValue();
if (lnew > l) { // Skip the value if it's equal to the last one, we don't want to double-count it
hashTable.add(lnew, 1);
}
l = lnew;
}
} else {
++missingCount;
}
}
// 2. select top-k facet values
final int pqSize = limit < 0 ? hashTable.size : Math.min(offset + limit, hashTable.size);
final PriorityQueue<Entry> pq;
if (FacetParams.FACET_SORT_COUNT.equals(sort) || FacetParams.FACET_SORT_COUNT_LEGACY.equals(sort)) {
pq = new PriorityQueue<Entry>(pqSize) {
@Override
protected boolean lessThan(Entry a, Entry b) {
if (a.count < b.count || (a.count == b.count && a.bits > b.bits)) {
return true;
} else {
return false;
}
}
};
} else {
// sort=index
pq = new PriorityQueue<Entry>(pqSize) {
@Override
protected boolean lessThan(Entry a, Entry b) {
return a.bits > b.bits;
}
};
}
Entry e = null;
for (int i = 0; i < hashTable.bits.length; ++i) {
if (hashTable.counts[i] >= mincount) {
if (e == null) {
e = new Entry();
}
e.bits = hashTable.bits[i];
e.count = hashTable.counts[i];
e = pq.insertWithOverflow(e);
}
}
// 4. build the NamedList
final NamedList<Integer> result = new NamedList<>(Math.max(pq.size() - offset + 1, 1));
final Deque<Entry> counts = new ArrayDeque<>(pq.size() - offset);
while (pq.size() > offset) {
counts.addFirst(pq.pop());
}
for (Entry entry : counts) {
result.add(bitsToStringValue(ft, entry.bits), entry.count); // TODO: convert to correct value
}
// Once facet.mincount=0 is supported we'll need to add logic similar to the SingleValue case, but obtaining values
// with count 0 from DocValues
if (missing) {
result.add(null, missingCount);
}
return result;
}
private static String bitsToStringValue(FieldType fieldType, long bits) {
switch (fieldType.getNumberType()) {
case LONG:
case INTEGER:
return String.valueOf(bits);
case FLOAT:
return String.valueOf(NumericUtils.sortableIntToFloat((int)bits));
case DOUBLE:
return String.valueOf(NumericUtils.sortableLongToDouble(bits));
//TODO: DATE
default:
throw new AssertionError("Unsupported NumberType: " + fieldType.getNumberType());
}
}
}

View File

@ -482,8 +482,8 @@ public class SimpleFacets {
counts = getFacetTermEnumCounts(searcher, docs, field, offset, limit, mincount,missing,sort,prefix, termFilter, exists);
break;
case FCS:
assert !multiToken;
if (ft.getNumberType() != null && !sf.multiValued()) {
assert ft.isPointField() || !multiToken;
if (ft.isPointField() || (ft.getNumberType() != null && !sf.multiValued())) {
// force numeric faceting
if (prefix != null && !prefix.isEmpty()) {
throw new SolrException(ErrorCode.BAD_REQUEST, FacetParams.FACET_PREFIX + " is not supported on numeric types");
@ -494,6 +494,10 @@ public class SimpleFacets {
throw new SolrException(ErrorCode.BAD_REQUEST, FacetParams.FACET_CONTAINS + " is not supported on numeric types");
}
}
// We should do this, but mincount=0 is currently the default
// if (ft.isPointField() && mincount <= 0) {
// throw new SolrException(ErrorCode.BAD_REQUEST, FacetParams.FACET_MINCOUNT + " <= 0 is not supported on point types");
// }
counts = NumericFacets.getCounts(searcher, docs, field, offset, limit, mincount, missing, sort);
} else {
PerSegmentSingleValuedFaceting ps = new PerSegmentSingleValuedFaceting(searcher, docs, field, offset, limit, mincount, missing, sort, prefix, termFilter);

View File

@ -27,10 +27,13 @@ import org.apache.lucene.index.IndexableField;
import org.apache.lucene.legacy.LegacyNumericType;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
import org.apache.lucene.queries.function.valuesource.MultiValuedDoubleFieldSource;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.SortedNumericSelector;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils;
import org.apache.solr.search.QParser;
import org.apache.solr.uninverting.UninvertingReader.Type;
import org.slf4j.Logger;
@ -91,6 +94,8 @@ public class DoublePointField extends PointField implements DoubleValueFieldType
if (val != null) {
if (f.fieldType().stored() == false && f.fieldType().docValuesType() == DocValuesType.NUMERIC) {
return Double.longBitsToDouble(val.longValue());
} else if (f.fieldType().stored() == false && f.fieldType().docValuesType() == DocValuesType.SORTED_NUMERIC) {
return NumericUtils.sortableLongToDouble(val.longValue());
} else {
return val;
}
@ -149,8 +154,7 @@ public class DoublePointField extends PointField implements DoubleValueFieldType
@Override
public Type getUninversionType(SchemaField sf) {
if (sf.multiValued()) {
throw new UnsupportedOperationException("MultiValued Point fields with DocValues is not currently supported");
// return Type.SORTED_DOUBLE;
return Type.SORTED_DOUBLE;
} else {
return Type.DOUBLE_POINT;
}
@ -161,6 +165,11 @@ public class DoublePointField extends PointField implements DoubleValueFieldType
field.checkFieldCacheSource();
return new DoubleFieldSource(field.getName());
}
@Override
protected ValueSource getSingleValueSource(SortedNumericSelector.Type choice, SchemaField f) {
return new MultiValuedDoubleFieldSource(f.getName(), choice);
}
@Override
public LegacyNumericType getNumericType() {

View File

@ -626,7 +626,7 @@ public abstract class FieldType extends FieldProperties {
/**
* Return the numeric type of this field, or null if this field is not a
* numeric field.
* numeric field.
*/
public NumberType getNumberType() {
return null;

View File

@ -27,10 +27,13 @@ import org.apache.lucene.index.IndexableField;
import org.apache.lucene.legacy.LegacyNumericType;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.valuesource.FloatFieldSource;
import org.apache.lucene.queries.function.valuesource.MultiValuedFloatFieldSource;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.SortedNumericSelector;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils;
import org.apache.solr.search.QParser;
import org.apache.solr.uninverting.UninvertingReader.Type;
import org.slf4j.Logger;
@ -91,7 +94,9 @@ public class FloatPointField extends PointField implements FloatValueFieldType {
if (val != null) {
if (f.fieldType().stored() == false && f.fieldType().docValuesType() == DocValuesType.NUMERIC) {
return Float.intBitsToFloat(val.intValue());
} else {
} else if (f.fieldType().stored() == false && f.fieldType().docValuesType() == DocValuesType.SORTED_NUMERIC) {
return NumericUtils.sortableIntToFloat(val.intValue());
} else {
return val;
}
} else {
@ -149,8 +154,7 @@ public class FloatPointField extends PointField implements FloatValueFieldType {
@Override
public Type getUninversionType(SchemaField sf) {
if (sf.multiValued()) {
throw new UnsupportedOperationException("MultiValued Point fields with DocValues is not currently supported");
// return Type.SORTED_FLOAT;
return Type.SORTED_FLOAT;
} else {
return Type.FLOAT_POINT;
}
@ -161,6 +165,12 @@ public class FloatPointField extends PointField implements FloatValueFieldType {
field.checkFieldCacheSource();
return new FloatFieldSource(field.getName());
}
@Override
protected ValueSource getSingleValueSource(SortedNumericSelector.Type choice, SchemaField f) {
return new MultiValuedFloatFieldSource(f.getName(), choice);
}
@Override
public LegacyNumericType getNumericType() {

View File

@ -26,8 +26,10 @@ import org.apache.lucene.index.IndexableField;
import org.apache.lucene.legacy.LegacyNumericType;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.valuesource.IntFieldSource;
import org.apache.lucene.queries.function.valuesource.MultiValuedIntFieldSource;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.SortedNumericSelector;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.solr.search.QParser;
@ -149,8 +151,7 @@ public class IntPointField extends PointField implements IntValueFieldType {
@Override
public Type getUninversionType(SchemaField sf) {
if (sf.multiValued()) {
throw new UnsupportedOperationException("MultiValued Point fields with DocValues is not currently supported");
// return Type.SORTED_INTEGER;
return Type.SORTED_INTEGER;
} else {
return Type.INTEGER_POINT;
}
@ -182,5 +183,10 @@ public class IntPointField extends PointField implements IntValueFieldType {
protected StoredField getStoredField(SchemaField sf, Object value) {
return new StoredField(sf.getName(), (Integer) this.toNativeType(value));
}
@Override
protected ValueSource getSingleValueSource(SortedNumericSelector.Type choice, SchemaField f) {
return new MultiValuedIntFieldSource(f.getName(), choice);
}
}

View File

@ -26,6 +26,7 @@ import org.apache.lucene.index.IndexableField;
import org.apache.lucene.legacy.LegacyNumericType;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.valuesource.LongFieldSource;
import org.apache.lucene.queries.function.valuesource.MultiValuedLongFieldSource;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.SortField;
import org.apache.lucene.util.BytesRef;
@ -149,8 +150,7 @@ public class LongPointField extends PointField implements LongValueFieldType {
@Override
public Type getUninversionType(SchemaField sf) {
if (sf.multiValued()) {
throw new UnsupportedOperationException("MultiValued Point fields with DocValues is not currently supported");
// return Type.SORTED_LONG;
return Type.SORTED_LONG;
} else {
return Type.LONG_POINT;
}
@ -161,6 +161,12 @@ public class LongPointField extends PointField implements LongValueFieldType {
field.checkFieldCacheSource();
return new LongFieldSource(field.getName());
}
@Override
protected ValueSource getSingleValueSource(org.apache.lucene.search.SortedNumericSelector.Type choice,
SchemaField field) {
return new MultiValuedLongFieldSource(field.getName(), choice);
}
@Override
public LegacyNumericType getNumericType() {

View File

@ -17,9 +17,11 @@
package org.apache.solr.schema;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.NumericUtils;
import org.apache.solr.common.SolrException;
import org.apache.solr.search.FunctionRangeQuery;
import org.apache.solr.search.QParser;
@ -49,28 +51,36 @@ public abstract class NumericFieldType extends PrimitiveFieldType {
protected Query getDocValuesRangeQuery(QParser parser, SchemaField field, String min, String max,
boolean minInclusive, boolean maxInclusive) {
assert field.hasDocValues() && !field.multiValued();
assert field.hasDocValues() && (field.getType().isPointField() || !field.multiValued());
switch (getNumberType()) {
case INTEGER:
return numericDocValuesRangeQuery(field.getName(),
min == null ? null : (long) Integer.parseInt(min),
max == null ? null : (long) Integer.parseInt(max),
minInclusive, maxInclusive);
minInclusive, maxInclusive, field.multiValued());
case FLOAT:
return getRangeQueryForFloatDoubleDocValues(field, min, max, minInclusive, maxInclusive);
if (field.multiValued()) {
return getRangeQueryForMultiValuedFloatDocValues(field, min, max, minInclusive, maxInclusive);
} else {
return getRangeQueryForFloatDoubleDocValues(field, min, max, minInclusive, maxInclusive);
}
case LONG:
return numericDocValuesRangeQuery(field.getName(),
min == null ? null : Long.parseLong(min),
max == null ? null : Long.parseLong(max),
minInclusive, maxInclusive);
minInclusive, maxInclusive, field.multiValued());
case DOUBLE:
return getRangeQueryForFloatDoubleDocValues(field, min, max, minInclusive, maxInclusive);
if (field.multiValued()) {
return getRangeQueryForMultiValuedDoubleDocValues(field, min, max, minInclusive, maxInclusive);
} else {
return getRangeQueryForFloatDoubleDocValues(field, min, max, minInclusive, maxInclusive);
}
case DATE:
return numericDocValuesRangeQuery(field.getName(),
min == null ? null : DateMathParser.parseMath(null, min).getTime(),
max == null ? null : DateMathParser.parseMath(null, max).getTime(),
minInclusive, maxInclusive);
minInclusive, maxInclusive, field.multiValued());
default:
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown type for numeric field");
}
@ -104,19 +114,32 @@ public abstract class NumericFieldType extends PrimitiveFieldType {
if ((minVal == null || minVal.doubleValue() < 0d || minBits == minusZeroBits) &&
(maxVal != null && (maxVal.doubleValue() < 0d || maxBits == minusZeroBits))) {
query = numericDocValuesRangeQuery
(fieldName, maxBits, (min == null ? negativeInfinityBits : minBits), maxInclusive, minInclusive);
(fieldName, maxBits, (min == null ? negativeInfinityBits : minBits), maxInclusive, minInclusive, false);
} else { // If both max and min are positive, then issue range query
query = numericDocValuesRangeQuery
(fieldName, minBits, (max == null ? positiveInfinityBits : maxBits), minInclusive, maxInclusive);
(fieldName, minBits, (max == null ? positiveInfinityBits : maxBits), minInclusive, maxInclusive, false);
}
}
return query;
}
protected Query getRangeQueryForMultiValuedDoubleDocValues(SchemaField sf, String min, String max, boolean minInclusive, boolean maxInclusive) {
Long minBits = min == null ? NumericUtils.doubleToSortableLong(Double.NEGATIVE_INFINITY): NumericUtils.doubleToSortableLong(Double.parseDouble(min));
Long maxBits = max == null ? NumericUtils.doubleToSortableLong(Double.POSITIVE_INFINITY): NumericUtils.doubleToSortableLong(Double.parseDouble(max));
return numericDocValuesRangeQuery(sf.getName(), minBits, maxBits, minInclusive, maxInclusive, true);
}
protected Query getRangeQueryForMultiValuedFloatDocValues(SchemaField sf, String min, String max, boolean minInclusive, boolean maxInclusive) {
Long minBits = (long)(min == null ? NumericUtils.floatToSortableInt(Float.NEGATIVE_INFINITY): NumericUtils.floatToSortableInt(Float.parseFloat(min)));
Long maxBits = (long)(max == null ? NumericUtils.floatToSortableInt(Float.POSITIVE_INFINITY): NumericUtils.floatToSortableInt(Float.parseFloat(max)));
return numericDocValuesRangeQuery(sf.getName(), minBits, maxBits, minInclusive, maxInclusive, true);
}
public static Query numericDocValuesRangeQuery(
String field,
Number lowerValue, Number upperValue,
boolean lowerInclusive, boolean upperInclusive) {
boolean lowerInclusive, boolean upperInclusive,
boolean multiValued) {
long actualLowerValue = Long.MIN_VALUE;
if (lowerValue != null) {
@ -139,6 +162,11 @@ public abstract class NumericFieldType extends PrimitiveFieldType {
--actualUpperValue;
}
}
return NumericDocValuesField.newRangeQuery(field, actualLowerValue, actualUpperValue);
if (multiValued) {
// In multiValued case use SortedNumericDocValuesField, this won't work for Trie*Fields wince they use BinaryDV in the multiValue case
return SortedNumericDocValuesField.newRangeQuery(field, actualLowerValue, actualUpperValue);
} else {
return NumericDocValuesField.newRangeQuery(field, actualLowerValue, actualUpperValue);
}
}
}

View File

@ -24,15 +24,17 @@ import java.util.Collections;
import java.util.List;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.SortedSetSelector;
import org.apache.lucene.search.SortedNumericSelector;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.CharsRefBuilder;
import org.apache.lucene.util.NumericUtils;
import org.apache.solr.common.SolrException;
import org.apache.solr.response.TextResponseWriter;
import org.apache.solr.search.QParser;
@ -75,7 +77,7 @@ public abstract class PointField extends NumericFieldType {
// multivalued Point fields all use SortedSetDocValues, so we give a clean error if that's
// not supported by the specified choice, else we delegate to a helper
SortedSetSelector.Type selectorType = choice.getSortedSetSelectorType();
SortedNumericSelector.Type selectorType = choice.getSortedNumericSelectorType();
if (null == selectorType) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
choice.toString() + " is not a supported option for picking a single value"
@ -95,9 +97,7 @@ public abstract class PointField extends NumericFieldType {
* @param field the field to use, guaranteed to be multivalued.
* @see #getSingleValueSource(MultiValueSelector,SchemaField,QParser)
*/
protected ValueSource getSingleValueSource(SortedSetSelector.Type choice, SchemaField field) {
throw new UnsupportedOperationException("MultiValued Point fields with DocValues is not currently supported");
}
protected abstract ValueSource getSingleValueSource(SortedNumericSelector.Type choice, SchemaField field);
@Override
public boolean isTokenized() {
@ -130,7 +130,7 @@ public abstract class PointField extends NumericFieldType {
@Override
public Query getRangeQuery(QParser parser, SchemaField field, String min, String max, boolean minInclusive,
boolean maxInclusive) {
if (!field.indexed() && field.hasDocValues() && !field.multiValued()) {
if (!field.indexed() && field.hasDocValues()) {
return getDocValuesRangeQuery(parser, field, min, max, minInclusive, maxInclusive);
} else {
return getPointRangeQuery(parser, field, min, max, minInclusive, maxInclusive);
@ -203,10 +203,8 @@ public abstract class PointField extends NumericFieldType {
fields.add(field);
if (sf.hasDocValues()) {
if (sf.multiValued()) {
throw new UnsupportedOperationException("MultiValued Point fields with DocValues is not currently supported. Field: '" + sf.getName() + "'");
} else {
final long bits;
final long bits;
if (!sf.multiValued()) {
if (field.numericValue() instanceof Integer || field.numericValue() instanceof Long) {
bits = field.numericValue().longValue();
} else if (field.numericValue() instanceof Float) {
@ -216,8 +214,19 @@ public abstract class PointField extends NumericFieldType {
bits = Double.doubleToLongBits(field.numericValue().doubleValue());
}
fields.add(new NumericDocValuesField(sf.getName(), bits));
} else {
// MultiValued
if (field.numericValue() instanceof Integer || field.numericValue() instanceof Long) {
bits = field.numericValue().longValue();
} else if (field.numericValue() instanceof Float) {
bits = NumericUtils.floatToSortableInt(field.numericValue().floatValue());
} else {
assert field.numericValue() instanceof Double;
bits = NumericUtils.doubleToSortableLong(field.numericValue().doubleValue());
}
fields.add(new SortedNumericDocValuesField(sf.getName(), bits));
}
}
}
if (sf.stored()) {
fields.add(getStoredField(sf, value));
}

View File

@ -56,6 +56,7 @@ import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.ReaderUtil;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.index.StoredFieldVisitor.Status;
@ -98,6 +99,7 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.NumericUtils;
import org.apache.solr.common.SolrDocumentBase;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
@ -816,110 +818,136 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
log.warn("Couldn't decorate docValues for field: [{}], schemaField: [{}]", fieldName, schemaField);
continue;
}
if (schemaField.multiValued()) {
final SortedSetDocValues values = leafReader.getSortedSetDocValues(fieldName);
if (values != null && values.getValueCount() > 0) {
if (values.advance(localId) == localId) {
final List<Object> outValues = new LinkedList<Object>();
for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) {
final BytesRef value = values.lookupOrd(ord);
outValues.add(schemaField.getType().toObject(schemaField, value));
}
assert outValues.size() > 0;
doc.addField(fieldName, outValues);
FieldInfo fi = fieldInfos.fieldInfo(fieldName);
if (fi == null) {
continue; // Searcher doesn't have info about this field, hence ignore it.
}
final DocValuesType dvType = fi.getDocValuesType();
switch (dvType) {
case NUMERIC:
final NumericDocValues ndv = leafReader.getNumericDocValues(fieldName);
if (ndv == null) {
continue;
}
}
} else {
FieldInfo fi = fieldInfos.fieldInfo(fieldName);
if (fi == null) {
continue; // Searcher doesn't have info about this field, hence ignore it.
}
final DocValuesType dvType = fi.getDocValuesType();
switch (dvType) {
case NUMERIC:
final NumericDocValues ndv = leafReader.getNumericDocValues(fieldName);
if (ndv == null) {
continue;
}
Long val;
if (ndv.advanceExact(localId)) {
val = ndv.longValue();
} else {
continue;
}
Object newVal = val;
if (schemaField.getType().isPointField()) {
NumberType type = schemaField.getType().getNumberType();
switch (type) {
case INTEGER:
newVal = val.intValue();
break;
case LONG:
newVal = val.longValue();
break;
case FLOAT:
newVal = Float.intBitsToFloat(val.intValue());
break;
case DOUBLE:
newVal = Double.longBitsToDouble(val);
break;
case DATE:
newVal = new Date(val);
break;
default:
throw new AssertionError("Unexpected PointType: " + type);
}
} else {
if (schemaField.getType() instanceof TrieIntField) {
Long val;
if (ndv.advanceExact(localId)) {
val = ndv.longValue();
} else {
continue;
}
Object newVal = val;
if (schemaField.getType().isPointField()) {
// TODO: Maybe merge PointField with TrieFields here
NumberType type = schemaField.getType().getNumberType();
switch (type) {
case INTEGER:
newVal = val.intValue();
} else if (schemaField.getType() instanceof TrieFloatField) {
break;
case LONG:
newVal = val.longValue();
break;
case FLOAT:
newVal = Float.intBitsToFloat(val.intValue());
} else if (schemaField.getType() instanceof TrieDoubleField) {
break;
case DOUBLE:
newVal = Double.longBitsToDouble(val);
} else if (schemaField.getType() instanceof TrieDateField) {
break;
case DATE:
newVal = new Date(val);
} else if (schemaField.getType() instanceof EnumField) {
newVal = ((EnumField) schemaField.getType()).intValueToStringValue(val.intValue());
}
break;
default:
throw new AssertionError("Unexpected PointType: " + type);
}
doc.addField(fieldName, newVal);
break;
case BINARY:
BinaryDocValues bdv = leafReader.getBinaryDocValues(fieldName);
if (bdv == null) {
continue;
} else {
if (schemaField.getType() instanceof TrieIntField) {
newVal = val.intValue();
} else if (schemaField.getType() instanceof TrieFloatField) {
newVal = Float.intBitsToFloat(val.intValue());
} else if (schemaField.getType() instanceof TrieDoubleField) {
newVal = Double.longBitsToDouble(val);
} else if (schemaField.getType() instanceof TrieDateField) {
newVal = new Date(val);
} else if (schemaField.getType() instanceof EnumField) {
newVal = ((EnumField) schemaField.getType()).intValueToStringValue(val.intValue());
}
BytesRef value;
if (bdv.advanceExact(localId)) {
value = BytesRef.deepCopyOf(bdv.binaryValue());
}
doc.addField(fieldName, newVal);
break;
case BINARY:
BinaryDocValues bdv = leafReader.getBinaryDocValues(fieldName);
if (bdv == null) {
continue;
}
BytesRef value;
if (bdv.advanceExact(localId)) {
value = BytesRef.deepCopyOf(bdv.binaryValue());
} else {
continue;
}
doc.addField(fieldName, value);
break;
case SORTED:
SortedDocValues sdv = leafReader.getSortedDocValues(fieldName);
if (sdv == null) {
continue;
}
if (sdv.advanceExact(localId)) {
final BytesRef bRef = sdv.binaryValue();
// Special handling for Boolean fields since they're stored as 'T' and 'F'.
if (schemaField.getType() instanceof BoolField) {
doc.addField(fieldName, schemaField.getType().toObject(schemaField, bRef));
} else {
continue;
doc.addField(fieldName, bRef.utf8ToString());
}
doc.addField(fieldName, value);
break;
case SORTED:
SortedDocValues sdv = leafReader.getSortedDocValues(fieldName);
if (sdv == null) {
continue;
}
if (sdv.advanceExact(localId)) {
final BytesRef bRef = sdv.binaryValue();
// Special handling for Boolean fields since they're stored as 'T' and 'F'.
if (schemaField.getType() instanceof BoolField) {
doc.addField(fieldName, schemaField.getType().toObject(schemaField, bRef));
} else {
doc.addField(fieldName, bRef.utf8ToString());
}
break;
case SORTED_NUMERIC:
final SortedNumericDocValues numericDv = leafReader.getSortedNumericDocValues(fieldName);
NumberType type = schemaField.getType().getNumberType();
if (numericDv != null) {
if (numericDv.advance(localId) == localId) {
final List<Object> outValues = new ArrayList<Object>(numericDv.docValueCount());
for (int i = 0; i < numericDv.docValueCount(); i++) {
long number = numericDv.nextValue();
switch (type) {
case INTEGER:
outValues.add((int)number);
break;
case LONG:
outValues.add(number);
break;
case FLOAT:
outValues.add(NumericUtils.sortableIntToFloat((int)number));
break;
case DOUBLE:
outValues.add(NumericUtils.sortableLongToDouble(number));
break;
case DATE:
newVal = new Date(number);
break;
default:
throw new AssertionError("Unexpected PointType: " + type);
}
}
assert outValues.size() > 0;
doc.addField(fieldName, outValues);
}
break;
case SORTED_NUMERIC:
throw new AssertionError("SORTED_NUMERIC not supported yet!");
case SORTED_SET:
throw new AssertionError("SORTED_SET fields should be multi-valued!");
case NONE:
break;
}
}
case SORTED_SET:
final SortedSetDocValues values = leafReader.getSortedSetDocValues(fieldName);
if (values != null && values.getValueCount() > 0) {
if (values.advance(localId) == localId) {
final List<Object> outValues = new LinkedList<Object>();
for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) {
value = values.lookupOrd(ord);
outValues.add(schemaField.getType().toObject(schemaField, value));
}
assert outValues.size() > 0;
doc.addField(fieldName, outValues);
}
}
case NONE:
break;
}
}
}

View File

@ -23,6 +23,7 @@ import java.util.Map;
import org.apache.lucene.document.BinaryDocValuesField; // javadocs
import org.apache.lucene.document.NumericDocValuesField; // javadocs
import org.apache.lucene.document.SortedDocValuesField; // javadocs
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.document.SortedSetDocValuesField; // javadocs
import org.apache.lucene.document.StringField; // javadocs
import org.apache.lucene.index.BinaryDocValues;
@ -169,7 +170,35 @@ public class UninvertingReader extends FilterLeafReader {
* Fields with this type act as if they were indexed with
* {@link SortedSetDocValuesField}.
*/
SORTED_SET_DOUBLE
SORTED_SET_DOUBLE,
/**
* Multi-valued Integer, (e.g. indexed with {@link org.apache.lucene.document.IntPoint})
* <p>
* Fields with this type act as if they were indexed with
* {@link SortedNumericDocValuesField}.
*/
SORTED_INTEGER,
/**
* Multi-valued Float, (e.g. indexed with {@link org.apache.lucene.document.FloatPoint})
* <p>
* Fields with this type act as if they were indexed with
* {@link SortedNumericDocValuesField}.
*/
SORTED_FLOAT,
/**
* Multi-valued Long, (e.g. indexed with {@link org.apache.lucene.document.LongPoint})
* <p>
* Fields with this type act as if they were indexed with
* {@link SortedNumericDocValuesField}.
*/
SORTED_LONG,
/**
* Multi-valued Double, (e.g. indexed with {@link org.apache.lucene.document.DoublePoint})
* <p>
* Fields with this type act as if they were indexed with
* {@link SortedNumericDocValuesField}.
*/
SORTED_DOUBLE
}
/**
@ -255,6 +284,12 @@ public class UninvertingReader extends FilterLeafReader {
case SORTED_SET_DOUBLE:
type = DocValuesType.SORTED_SET;
break;
case SORTED_INTEGER:
case SORTED_FLOAT:
case SORTED_LONG:
case SORTED_DOUBLE:
type = DocValuesType.SORTED_NUMERIC;
break;
default:
throw new AssertionError();
}

View File

@ -37,20 +37,20 @@
<dynamicField name="*_i" type="int" indexed="true" stored="false" docValues="false"/>
<dynamicField name="*_i_dv" type="${solr.tests.intClass:pint}" indexed="false" stored="false" docValues="true"/>
<dynamicField name="*_is" type="int" indexed="true" stored="false" docValues="false" multiValued="true"/>
<dynamicField name="*_is_dv" type="int" indexed="false" stored="false" docValues="true" multiValued="true"/>
<dynamicField name="*_is_dv" type="${solr.tests.intClass:pint}" indexed="false" stored="false" docValues="true" multiValued="true"/>
<dynamicField name="*_s" type="string" indexed="true" stored="false" docValues="false"/>
<dynamicField name="*_s_dv" type="string" indexed="false" stored="false" docValues="true"/>
<dynamicField name="*_ss" type="string" indexed="true" stored="false" docValues="false" multiValued="true"/>
<dynamicField name="*_ss_dv" type="string" indexed="false" stored="false" docValues="true" multiValued="true"/>
<dynamicField name="*_f" type="float" indexed="true" stored="false" docValues="false"/>
<dynamicField name="*_f_dv" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="false" docValues="true"/>
<dynamicField name="*_fs_dv" type="float" indexed="true" stored="false" docValues="true" multiValued="true"/>
<dynamicField name="*_fs_dv" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="false" docValues="true" multiValued="true"/>
<dynamicField name="*_l" type="long" indexed="true" stored="false" docValues="false"/>
<dynamicField name="*_l_dv" type="${solr.tests.longClass:plong}" indexed="true" stored="false" docValues="true"/>
<dynamicField name="*_ls_dv" type="long" indexed="true" stored="false" docValues="true" multiValued="true"/>
<dynamicField name="*_ls_dv" type="${solr.tests.longClass:plong}" indexed="true" stored="false" docValues="true" multiValued="true"/>
<dynamicField name="*_d" type="double" indexed="true" stored="false" docValues="false"/>
<dynamicField name="*_d_dv" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="false" docValues="true"/>
<dynamicField name="*_ds_dv" type="double" indexed="true" stored="false" docValues="true" multiValued="true"/>
<dynamicField name="*_ds_dv" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="false" docValues="true" multiValued="true"/>
<dynamicField name="*_dt" type="date" indexed="true" stored="false" docValues="false"/>
<dynamicField name="*_dt_dv" type="date" indexed="true" stored="false" docValues="true"/>
<dynamicField name="*_dts_dv" type="date" indexed="true" stored="false" docValues="true" multiValued="true"/>

View File

@ -34,8 +34,9 @@
<dynamicField name="*_i" type="int" indexed="true" stored="false" docValues="false"/>
<dynamicField name="*_i_dv" type="int" indexed="false" stored="false" docValues="true"/>
<dynamicField name="*_i_p" type="pint" indexed="true" stored="false" docValues="true"/>
<dynamicField name="*_is" type="int" indexed="true" stored="false" docValues="false" multiValued="true"/>
<dynamicField name="*_is" type="int" indexed="true" stored="true" docValues="false" multiValued="true"/>
<dynamicField name="*_is_dv" type="int" indexed="false" stored="false" docValues="true" multiValued="true"/>
<dynamicField name="*_is_p" type="pint" indexed="true" stored="false" docValues="true" multiValued="true"/>
<dynamicField name="*_s" type="string" indexed="true" stored="false" docValues="false"/>
<dynamicField name="*_s_dv" type="string" indexed="false" stored="false" docValues="true"/>
<dynamicField name="*_ss" type="string" indexed="true" stored="false" docValues="false" multiValued="true"/>
@ -45,16 +46,19 @@
<dynamicField name="*_f_p" type="pfloat" indexed="true" stored="false" docValues="true"/>
<dynamicField name="*_fs" type="float" indexed="true" stored="false" docValues="false" multiValued="true"/>
<dynamicField name="*_fs_dv" type="float" indexed="true" stored="false" docValues="true" multiValued="true"/>
<dynamicField name="*_fs_p" type="pfloat" indexed="true" stored="false" docValues="true" multiValued="true"/>
<dynamicField name="*_l" type="long" indexed="true" stored="false" docValues="false"/>
<dynamicField name="*_l_dv" type="long" indexed="true" stored="false" docValues="true"/>
<dynamicField name="*_l_p" type="plong" indexed="true" stored="false" docValues="true"/>
<dynamicField name="*_ls" type="long" indexed="true" stored="false" docValues="false" multiValued="true"/>
<dynamicField name="*_ls_dv" type="long" indexed="true" stored="false" docValues="true" multiValued="true"/>
<dynamicField name="*_ls_p" type="plong" indexed="true" stored="false" docValues="true" multiValued="true"/>
<dynamicField name="*_d" type="double" indexed="true" stored="false" docValues="false"/>
<dynamicField name="*_d_dv" type="double" indexed="true" stored="false" docValues="true"/>
<dynamicField name="*_d_p" type="pdouble" indexed="true" stored="false" docValues="true"/>
<dynamicField name="*_ds" type="double" indexed="true" stored="false" docValues="false" multiValued="true"/>
<dynamicField name="*_ds_dv" type="double" indexed="true" stored="false" docValues="true" multiValued="true"/>
<dynamicField name="*_ds_p" type="pdouble" indexed="true" stored="false" docValues="true" multiValued="true"/>
<dynamicField name="*_dt" type="date" indexed="true" stored="false" docValues="false"/>
<dynamicField name="*_dt_dv" type="date" indexed="true" stored="false" docValues="true"/>
<dynamicField name="*_dts_dv" type="date" indexed="true" stored="false" docValues="true" multiValued="true"/>
@ -67,15 +71,18 @@
<copyField source="*_f" dest="*_f_dv"/>
<copyField source="*_f" dest="*_f_p"/>
<copyField source="*_is" dest="*_is_dv"/>
<copyField source="*_is" dest="*_is_p"/>
<copyField source="*_s" dest="*_s_dv"/>
<copyField source="*_l" dest="*_l_dv"/>
<copyField source="*_l" dest="*_l_p"/>
<copyField source="*_d" dest="*_d_dv"/>
<copyField source="*_d" dest="*_d_p"/>
<copyField source="*_ss" dest="*_ss_dv"/>
<copyField source="*_is" dest="*_is_dv"/>
<copyField source="*_fs" dest="*_fs_dv"/>
<copyField source="*_fs" dest="*_fs_p"/>
<copyField source="*_ls" dest="*_ls_dv"/>
<copyField source="*_ls" dest="*_ls_p"/>
<copyField source="*_ds" dest="*_ds_dv"/>
<copyField source="*_ds" dest="*_ds_p"/>
<copyField source="id" dest="id_dv"/>
</schema>

View File

@ -84,6 +84,12 @@
<dynamicField name="*_p_d_ni_ns_dv" type="pdouble" indexed="false" stored="false" docValues="true" useDocValuesAsStored="true"/>
<dynamicField name="*_p_f_ni_ns_dv" type="pfloat" indexed="false" stored="false" docValues="true" useDocValuesAsStored="true"/>
<dynamicField name="*_p_i_dv_ns_mv" type="pint" indexed="true" stored="false" docValues="true" useDocValuesAsStored="true" multiValued="true"/>
<dynamicField name="*_p_d_dv_ns_mv" type="pdouble" indexed="true" stored="false" docValues="true" useDocValuesAsStored="true" multiValued="true"/>
<dynamicField name="*_p_l_dv_ns_mv" type="plong" indexed="true" stored="false" docValues="true" useDocValuesAsStored="true" multiValued="true"/>
<dynamicField name="*_p_f_dv_ns_mv" type="pfloat" indexed="true" stored="false" docValues="true" useDocValuesAsStored="true" multiValued="true"/>
</fields>
<uniqueKey>id</uniqueKey>

View File

@ -571,18 +571,17 @@
<field name="timestamp" type="date" indexed="true" stored="true" docValues="true" default="NOW" multiValued="false"/>
<field name="multiDefault" type="string" indexed="true" stored="true" default="muLti-Default" multiValued="true"/>
<field name="intDefault" type="int" indexed="true" stored="true" default="42" multiValued="false"/>
<field name="intDvoDefault" type="int" indexed="false" stored="false" multiValued="false"
<field name="intDefault" type="${solr.tests.intClass:pint}" indexed="true" stored="true" default="42" multiValued="false"/>
<field name="intDvoDefault" type="${solr.tests.intClass:pint}" indexed="false" stored="false" multiValued="false"
useDocValuesAsStored="true" docValues="true" default="42" />
<field name="intRemove" type="int" indexed="true" stored="true" multiValued="true"/>
<field name="intRemove" type="${solr.tests.intClass:pint}" indexed="true" stored="true" multiValued="true"/>
<field name="dateRemove" type="date" indexed="true" stored="true" multiValued="true"/>
<field name="floatRemove" type="float" indexed="true" stored="true" multiValued="true"/>
<field name="floatRemove" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" multiValued="true"/>
<field name="nopositionstext" type="nopositions" indexed="true" stored="true"/>
<field name="tlong" type="tlong" indexed="true" stored="true"/>
<field name="_version_" type="long" indexed="false" stored="false" docValues="true" multiValued="false" useDocValuesAsStored="true"/>
<field name="_version_" type="${solr.tests.longClass:plong}" indexed="false" stored="false" docValues="true" multiValued="false" useDocValuesAsStored="true"/>
<field name="title_stringNoNorms" type="string" omitNorms="true" indexed="true" stored="true"/>
@ -612,14 +611,14 @@
<dynamicField name="*_s" type="string" indexed="true" stored="true"/>
<dynamicField name="*_s1" type="string" indexed="true" stored="true" multiValued="false"/>
<dynamicField name="*_l" type="long" indexed="true" stored="true"/>
<dynamicField name="*_l1" type="long" indexed="true" stored="true" multiValued="false"/>
<dynamicField name="*_l" type="${solr.tests.longClass:plong}" indexed="true" stored="true"/>
<dynamicField name="*_l1" type="${solr.tests.longClass:plong}" indexed="true" stored="true" multiValued="false"/>
<dynamicField name="*_t" type="text" indexed="true" stored="true"/>
<dynamicField name="*_b" type="boolean" indexed="true" stored="true"/>
<dynamicField name="*_b1" type="boolean" indexed="true" stored="true" multiValued="false"/>
<dynamicField name="*_f" type="float" indexed="true" stored="true"/>
<dynamicField name="*_f" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true"/>
<dynamicField name="*_f1" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" multiValued="false"/>
<dynamicField name="*_d" type="double" indexed="true" stored="true"/>
<dynamicField name="*_d" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true"/>
<dynamicField name="*_d1" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true" multiValued="false"/>
<dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
<dynamicField name="*_dt1" type="date" indexed="true" stored="true" multiValued="false"/>
@ -679,10 +678,10 @@
<dynamicField name="*_sev_enum" type="severityType" indexed="true" stored="true" docValues="true" multiValued="true"/>
<!-- With DocValues=true -->
<dynamicField name="*_i_dv" type="int" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_l_dv" type="long" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_f_dv" type="float" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_d_dv" type="double" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_i_dv" type="${solr.tests.intClass:pint}" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_l_dv" type="${solr.tests.longClass:plong}" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_f_dv" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_d_dv" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_dt_dv" type="date" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_f1_dv" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" docValues="true" multiValued="false"/>
@ -693,21 +692,41 @@
useDocValuesAsStored="true"/>
<dynamicField name="*_s_dvo" multiValued="false" type="string" docValues="true" indexed="false" stored="false"
useDocValuesAsStored="true"/>
<dynamicField name="*_ii_dvo" multiValued="true" type="int" docValues="true" indexed="false" stored="false"
<dynamicField name="*_ii_dvo" multiValued="true" type="${solr.tests.intClass:pint}" docValues="true" indexed="false" stored="false"
useDocValuesAsStored="true"/>
<dynamicField name="*_dd_dvo" multiValued="true" type="double" docValues="true" indexed="false" stored="false"
<dynamicField name="*_dd_dvo" multiValued="true" type="${solr.tests.doubleClass:pdouble}" docValues="true" indexed="false" stored="false"
useDocValuesAsStored="true"/>
<!-- Non-stored, DocValues=true, useDocValuesAsStored=false -->
<field name="single_i_dvn" multiValued="false" type="${solr.tests.intClass:pint}" indexed="true" stored="true"/>
<field name="single_d_dvn" multiValued="false" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true"/>
<field name="single_s_dvn" multiValued="false" type="string" indexed="true" stored="true"/>
<field name="copy_single_i_dvn" multiValued="false" type="int" docValues="true" indexed="true" stored="false"
<field name="copy_single_i_dvn" multiValued="false" type="${solr.tests.intClass:pint}" docValues="true" indexed="true" stored="false"
useDocValuesAsStored="false"/>
<field name="copy_single_d_dvn" multiValued="false" type="double" docValues="true" indexed="true" stored="false"
<field name="copy_single_d_dvn" multiValued="false" type="${solr.tests.doubleClass:pdouble}" docValues="true" indexed="true" stored="false"
useDocValuesAsStored="false"/>
<field name="copy_single_s_dvn" multiValued="false" type="string" docValues="true" indexed="true" stored="false"
useDocValuesAsStored="false"/>
<!-- Test point fields explicitly -->
<dynamicField name="*_i_p" type="pint" indexed="true" stored="true" docValues="true" multiValued="false"/>
<dynamicField name="*_is_p" type="pint" indexed="true" stored="true" docValues="true" multiValued="true"/>
<dynamicField name="*_i_ni_p" type="pint" indexed="false" stored="true" docValues="true" multiValued="false"/>
<dynamicField name="*_is_ni_p" type="pint" indexed="false" stored="true" docValues="true" multiValued="true"/>
<dynamicField name="*_l_p" type="plong" indexed="true" stored="true" docValues="true" multiValued="false"/>
<dynamicField name="*_ls_p" type="plong" indexed="true" stored="true" docValues="true" multiValued="true"/>
<dynamicField name="*_l_ni_p" type="plong" indexed="false" stored="true" docValues="true" multiValued="false"/>
<dynamicField name="*_ls_ni_p" type="plong" indexed="false" stored="true" docValues="true" multiValued="true"/>
<dynamicField name="*_f_p" type="pfloat" indexed="true" stored="true" docValues="true" multiValued="false"/>
<dynamicField name="*_fs_p" type="pfloat" indexed="true" stored="true" docValues="true" multiValued="true"/>
<dynamicField name="*_f_ni_p" type="pfloat" indexed="false" stored="true" docValues="true" multiValued="false"/>
<dynamicField name="*_fs_ni_p" type="pfloat" indexed="false" stored="true" docValues="true" multiValued="true"/>
<dynamicField name="*_d_p" type="pdouble" indexed="true" stored="true" docValues="true" multiValued="false"/>
<dynamicField name="*_ds_p" type="pdouble" indexed="true" stored="true" docValues="true" multiValued="true"/>
<dynamicField name="*_d_ni_p" type="pdouble" indexed="false" stored="true" docValues="true" multiValued="false"/>
<dynamicField name="*_ds_ni_p" type="pdouble" indexed="false" stored="true" docValues="true" multiValued="true"/>
<copyField source="single_i_dvn" dest="copy_single_i_dvn"/>
<copyField source="single_d_dvn" dest="copy_single_d_dvn"/>
<copyField source="single_s_dvn" dest="copy_single_s_dvn"/>

View File

@ -322,12 +322,12 @@ valued. -->
<field name="text_no_analyzer" type="text_no_analyzer" indexed="true" />
<field name="_version_" type="long" indexed="true" stored="true" multiValued="false" />
<field name="_version_" type="${solr.tests.longClass:plong}" indexed="true" stored="true" multiValued="false" />
<field name="cat" type="string" indexed="true" stored="true" multiValued="true"/>
<field name="cat_docValues" type="string" indexed="true" stored="true" docValues="true" multiValued="true" />
<field name="cat_intDocValues" type="int" indexed="true" stored="true" docValues="true" multiValued="true" />
<field name="cat_floatDocValues" type="float" indexed="true" stored="true" docValues="true" multiValued="true" />
<field name="cat_intDocValues" type="${solr.tests.intClass:pint}" indexed="true" stored="true" docValues="true" multiValued="true" />
<field name="cat_floatDocValues" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" docValues="true" multiValued="true" />
<field name="cat_length" type="text_length" indexed="true" stored="true" multiValued="true"/>
@ -346,14 +346,14 @@ valued. -->
<dynamicField name="*_ss" type="string" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_sS" type="string" indexed="false" stored="true"/>
<dynamicField name="*_i" type="${solr.tests.intClass:pint}" indexed="true" stored="true"/>
<dynamicField name="*_ii" type="int" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_ii" type="${solr.tests.intClass:pint}" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_l" type="${solr.tests.longClass:plong}" indexed="true" stored="true"/>
<dynamicField name="*_f" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true"/>
<dynamicField name="*_d" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true"/>
<dynamicField name="*_ti" type="tint" indexed="true" stored="true"/>
<dynamicField name="*_ti_dv" type="tint" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_ti_ni_dv" type="tint" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_ti_ni_dv" type="tint" indexed="false" stored="true" docValues="true"/>
<dynamicField name="*_tl" type="tlong" indexed="true" stored="true"/>
<dynamicField name="*_tl_dv" type="tlong" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_tl_ni_dv" type="tlong" indexed="false" stored="true" docValues="true"/>
@ -382,6 +382,24 @@ valued. -->
<dynamicField name="*_tdts" type="tdates" indexed="true" stored="true"/>
<dynamicField name="*_tdts_dv" type="tdates" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_tdts_ni_dv" type="tdates" indexed="false" stored="true" docValues="true"/>
<!-- Test point fields explicitly -->
<dynamicField name="*_i_p" type="pint" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_is_p" type="pint" indexed="true" stored="true" docValues="true" multiValued="true"/>
<dynamicField name="*_i_ni_p" type="pint" indexed="false" stored="true" docValues="true"/>
<dynamicField name="*_is_ni_p" type="pint" indexed="false" stored="true" docValues="true" multiValued="true"/>
<dynamicField name="*_l_p" type="plong" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_ls_p" type="plong" indexed="true" stored="true" docValues="true" multiValued="true"/>
<dynamicField name="*_l_ni_p" type="plong" indexed="false" stored="true" docValues="true"/>
<dynamicField name="*_ls_ni_p" type="plong" indexed="false" stored="true" docValues="true" multiValued="true"/>
<dynamicField name="*_f_p" type="pfloat" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_fs_p" type="pfloat" indexed="true" stored="true" docValues="true" multiValued="true"/>
<dynamicField name="*_f_ni_p" type="pfloat" indexed="false" stored="true" docValues="true"/>
<dynamicField name="*_fs_ni_p" type="pfloat" indexed="false" stored="true" docValues="true" multiValued="true"/>
<dynamicField name="*_d_p" type="pdouble" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_ds_p" type="pdouble" indexed="true" stored="true" docValues="true" multiValued="true"/>
<dynamicField name="*_d_ni_p" type="pdouble" indexed="false" stored="true" docValues="true"/>
<dynamicField name="*_ds_ni_p" type="pdouble" indexed="false" stored="true" docValues="true" multiValued="true"/>
<dynamicField name="*_t" type="text" indexed="true" stored="true"/>
<dynamicField name="*_b" type="boolean" indexed="true" stored="true"/>

View File

@ -458,8 +458,8 @@
<field name="bind" type="boolean" indexed="true" stored="false"/>
<field name="bsto" type="boolean" indexed="false" stored="true"/>
<field name="bindsto" type="boolean" indexed="true" stored="true"/>
<field name="isto" type="int" indexed="false" stored="true"/>
<field name="iind" type="int" indexed="true" stored="false"/>
<field name="isto" type="${solr.tests.intClass:pint}" indexed="false" stored="true"/>
<field name="iind" type="${solr.tests.intClass:pint}" indexed="true" stored="false"/>
<field name="ssto" type="string" indexed="false" stored="true"/>
<field name="sind" type="string" indexed="true" stored="false"/>
<field name="sindsto" type="string" indexed="true" stored="true"/>
@ -554,20 +554,20 @@
<dynamicField name="*_sw" type="text_sw" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_i" type="int" indexed="true" stored="true"/>
<dynamicField name="*_is" type="int" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_i" type="${solr.tests.intClass:pint}" indexed="true" stored="true"/>
<dynamicField name="*_is" type="${solr.tests.intClass:pint}" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_i_dv" type="${solr.tests.intClass:pint}" indexed="true" stored="true" docValues="true" multiValued="false"/>
<dynamicField name="*_is_dv" type="${solr.tests.intClass:pint}" indexed="true" stored="true" docValues="true" multiValued="false"/>
<dynamicField name="*_is_dv" type="${solr.tests.intClass:pint}" indexed="true" stored="true" docValues="true" multiValued="true"/>
<dynamicField name="*_s1" type="string" indexed="true" stored="true" multiValued="false"/>
<!-- :TODO: why are these identical?!?!?! -->
<dynamicField name="*_s" type="string" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_ss" type="string" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_l" type="long" indexed="true" stored="true"/>
<dynamicField name="*_l" type="${solr.tests.longClass:plong}" indexed="true" stored="true"/>
<dynamicField name="*_t" type="text" indexed="true" stored="true"/>
<dynamicField name="*_tt" type="text" indexed="true" stored="true"/>
<dynamicField name="*_b" type="boolean" indexed="true" stored="true"/>
<dynamicField name="*_f" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true"/>
<dynamicField name="*_d" type="double" indexed="true" stored="true"/>
<dynamicField name="*_d" type="${solr.tests.doubleClass:pdouble}" 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"/>

View File

@ -109,7 +109,7 @@
<!-- If you remove this field, you must _also_ disable the update log in solrconfig.xml
or Solr won't start. _version_ and update log are required for SolrCloud
-->
<field name="_version_" type="long" indexed="false" stored="false" docValues="true"/>
<field name="_version_" type="${solr.tests.longClass:plong}" indexed="false" stored="false" docValues="true"/>
<!-- points to the root document of a block of nested documents. Required for nested
document support, may be removed otherwise
@ -130,9 +130,9 @@
<field name="includes" type="text_general" indexed="true" stored="true" termVectors="true" termPositions="true"
termOffsets="true"/>
<field name="weight" type="float" indexed="true" stored="true"/>
<field name="price" type="float" indexed="true" stored="true"/>
<field name="popularity" type="int" indexed="true" stored="true"/>
<field name="weight" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true"/>
<field name="price" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true"/>
<field name="popularity" type="${solr.tests.intClass:pint}" indexed="true" stored="true"/>
<field name="inStock" type="boolean" indexed="true" stored="true"/>
<field name="store" type="location" indexed="true" stored="true"/>
@ -200,27 +200,27 @@
a "*" only at the start or the end. -->
<!-- docvalues and stored are exclusive -->
<dynamicField name="*_i" type="int" indexed="true" stored="true"/>
<dynamicField name="*_is" type="int" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_i" type="${solr.tests.intClass:pint}" indexed="true" stored="true"/>
<dynamicField name="*_is" type="${solr.tests.intClass:pint}" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_id" type="${solr.tests.intClass:pint}" indexed="true" stored="false" docValues="true"/>
<dynamicField name="*_ids" type="int" indexed="true" stored="false" multiValued="true" docValues="true"/>
<dynamicField name="*_ids" type="${solr.tests.intClass:pint}" indexed="true" stored="false" multiValued="true" docValues="true"/>
<dynamicField name="*_s" type="string" indexed="true" stored="true"/>
<dynamicField name="*_s1" type="string" indexed="true" stored="true"/>
<dynamicField name="*_ss" type="string" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_sd" type="string" indexed="true" stored="false" docValues="true"/>
<dynamicField name="*_sds" type="string" indexed="true" stored="false" multiValued="true" docValues="true"/>
<dynamicField name="*_l" type="long" indexed="true" stored="true"/>
<dynamicField name="*_ls" type="long" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_ld" type="long" indexed="true" stored="false" docValues="true"/>
<dynamicField name="*_lds" type="long" indexed="true" stored="false" multiValued="true" docValues="true"/>
<dynamicField name="*_f" type="float" indexed="true" stored="true"/>
<dynamicField name="*_fs" type="float" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_l" type="${solr.tests.longClass:plong}" indexed="true" stored="true"/>
<dynamicField name="*_ls" type="${solr.tests.longClass:plong}" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_ld" type="${solr.tests.longClass:plong}" indexed="true" stored="false" docValues="true"/>
<dynamicField name="*_lds" type="${solr.tests.longClass:plong}" indexed="true" stored="false" multiValued="true" docValues="true"/>
<dynamicField name="*_f" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true"/>
<dynamicField name="*_fs" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_fd" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="false" docValues="true"/>
<dynamicField name="*_fds" type="float" indexed="true" stored="false" multiValued="true" docValues="true"/>
<dynamicField name="*_d" type="double" indexed="true" stored="true"/>
<dynamicField name="*_ds" type="double" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_fds" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="false" multiValued="true" docValues="true"/>
<dynamicField name="*_d" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true"/>
<dynamicField name="*_ds" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_dd" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="false" docValues="true"/>
<dynamicField name="*_dds" type="double" indexed="true" stored="false" multiValued="true" docValues="true"/>
<dynamicField name="*_dds" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="false" multiValued="true" docValues="true"/>
<dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
<dynamicField name="*_dts" type="date" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_dtd" type="date" indexed="true" stored="false" docValues="true"/>
@ -228,15 +228,15 @@
<!-- docvalues and stored (S suffix) -->
<dynamicField name="*_idS" type="${solr.tests.intClass:pint}" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_idsS" type="int" indexed="true" stored="true" multiValued="true" docValues="true"/>
<dynamicField name="*_idsS" type="${solr.tests.intClass:pint}" indexed="true" stored="true" multiValued="true" docValues="true"/>
<dynamicField name="*_sdS" type="string" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_sdsS" type="string" indexed="true" stored="true" multiValued="true" docValues="true"/>
<dynamicField name="*_ldS" type="${solr.tests.longClass:plong}" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_ldsS" type="long" indexed="true" stored="true" multiValued="true" docValues="true"/>
<dynamicField name="*_ldsS" type="${solr.tests.longClass:plong}" indexed="true" stored="true" multiValued="true" docValues="true"/>
<dynamicField name="*_fdS" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_fdsS" type="float" indexed="true" stored="true" multiValued="true" docValues="true"/>
<dynamicField name="*_fdsS" type="${solr.tests.floatClass:pfloat}" indexed="true" stored="true" multiValued="true" docValues="true"/>
<dynamicField name="*_ddS" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_ddsS" type="double" indexed="true" stored="true" multiValued="true" docValues="true"/>
<dynamicField name="*_ddsS" type="${solr.tests.doubleClass:pdouble}" indexed="true" stored="true" multiValued="true" docValues="true"/>
<dynamicField name="*_dtdS" type="date" indexed="true" stored="true" docValues="true"/>
<dynamicField name="*_dtdsS" type="date" indexed="true" stored="true" multiValued="true" docValues="true"/>
@ -396,10 +396,10 @@
<fieldType name="tdouble" class="solr.TrieDoubleField" precisionStep="8" positionIncrementGap="0"/>
<!-- Point Fields -->
<fieldType name="pint" class="solr.IntPointField"/>
<fieldType name="plong" class="solr.LongPointField"/>
<fieldType name="pdouble" class="solr.DoublePointField"/>
<fieldType name="pfloat" class="solr.FloatPointField"/>
<fieldType name="pint" class="solr.IntPointField" docValues="true"/>
<fieldType name="plong" class="solr.LongPointField" docValues="true"/>
<fieldType name="pdouble" class="solr.DoublePointField" docValues="true"/>
<fieldType name="pfloat" class="solr.FloatPointField" docValues="true"/>
<!-- The format for this date field is of the form 1995-12-31T23:59:59Z, and

View File

@ -35,6 +35,8 @@
<arr name="typeClass">
<str>solr.TrieIntField</str>
<str>solr.TrieLongField</str>
<str>solr.IntPointField</str>
<str>solr.LongPointField</str>
</arr>
</processor>
<processor class="solr.MinFieldValueUpdateProcessorFactory">

View File

@ -76,8 +76,8 @@ public class TestDistributedSearch extends BaseDistributedSearchTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
String t1="a_t";
String i1="a_i1";
String nint = "n_i";
String i1 = pickRandom("a_i1", "a_i_p", "a_i_ni_p");
String nint = pickRandom("n_i", "n_is_p", "n_is_ni_p");
String tint = "n_ti";
String tlong = "other_tl1";
String tdate_a = "a_n_tdt";
@ -187,7 +187,8 @@ public class TestDistributedSearch extends BaseDistributedSearchTestCase {
// these queries should be exactly ordered and scores should exactly match
query("q","*:*", "sort",i1+" desc");
query("q","*:*", "sort","{!func}testfunc(add("+i1+",5))"+" desc");
query("q","*:*", "sort",i1+" asc");
query("q",i1 + "[* TO *]", "sort",i1+" asc");
query("q","*:*", "sort",i1+" asc, id desc");
query("q","*:*", "sort",i1+" desc", "fl","*,score");
query("q","*:*", "sort","n_tl1 asc", "fl","*,score");
query("q","*:*", "sort","n_tl1 desc");
@ -422,6 +423,7 @@ public class TestDistributedSearch extends BaseDistributedSearchTestCase {
query("q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", "stats_dt");
query("q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", i1);
query("q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", nint);
handle.put("stddev", FUZZY);
handle.put("sumOfSquares", FUZZY);

View File

@ -50,6 +50,7 @@ import org.noggit.ObjectBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SolrTestCaseJ4.SuppressPointFields(bugUrl="https://issues.apache.org/jira/browse/SOLR-9992")
public class TestGroupingSearch extends SolrTestCaseJ4 {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

View File

@ -74,6 +74,12 @@ public class TestRandomDVFaceting extends SolrTestCaseJ4 {
types.add(new FldType("small2_i",ZERO_ONE, new IRange(0,5+indexSize/3)));
types.add(new FldType("small2_is",ZERO_TWO, new IRange(0,5+indexSize/3)));
types.add(new FldType("small3_is",new IRange(0,25), new IRange(0,100)));
types.add(new FldType("foo_fs", new IRange(0,25), new FVal(0,indexSize)));
types.add(new FldType("foo_f", ZERO_ONE, new FVal(0,indexSize)));
types.add(new FldType("foo_ds", new IRange(0,25), new FVal(0,indexSize)));
types.add(new FldType("foo_d", ZERO_ONE, new FVal(0,indexSize)));
types.add(new FldType("foo_ls", new IRange(0,25), new IRange(0,indexSize)));
types.add(new FldType("missing_i",new IRange(0,0), new IRange(0,100)));
types.add(new FldType("missing_is",new IRange(0,0), new IRange(0,100)));
@ -156,10 +162,9 @@ public class TestRandomDVFaceting extends SolrTestCaseJ4 {
Random rand = random();
boolean validate = validateResponses;
ModifiableSolrParams params = params("facet","true", "wt","json", "indent","true", "omitHeader","true");
params.add("q","*:*", "rows","0"); // TODO: select subsets
params.add("q","*:*"); // TODO: select subsets
params.add("rows","0");
SchemaField sf = req.getSchema().getField(ftype.fname);
boolean multiValued = sf.getType().multiValuedFieldCache();
boolean indexed = sf.indexed();
@ -173,12 +178,16 @@ public class TestRandomDVFaceting extends SolrTestCaseJ4 {
params.add("facet.offset", Integer.toString(offset));
}
int limit = 100;
if (rand.nextInt(100) < 20) {
if (rand.nextBoolean()) {
limit = rand.nextInt(100) < 10 ? rand.nextInt(indexSize/2+1) : rand.nextInt(indexSize*2);
if(rarely()) {
params.add("facet.limit", "-1");
} else {
int limit = 100;
if (rand.nextBoolean()) {
limit = rand.nextInt(100) < 10 ? rand.nextInt(indexSize/2+1) : rand.nextInt(indexSize*2);
}
params.add("facet.limit", Integer.toString(limit));
}
params.add("facet.limit", Integer.toString(limit));
}
// the following two situations cannot work for unindexed single-valued numerics:
@ -234,7 +243,10 @@ public class TestRandomDVFaceting extends SolrTestCaseJ4 {
responses.add(strResponse);
}
// If there is a PointField option for this test, also test it
if (h.getCore().getLatestSchema().getFieldOrNull(facet_field + "_p") != null) {
// Don't check points if facet.mincount=0
if (h.getCore().getLatestSchema().getFieldOrNull(facet_field + "_p") != null
&& params.get("facet.mincount") != null
&& params.getInt("facet.mincount").intValue() > 0) {
params.set("facet.field", "{!key="+facet_field+"}"+facet_field+"_p");
String strResponse = h.query(req(params));
responses.add(strResponse);

View File

@ -22,6 +22,7 @@ import java.util.Map;
import java.util.Set;
import java.util.Random;
import org.apache.solr.SolrTestCaseJ4.SuppressPointFields;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.response.QueryResponse;
@ -31,7 +32,8 @@ import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ShardParams;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.response.SolrQueryResponse;
// This test uses grouping requests, which doesn't work yet with PointFields
@SuppressPointFields(bugUrl="https://issues.apache.org/jira/browse/SOLR-9992")
class SegmentTerminateEarlyTestState {
final String keyField = "id";

View File

@ -22,6 +22,7 @@ import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.util.TestUtil;
import org.apache.solr.SolrTestCaseJ4.SuppressPointFields;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.schema.SchemaRequest.Field;
@ -41,6 +42,7 @@ import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressPointFields(bugUrl="https://issues.apache.org/jira/browse/SOLR-9992")
public class TestSegmentSorting extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

View File

@ -84,7 +84,8 @@ public class StatsComponentTest extends AbstractSolrTestCase {
"stats_i","stats_l","stats_f","stats_d",
"stats_ti","stats_tl","stats_tf","stats_td",
"stats_ti_dv","stats_tl_dv","stats_tf_dv","stats_td_dv",
"stats_ti_ni_dv","stats_tl_ni_dv","stats_tf_ni_dv","stats_td_ni_dv"
"stats_ti_ni_dv","stats_tl_ni_dv","stats_tf_ni_dv","stats_td_ni_dv",
"stats_i_ni_p","stats_l_ni_p","stats_f_ni_p","stats_d_ni_p",
}) {
// all of our checks should work with all of these params
@ -111,7 +112,9 @@ public class StatsComponentTest extends AbstractSolrTestCase {
for (String f : new String[] {"stats_ii",
"stats_tis","stats_tfs","stats_tls","stats_tds", // trie fields
"stats_tis_dv","stats_tfs_dv","stats_tls_dv","stats_tds_dv", // Doc Values
"stats_tis_ni_dv","stats_tfs_ni_dv","stats_tls_ni_dv","stats_tds_ni_dv" // Doc Values Not indexed
"stats_tis_ni_dv","stats_tfs_ni_dv","stats_tls_ni_dv","stats_tds_ni_dv", // Doc Values Not indexed
"stats_is_p", "stats_fs_p", "stats_ls_p", "stats_ds_p", // Point Fields
"stats_is_ni_p","stats_fs_ni_p","stats_ls_ni_p" // Point Doc Values Not indexed
}) {
doTestMVFieldStatisticsResult(f);

View File

@ -27,6 +27,8 @@ import java.util.regex.Pattern;
*
*
**/
// TermsComponent not currently supported for PointFields
@SolrTestCaseJ4.SuppressPointFields
public class TermsComponentTest extends SolrTestCaseJ4 {
@BeforeClass

View File

@ -17,6 +17,9 @@
package org.apache.solr.request;
import static junit.framework.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import org.apache.solr.request.SimpleFacets.FacetMethod;
import org.apache.solr.schema.BoolField;
import org.apache.solr.schema.IntPointField;
@ -25,8 +28,6 @@ import org.apache.solr.schema.StrField;
import org.apache.solr.schema.TrieIntField;
import org.junit.Test;
import static junit.framework.Assert.assertEquals;
public class TestFacetMethods {
// TODO - make these public in FieldProperties?
@ -214,6 +215,12 @@ public class TestFacetMethods {
assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.ENUM, 0));
assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.FC, 0));
assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.FCS, 0));
field = new SchemaField("fooMV", new IntPointField(), 0x00000200, "0"); //MultiValued
assertTrue(field.multiValued());
assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, null, 0));
assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.ENUM, 0));
assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.FC, 0));
assertEquals(SimpleFacets.FacetMethod.FCS, SimpleFacets.selectFacetMethod(field, FacetMethod.FCS, 0));
}
}

View File

@ -105,10 +105,11 @@ public class TestIntervalFaceting extends SolrTestCaseJ4 {
assertU(adoc("id", "10"));
assertU(commit());
int i = 11;
while (getNumberOfReaders() < 2 && i < 10) {
while (getNumberOfReaders() < 2 && i < 20) {
//try to get more than one segment
assertU(adoc("id", String.valueOf(i), "test_i_dv", String.valueOf(i)));
assertU(commit());
i++;
}
if (getNumberOfReaders() < 2) {
// It is OK if for some seeds we fall into this case (for example, TieredMergePolicy with
@ -244,9 +245,11 @@ public class TestIntervalFaceting extends SolrTestCaseJ4 {
// All field values will be a number between 0 and cardinality
int cardinality = 100000;
// Fields to use for interval faceting
String[] fields = new String[]{"test_s_dv", "test_i_dv", "test_l_dv", "test_f_dv", "test_d_dv",
String[] fields = new String[]{
"test_s_dv", "test_i_dv", "test_l_dv", "test_f_dv", "test_d_dv",
"test_ss_dv", "test_is_dv", "test_fs_dv", "test_ls_dv", "test_ds_dv", "test_s", "test_i",
"test_l", "test_f", "test_d", "test_ss", "test_is", "test_fs", "test_ls", "test_ds"};
"test_l", "test_f", "test_d", "test_ss", "test_is", "test_fs", "test_ls", "test_ds",
"test_i_p", "test_is_p", "test_l_p", "test_ls_p", "test_f_p", "test_fs_p", "test_d_p", "test_ds_p"};
for (int i = 0; i < atLeast(500); i++) {
if (random().nextInt(50) == 0) {
//have some empty docs
@ -743,10 +746,11 @@ public class TestIntervalFaceting extends SolrTestCaseJ4 {
assertU(commit());
int i = 12;
while (getNumberOfReaders() < 2 && i < 10) {
while (getNumberOfReaders() < 2 && i < 20) {
//try to get more than one segment
assertU(adoc("id", String.valueOf(i), "test_s_dv", String.valueOf(i)));
assertU(commit());
i++;
}
if (getNumberOfReaders() < 2) {
// It is OK if for some seeds we fall into this case (for example, TieredMergePolicy with

View File

@ -23,15 +23,11 @@ import java.util.TreeSet;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.SolrException;
import org.apache.solr.schema.DoublePointField;
import org.apache.solr.schema.IntPointField;
import org.apache.solr.schema.PointField;
import org.apache.solr.schema.SchemaField;
import org.junit.After;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
/**
@ -60,8 +56,7 @@ public class TestPointFields extends SolrTestCaseJ4 {
doTestIntPointFieldExactQuery("number_p_i_mv", false);
doTestIntPointFieldExactQuery("number_p_i_ni_dv", false);
doTestIntPointFieldExactQuery("number_p_i_ni_ns_dv", false);
// uncomment once MultiValued docValues are supported in PointFields
// doTestIntPointFieldExactQuery("number_p_i_ni_mv_dv", false);
doTestIntPointFieldExactQuery("number_p_i_ni_mv_dv", false);
}
@Test
@ -104,110 +99,42 @@ public class TestPointFields extends SolrTestCaseJ4 {
public void testIntPointStats() throws Exception {
testPointStats("number_p_i", "number_p_i_dv", new String[]{"0", "1", "2", "3", "4", "5", "6", "7", "8", "9"},
0D, 9D, "10", "1", 0D);
testPointStats("number_p_i", "number_p_i_mv_dv", new String[]{"0", "1", "2", "3", "4", "5", "6", "7", "8", "9"},
0D, 9D, "10", "1", 0D);
}
@Test
public void testIntPointFieldMultiValuedExactQuery() throws Exception {
testPointFieldMultiValuedExactQuery("number_p_i_mv", getSequentialStringArrayWithInts(20));
testPointFieldMultiValuedExactQuery("number_p_i_ni_mv_dv", getSequentialStringArrayWithInts(20));
}
@Test
public void testIntPointFieldMultiValuedReturn() throws Exception {
testPointFieldMultiValuedReturn("number_p_i_mv", "int", getSequentialStringArrayWithInts(20));
testPointFieldMultiValuedReturn("number_p_i_ni_mv_dv", "int", getSequentialStringArrayWithInts(20));
testPointFieldMultiValuedReturn("number_p_i_dv_ns_mv", "int", getSequentialStringArrayWithInts(20));
}
@Test
public void testIntPointFieldMultiValuedRangeQuery() throws Exception {
testPointFieldMultiValuedRangeQuery("number_p_i_mv", "int", getSequentialStringArrayWithInts(20));
testPointFieldMultiValuedRangeQuery("number_p_i_ni_mv_dv", "int", getSequentialStringArrayWithInts(20));
}
//TODO MV SORT?
@Test
@Ignore("Enable once MultiValued docValues are supported in PointFields")
public void testIntPointFieldMultiValuedFacetField() throws Exception {
testPointFieldMultiValuedFacetField("number_p_i_mv", "number_p_i_mv_dv", getSequentialStringArrayWithInts(20));
testPointFieldMultiValuedFacetField("number_p_i_mv", "number_p_i_mv_dv", getRandomStringArrayWithInts(20, false));
}
@Test
@Ignore("Enable once MultiValued docValues are supported in PointFields")
public void testIntPointFieldMultiValuedRangeFacet() throws Exception {
String docValuesField = "number_p_i_mv_dv";
String nonDocValuesField = "number_p_i_mv";
for (int i = 0; i < 10; i++) {
assertU(adoc("id", String.valueOf(i), docValuesField, String.valueOf(i), docValuesField, String.valueOf(i + 10),
nonDocValuesField, String.valueOf(i), nonDocValuesField, String.valueOf(i + 10)));
}
assertU(commit());
assertTrue(h.getCore().getLatestSchema().getField(docValuesField).hasDocValues());
assertTrue(h.getCore().getLatestSchema().getField(docValuesField).getType() instanceof IntPointField);
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", docValuesField, "facet.range.start", "-10", "facet.range.end", "20", "facet.range.gap", "2"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='2'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='4'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='6'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='8'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='10'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='12'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='14'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='16'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='18'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='-10'][.='0']");
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", docValuesField, "facet.range.start", "-10", "facet.range.end", "20", "facet.range.gap", "2", "facet.range.method", "dv"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='2'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='4'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='6'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='8'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='10'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='12'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='14'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='16'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='18'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='-10'][.='0']");
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", docValuesField, "facet.range.start", "0", "facet.range.end", "20", "facet.range.gap", "100"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='0'][.='10']");
assertFalse(h.getCore().getLatestSchema().getField(nonDocValuesField).hasDocValues());
assertTrue(h.getCore().getLatestSchema().getField(nonDocValuesField).getType() instanceof IntPointField);
// Range Faceting with method = filter should work
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", nonDocValuesField, "facet.range.start", "-10", "facet.range.end", "20", "facet.range.gap", "2", "facet.range.method", "filter"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='2'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='4'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='6'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='8'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='10'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='12'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='14'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='16'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='18'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='-10'][.='0']");
// this should actually use filter method instead of dv
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", nonDocValuesField, "facet.range.start", "-10", "facet.range.end", "20", "facet.range.gap", "2", "facet.range.method", "dv"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='2'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='4'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='6'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='8'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='10'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='12'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='14'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='16'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='18'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='-10'][.='0']");
doTestIntPointFieldMultiValuedRangeFacet("number_p_i_mv_dv", "number_p_i_mv");
}
@Test
@Ignore("Enable once MultiValued docValues are supported in PointFields")
public void testIntPointMultiValuedFunctionQuery() throws Exception {
testPointMultiValuedFunctionQuery("number_p_i_mv", "number_p_i_mv_dv", "int", getSequentialStringArrayWithInts(20));
}
@ -222,6 +149,16 @@ public class TestPointFields extends SolrTestCaseJ4 {
testIntPointFieldsAtomicUpdates("number_p_i_dv_ns", "int");
}
@Test
public void testMultiValuedIntPointFieldsAtomicUpdates() throws Exception {
if (!Boolean.getBoolean("enable.update.log")) {
return;
}
testMultiValuedIntPointFieldsAtomicUpdates("number_p_i_mv", "int");
testMultiValuedIntPointFieldsAtomicUpdates("number_p_i_ni_mv_dv", "int");
testMultiValuedIntPointFieldsAtomicUpdates("number_p_i_dv_ns_mv", "int");
}
@Test
public void testIntPointSetQuery() throws Exception {
doTestSetQueries("number_p_i", getRandomStringArrayWithInts(10, false), false);
@ -238,8 +175,7 @@ public class TestPointFields extends SolrTestCaseJ4 {
doTestFloatPointFieldExactQuery("number_p_d_mv");
doTestFloatPointFieldExactQuery("number_p_d_ni_dv");
doTestFloatPointFieldExactQuery("number_p_d_ni_ns_dv");
// TODO enable once MuultiValued docValues are supported with PointFields
// doTestFloatPointFieldExactQuery("number_p_d_ni_mv_dv");
doTestFloatPointFieldExactQuery("number_p_d_ni_mv_dv");
}
@Test
@ -292,112 +228,41 @@ public class TestPointFields extends SolrTestCaseJ4 {
public void testDoublePointStats() throws Exception {
testPointStats("number_p_d", "number_p_d_dv", new String[]{"-10.0", "1.1", "2.2", "3.3", "4.4", "5.5", "6.6", "7.7", "8.8", "9.9"},
-10.0D, 9.9D, "10", "1", 1E-10D);
testPointStats("number_p_d_mv", "number_p_d_mv_dv", new String[]{"-10.0", "1.1", "2.2", "3.3", "4.4", "5.5", "6.6", "7.7", "8.8", "9.9"},
-10.0D, 9.9D, "10", "1", 1E-10D);
}
@Test
public void testDoublePointFieldMultiValuedExactQuery() throws Exception {
testPointFieldMultiValuedExactQuery("number_p_d_mv", getRandomStringArrayWithDoubles(20, false));
testPointFieldMultiValuedExactQuery("number_p_d_ni_mv_dv", getRandomStringArrayWithDoubles(20, false));
}
@Test
public void testDoublePointFieldMultiValuedReturn() throws Exception {
testPointFieldMultiValuedReturn("number_p_d_mv", "double", getSequentialStringArrayWithDoubles(20));
testPointFieldMultiValuedReturn("number_p_d_ni_mv_dv", "double", getSequentialStringArrayWithDoubles(20));
testPointFieldMultiValuedReturn("number_p_d_dv_ns_mv", "double", getSequentialStringArrayWithDoubles(20));
}
@Test
public void testDoublePointFieldMultiValuedRangeQuery() throws Exception {
testPointFieldMultiValuedRangeQuery("number_p_d_mv", "double", getSequentialStringArrayWithDoubles(20));
testPointFieldMultiValuedRangeQuery("number_p_d_ni_mv_dv", "double", getSequentialStringArrayWithDoubles(20));
}
@Test
@Ignore("Enable once MultiValued docValues are supported in PointFields")
public void testDoublePointFieldMultiValuedFacetField() throws Exception {
testPointFieldMultiValuedFacetField("number_p_d_mv", "number_p_d_mv_dv", getSequentialStringArrayWithDoubles(20));
testPointFieldMultiValuedFacetField("number_p_d_mv", "number_p_d_mv_dv", getRandomStringArrayWithDoubles(20, false));
}
@Test
@Ignore("Enable once MultiValued docValues are supported in PointFields")
public void testDoublePointFieldMultiValuedRangeFacet() throws Exception {
String docValuesField = "number_p_d_mv_dv";
String nonDocValuesField = "number_p_d_mv";
for (int i = 0; i < 10; i++) {
assertU(adoc("id", String.valueOf(i), docValuesField, String.valueOf(i), docValuesField, String.valueOf(i + 10),
nonDocValuesField, String.valueOf(i), nonDocValuesField, String.valueOf(i + 10)));
}
assertU(commit());
assertTrue(h.getCore().getLatestSchema().getField(docValuesField).hasDocValues());
assertTrue(h.getCore().getLatestSchema().getField(docValuesField).multiValued());
assertTrue(h.getCore().getLatestSchema().getField(docValuesField).getType() instanceof DoublePointField);
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", docValuesField, "facet.range.start", "-10", "facet.range.end", "20", "facet.range.gap", "2"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='0.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='2.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='4.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='6.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='8.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='10.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='12.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='14.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='16.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='18.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='-10.0'][.='0']");
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", docValuesField, "facet.range.start", "-10", "facet.range.end", "20", "facet.range.gap", "2", "facet.range.method", "dv"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='0.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='2.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='4.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='6.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='8.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='10.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='12.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='14.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='16.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='18.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='-10.0'][.='0']");
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", docValuesField, "facet.range.start", "0", "facet.range.end", "20", "facet.range.gap", "100"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='0.0'][.='10']");
assertFalse(h.getCore().getLatestSchema().getField(nonDocValuesField).hasDocValues());
assertTrue(h.getCore().getLatestSchema().getField(nonDocValuesField).multiValued());
assertTrue(h.getCore().getLatestSchema().getField(nonDocValuesField).getType() instanceof DoublePointField);
// Range Faceting with method = filter should work
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", nonDocValuesField, "facet.range.start", "-10", "facet.range.end", "20", "facet.range.gap", "2", "facet.range.method", "filter"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='0.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='2.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='4.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='6.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='8.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='10.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='12.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='14.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='16.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='18.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='-10.0'][.='0']");
// this should actually use filter method instead of dv
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", nonDocValuesField, "facet.range.start", "-10", "facet.range.end", "20", "facet.range.gap", "2", "facet.range.method", "dv"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='0.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='2.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='4.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='6.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='8.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='10.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='12.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='14.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='16.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='18.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='-10.0'][.='0']");
doTestDoublePointFieldMultiValuedRangeFacet("number_p_d_mv_dv", "number_p_d_mv");
}
@Test
@Ignore("Enable once MultiValued docValues are supported in PointFields")
public void testDoublePointMultiValuedFunctionQuery() throws Exception {
testPointMultiValuedFunctionQuery("number_p_d_mv", "number_p_d_mv_dv", "double", getSequentialStringArrayWithDoubles(20));
testPointMultiValuedFunctionQuery("number_p_d_mv", "number_p_d_mv_dv", "double", getRandomStringArrayWithFloats(20, true));
@ -413,6 +278,17 @@ public class TestPointFields extends SolrTestCaseJ4 {
doTestFloatPointFieldsAtomicUpdates("number_p_d_dv_ns", "double");
}
@Test
public void testMultiValuedDoublePointFieldsAtomicUpdates() throws Exception {
if (!Boolean.getBoolean("enable.update.log")) {
return;
}
testMultiValuedFloatPointFieldsAtomicUpdates("number_p_d_mv", "double");
testMultiValuedFloatPointFieldsAtomicUpdates("number_p_d_ni_mv_dv", "double");
testMultiValuedFloatPointFieldsAtomicUpdates("number_p_d_dv_ns_mv", "double");
}
private void doTestFloatPointFieldsAtomicUpdates(String field, String type) throws Exception {
assertU(adoc(sdoc("id", "1", field, "1.1234")));
assertU(commit());
@ -454,7 +330,6 @@ public class TestPointFields extends SolrTestCaseJ4 {
}
// Float
@Test
public void testFloatPointFieldExactQuery() throws Exception {
@ -462,7 +337,7 @@ public class TestPointFields extends SolrTestCaseJ4 {
doTestFloatPointFieldExactQuery("number_p_f_mv");
doTestFloatPointFieldExactQuery("number_p_f_ni_dv");
doTestFloatPointFieldExactQuery("number_p_f_ni_ns_dv");
// doTestFloatPointFieldExactQuery("number_p_f_ni_mv_dv");
doTestFloatPointFieldExactQuery("number_p_f_ni_mv_dv");
}
@Test
@ -515,23 +390,47 @@ public class TestPointFields extends SolrTestCaseJ4 {
public void testFloatPointStats() throws Exception {
testPointStats("number_p_f", "number_p_f_dv", new String[]{"-10.0", "1.1", "2.2", "3.3", "4.4", "5.5", "6.6", "7.7", "8.8", "9.9"},
-10D, 9.9D, "10", "1", 1E-6D);
testPointStats("number_p_f_mv", "number_p_f_mv_dv", new String[]{"-10.0", "1.1", "2.2", "3.3", "4.4", "5.5", "6.6", "7.7", "8.8", "9.9"},
-10D, 9.9D, "10", "1", 1E-6D);
}
@Test
public void testFloatPointFieldMultiValuedExactQuery() throws Exception {
testPointFieldMultiValuedExactQuery("number_p_f_mv", getRandomStringArrayWithFloats(20, false));
testPointFieldMultiValuedExactQuery("number_p_f_ni_mv_dv", getRandomStringArrayWithFloats(20, false));
}
@Test
public void testFloatPointFieldMultiValuedReturn() throws Exception {
testPointFieldMultiValuedReturn("number_p_f_mv", "float", getSequentialStringArrayWithDoubles(20));
testPointFieldMultiValuedReturn("number_p_f_ni_mv_dv", "float", getSequentialStringArrayWithDoubles(20));
testPointFieldMultiValuedReturn("number_p_f_dv_ns_mv", "float", getSequentialStringArrayWithDoubles(20));
}
@Test
public void testFloatPointFieldMultiValuedRangeQuery() throws Exception {
testPointFieldMultiValuedRangeQuery("number_p_f_mv", "float", getSequentialStringArrayWithDoubles(20));
testPointFieldMultiValuedRangeQuery("number_p_f_ni_mv_dv", "float", getSequentialStringArrayWithDoubles(20));
}
@Test
public void testFloatPointFieldMultiValuedRangeFacet() throws Exception {
doTestDoublePointFieldMultiValuedRangeFacet("number_p_f_mv_dv", "number_p_f_mv");
}
@Test
public void testFloatPointFieldMultiValuedFacetField() throws Exception {
testPointFieldMultiValuedFacetField("number_p_f_mv", "number_p_f_mv_dv", getSequentialStringArrayWithDoubles(20));
testPointFieldMultiValuedFacetField("number_p_f_mv", "number_p_f_mv_dv", getRandomStringArrayWithFloats(20, false));
}
@Test
public void testFloatPointMultiValuedFunctionQuery() throws Exception {
testPointMultiValuedFunctionQuery("number_p_f_mv", "number_p_f_mv_dv", "float", getSequentialStringArrayWithDoubles(20));
testPointMultiValuedFunctionQuery("number_p_f_mv", "number_p_f_mv_dv", "float", getRandomStringArrayWithFloats(20, true));
}
@Test
public void testFloatPointFieldsAtomicUpdates() throws Exception {
if (!Boolean.getBoolean("enable.update.log")) {
@ -542,6 +441,15 @@ public class TestPointFields extends SolrTestCaseJ4 {
doTestFloatPointFieldsAtomicUpdates("number_p_f_dv_ns", "float");
}
@Test
public void testMultiValuedFloatePointFieldsAtomicUpdates() throws Exception {
if (!Boolean.getBoolean("enable.update.log")) {
return;
}
testMultiValuedFloatPointFieldsAtomicUpdates("number_p_f_mv", "float");
testMultiValuedFloatPointFieldsAtomicUpdates("number_p_f_ni_mv_dv", "float");
testMultiValuedFloatPointFieldsAtomicUpdates("number_p_f_dv_ns_mv", "float");
}
@Test
public void testFloatPointSetQuery() throws Exception {
@ -558,7 +466,7 @@ public class TestPointFields extends SolrTestCaseJ4 {
doTestIntPointFieldExactQuery("number_p_l_mv", true);
doTestIntPointFieldExactQuery("number_p_l_ni_dv", true);
doTestIntPointFieldExactQuery("number_p_l_ni_ns_dv", true);
// doTestIntPointFieldExactQuery("number_p_i_ni_mv_dv", true);
doTestIntPointFieldExactQuery("number_p_l_ni_mv_dv", true);
}
@Test
@ -604,21 +512,43 @@ public class TestPointFields extends SolrTestCaseJ4 {
public void testLongPointStats() throws Exception {
testPointStats("number_p_l", "number_p_l_dv", new String[]{"0", "1", "2", "3", "4", "5", "6", "7", "8", "9"},
0D, 9D, "10", "1", 0D);
testPointStats("number_p_l_mv", "number_p_l_mv_dv", new String[]{"0", "1", "2", "3", "4", "5", "6", "7", "8", "9"},
0D, 9D, "10", "1", 0D);
}
@Test
public void testLongPointFieldMultiValuedExactQuery() throws Exception {
testPointFieldMultiValuedExactQuery("number_p_l_mv", getSequentialStringArrayWithInts(20));
testPointFieldMultiValuedExactQuery("number_p_l_ni_mv_dv", getSequentialStringArrayWithInts(20));
}
@Test
public void testLongPointFieldMultiValuedReturn() throws Exception {
testPointFieldMultiValuedReturn("number_p_l_mv", "long", getSequentialStringArrayWithInts(20));
testPointFieldMultiValuedReturn("number_p_l_ni_mv_dv", "long", getSequentialStringArrayWithInts(20));
testPointFieldMultiValuedReturn("number_p_l_dv_ns_mv", "long", getSequentialStringArrayWithInts(20));
}
@Test
public void testLongPointFieldMultiValuedRangeQuery() throws Exception {
testPointFieldMultiValuedRangeQuery("number_p_l_mv", "long", getSequentialStringArrayWithInts(20));
testPointFieldMultiValuedRangeQuery("number_p_l_ni_mv_dv", "long", getSequentialStringArrayWithInts(20));
}
@Test
public void testLongPointFieldMultiValuedFacetField() throws Exception {
testPointFieldMultiValuedFacetField("number_p_l_mv", "number_p_l_mv_dv", getSequentialStringArrayWithInts(20));
testPointFieldMultiValuedFacetField("number_p_l_mv", "number_p_l_mv_dv", getRandomStringArrayWithLongs(20, false));
}
@Test
public void testLongPointFieldMultiValuedRangeFacet() throws Exception {
doTestIntPointFieldMultiValuedRangeFacet("number_p_l_mv_dv", "number_p_l_mv");
}
@Test
public void testLongPointMultiValuedFunctionQuery() throws Exception {
testPointMultiValuedFunctionQuery("number_p_l_mv", "number_p_l_mv_dv", "long", getSequentialStringArrayWithInts(20));
}
@Test
@ -631,6 +561,16 @@ public class TestPointFields extends SolrTestCaseJ4 {
testIntPointFieldsAtomicUpdates("number_p_l_dv_ns", "long");
}
@Test
public void testMultiValuedLongPointFieldsAtomicUpdates() throws Exception {
if (!Boolean.getBoolean("enable.update.log")) {
return;
}
testMultiValuedIntPointFieldsAtomicUpdates("number_p_l_mv", "long");
testMultiValuedIntPointFieldsAtomicUpdates("number_p_l_ni_mv_dv", "long");
testMultiValuedIntPointFieldsAtomicUpdates("number_p_l_dv_ns_mv", "long");
}
@Test
public void testLongPointSetQuery() throws Exception {
doTestSetQueries("number_p_l", getRandomStringArrayWithLongs(10, false), false);
@ -910,6 +850,14 @@ public class TestPointFields extends SolrTestCaseJ4 {
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + docValuesField +"']/int[@name='" + numbers[2] + "'][.='1']",
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + docValuesField +"']/int[@name='" + numbers[3] + "'][.='1']");
// assertU(commit());
// assertQ(req("q", "id:0", "fl", "id, " + docValuesField, "facet", "true", "facet.field", docValuesField, "facet.mincount", "0"),
// "//*[@numFound='1']",
// "//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + docValuesField +"']/int[@name='" + numbers[0] + "'][.='1']",
// "//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + docValuesField +"']/int[@name='" + numbers[1] + "'][.='0']",
// "//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + docValuesField +"']/int[@name='" + numbers[2] + "'][.='0']",
// "count(//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + docValuesField +"']/int))==10");
assertFalse(h.getCore().getLatestSchema().getField(nonDocValuesField).hasDocValues());
assertTrue(h.getCore().getLatestSchema().getField(nonDocValuesField).getType() instanceof PointField);
assertQEx("Expecting Exception",
@ -1057,7 +1005,25 @@ public class TestPointFields extends SolrTestCaseJ4 {
for (int i=0; i < 10; i++) {
assertU(adoc("id", String.valueOf(i), fieldName, numbers[i], fieldName, numbers[i+10]));
}
// Check using RTG before commit
if (Boolean.getBoolean("enable.update.log")) {
for (int i = 0; i < 10; i++) {
assertQ(req("qt", "/get", "id", String.valueOf(i)),
"//doc/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[i] + "']",
"//doc/arr[@name='" + fieldName + "']/" + type + "[2][.='" + numbers[i+10] + "']",
"count(//doc/arr[@name='" + fieldName + "']/" + type + ")=2");
}
}
// Check using RTG after commit
assertU(commit());
if (Boolean.getBoolean("enable.update.log")) {
for (int i = 0; i < 10; i++) {
assertQ(req("qt", "/get", "id", String.valueOf(i)),
"//doc/arr[@name='" + fieldName + "']/" + type + "[1][.='" + numbers[i] + "']",
"//doc/arr[@name='" + fieldName + "']/" + type + "[2][.='" + numbers[i+10] + "']",
"count(//doc/arr[@name='" + fieldName + "']/" + type + ")=2");
}
}
String[] expected = new String[11];
String[] expected2 = new String[11];
expected[0] = "//*[@numFound='10']";
@ -1137,6 +1103,9 @@ public class TestPointFields extends SolrTestCaseJ4 {
for (int i = 0; i < 10; i++) {
assertU(adoc("id", String.valueOf(i), dvFieldName, numbers[i], dvFieldName, numbers[i + 10],
nonDocValuesField, numbers[i], nonDocValuesField, numbers[i + 10]));
if (rarely()) {
assertU(commit());
}
}
assertU(commit());
@ -1159,12 +1128,85 @@ public class TestPointFields extends SolrTestCaseJ4 {
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int[@name='" + numbers[3] + "'][.='1']",
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int[@name='" + numbers[10] + "'][.='1']");
assertU(adoc("id", "10", dvFieldName, numbers[1], nonDocValuesField, numbers[1], dvFieldName, numbers[1], nonDocValuesField, numbers[1]));
assertU(commit());
assertQ(req("q", "*:*", "fl", "id, " + dvFieldName, "facet", "true", "facet.field", dvFieldName, "facet.missing", "true"),
"//*[@numFound='11']",
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int[@name='" + numbers[1] + "'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int[@name='" + numbers[2] + "'][.='1']",
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int[@name='" + numbers[3] + "'][.='1']",
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int[@name='" + numbers[10] + "'][.='1']",
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int[not(@name)][.='0']"
);
assertU(adoc("id", "10")); // add missing values
assertU(commit());
assertQ(req("q", "*:*", "fl", "id, " + dvFieldName, "facet", "true", "facet.field", dvFieldName, "facet.missing", "true"),
"//*[@numFound='11']",
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int[@name='" + numbers[1] + "'][.='1']",
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int[@name='" + numbers[2] + "'][.='1']",
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int[@name='" + numbers[3] + "'][.='1']",
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int[@name='" + numbers[10] + "'][.='1']",
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int[not(@name)][.='1']"
);
assertQ(req("q", "*:*", "fl", "id, " + dvFieldName, "facet", "true", "facet.field", dvFieldName, "facet.mincount", "3"),
"//*[@numFound='11']",
"count(//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int)=0");
assertQ(req("q", "id:0", "fl", "id, " + dvFieldName, "facet", "true", "facet.field", dvFieldName),
"//*[@numFound='1']",
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int[@name='" + numbers[0] + "'][.='1']",
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int[@name='" + numbers[10] + "'][.='1']",
"count(//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int)=2");
assertFalse(h.getCore().getLatestSchema().getField(nonDocValuesField).hasDocValues());
assertTrue(h.getCore().getLatestSchema().getField(nonDocValuesField).getType() instanceof PointField);
assertQEx("Expecting Exception",
"Can't facet on a PointField without docValues",
req("q", "*:*", "fl", "id, " + nonDocValuesField, "facet", "true", "facet.field", nonDocValuesField),
SolrException.ErrorCode.BAD_REQUEST);
clearIndex();
assertU(commit());
String smaller, larger;
try {
if (Long.parseLong(numbers[1]) < Long.parseLong(numbers[2])) {
smaller = numbers[1];
larger = numbers[2];
} else {
smaller = numbers[2];
larger = numbers[1];
}
} catch (NumberFormatException e) {
if (Double.valueOf(numbers[1]) < Double.valueOf(numbers[2])) {
smaller = numbers[1];
larger = numbers[2];
} else {
smaller = numbers[2];
larger = numbers[1];
}
}
assertU(adoc("id", "1", dvFieldName, smaller, dvFieldName, larger));
assertU(adoc("id", "2", dvFieldName, larger));
assertU(commit());
assertQ(req("q", "*:*", "fl", "id, " + dvFieldName, "facet", "true", "facet.field", dvFieldName),
"//*[@numFound='2']",
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int[@name='" + larger + "'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int[@name='" + smaller + "'][.='1']",
"count(//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int)=2");
assertQ(req("q", "*:*", "fl", "id, " + dvFieldName, "facet", "true", "facet.field", dvFieldName, "facet.sort", "index"),
"//*[@numFound='2']",
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int[@name='" + smaller +"'][.='1']",
"//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int[@name='"+ larger + "'][.='2']",
"count(//lst[@name='facet_counts']/lst[@name='facet_fields']/lst[@name='" + dvFieldName +"']/int)=2");
clearIndex();
assertU(commit());
}
private void testPointMultiValuedFunctionQuery(String nonDocValuesField, String docValuesField, String type, String[] numbers) throws Exception {
@ -1179,23 +1221,20 @@ public class TestPointFields extends SolrTestCaseJ4 {
assertTrue(h.getCore().getLatestSchema().getField(docValuesField).getType() instanceof PointField);
String function = "field(" + docValuesField + ", min)";
assertQ(req("q", "*:*", "fl", "id, " + function),
// assertQ(req("q", "*:*", "fl", "id, " + function),
// "//*[@numFound='10']",
// "//result/doc[1]/" + type + "[@name='" + function + "'][.='" + numbers[0] + "']",
// "//result/doc[2]/" + type + "[@name='" + function + "'][.='" + numbers[1] + "']",
// "//result/doc[3]/" + type + "[@name='" + function + "'][.='" + numbers[2] + "']",
// "//result/doc[10]/" + type + "[@name='" + function + "'][.='" + numbers[9] + "']");
assertQ(req("q", "*:*", "fl", "id, " + docValuesField, "sort", function + " desc"),
"//*[@numFound='10']",
"//result/doc[1]/" + type + "[@name='" + function + "'][.='" + numbers[0] + "']",
"//result/doc[2]/" + type + "[@name='" + function + "'][.='" + numbers[1] + "']",
"//result/doc[3]/" + type + "[@name='" + function + "'][.='" + numbers[2] + "']",
"//result/doc[10]/" + type + "[@name='" + function + "'][.='" + numbers[9] + "']");
// if (dvIsRandomAccessOrds(docValuesField)) {
// function = "field(" + docValuesField + ", max)";
// assertQ(req("q", "*:*", "fl", "id, " + function),
// "//*[@numFound='10']",
// "//result/doc[1]/int[@name='" + function + "'][.='10']",
// "//result/doc[2]/int[@name='" + function + "'][.='11']",
// "//result/doc[3]/int[@name='" + function + "'][.='12']",
// "//result/doc[10]/int[@name='" + function + "'][.='19']");
// }
"//result/doc[1]/str[@name='id'][.='9']",
"//result/doc[2]/str[@name='id'][.='8']",
"//result/doc[3]/str[@name='id'][.='7']",
"//result/doc[10]/str[@name='id'][.='0']");
assertFalse(h.getCore().getLatestSchema().getField(nonDocValuesField).hasDocValues());
assertTrue(h.getCore().getLatestSchema().getField(nonDocValuesField).multiValued());
assertTrue(h.getCore().getLatestSchema().getField(nonDocValuesField).getType() instanceof PointField);
@ -1219,6 +1258,86 @@ public class TestPointFields extends SolrTestCaseJ4 {
SolrException.ErrorCode.BAD_REQUEST);
}
private void testMultiValuedIntPointFieldsAtomicUpdates(String field, String type) throws Exception {
assertU(adoc(sdoc("id", "1", field, "1")));
assertU(commit());
assertQ(req("q", "id:1"),
"//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='1']",
"count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=1");
assertU(adoc(sdoc("id", "1", field, ImmutableMap.of("add", 2))));
assertU(commit());
assertQ(req("q", "id:1"),
"//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='1']",
"//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='2']",
"count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=2");
assertU(adoc(sdoc("id", "1", field, ImmutableMap.of("remove", 1))));
assertU(commit());
assertQ(req("q", "id:1"),
"//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='2']",
"count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=1");
assertU(adoc(sdoc("id", "1", field, ImmutableMap.of("set", ImmutableList.of(1, 2, 3)))));
assertU(commit());
assertQ(req("q", "id:1"),
"//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='1']",
"//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='2']",
"//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='3']",
"count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=3");
assertU(adoc(sdoc("id", "1", field, ImmutableMap.of("removeregex", ".*"))));
assertU(commit());
assertQ(req("q", "id:1"),
"count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=0");
}
private void testMultiValuedFloatPointFieldsAtomicUpdates(String field, String type) throws Exception {
assertU(adoc(sdoc("id", "1", field, "1.0")));
assertU(commit());
assertQ(req("q", "id:1"),
"//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='1.0']",
"count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=1");
assertU(adoc(sdoc("id", "1", field, ImmutableMap.of("add", 2.1f))));
assertU(commit());
assertQ(req("q", "id:1"),
"//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='1.0']",
"//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='2.1']",
"count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=2");
assertU(adoc(sdoc("id", "1", field, ImmutableMap.of("remove", 1f))));
assertU(commit());
assertQ(req("q", "id:1"),
"//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='2.1']",
"count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=1");
assertU(adoc(sdoc("id", "1", field, ImmutableMap.of("set", ImmutableList.of(1f, 2f, 3f)))));
assertU(commit());
assertQ(req("q", "id:1"),
"//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='1.0']",
"//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='2.0']",
"//result/doc[1]/arr[@name='" + field + "']/" + type + "[.='3.0']",
"count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=3");
assertU(adoc(sdoc("id", "1", field, ImmutableMap.of("removeregex", ".*"))));
assertU(commit());
assertQ(req("q", "id:1"),
"count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=0");
}
private void testIntPointFieldsAtomicUpdates(String field, String type) throws Exception {
assertU(adoc(sdoc("id", "1", field, "1")));
assertU(commit());
@ -1492,4 +1611,152 @@ public class TestPointFields extends SolrTestCaseJ4 {
}
}
}
private void doTestDoublePointFieldMultiValuedRangeFacet(String docValuesField, String nonDocValuesField) throws Exception {
for (int i = 0; i < 10; i++) {
assertU(adoc("id", String.valueOf(i), docValuesField, String.valueOf(i), docValuesField, String.valueOf(i + 10),
nonDocValuesField, String.valueOf(i), nonDocValuesField, String.valueOf(i + 10)));
}
assertU(commit());
assertTrue(h.getCore().getLatestSchema().getField(docValuesField).hasDocValues());
assertTrue(h.getCore().getLatestSchema().getField(docValuesField).multiValued());
assertTrue(h.getCore().getLatestSchema().getField(docValuesField).getType() instanceof PointField);
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", docValuesField, "facet.range.start", "-10", "facet.range.end", "20", "facet.range.gap", "2"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='0.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='2.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='4.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='6.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='8.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='10.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='12.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='14.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='16.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='18.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='-10.0'][.='0']");
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", docValuesField, "facet.range.start", "-10", "facet.range.end", "20", "facet.range.gap", "2", "facet.range.method", "dv"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='0.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='2.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='4.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='6.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='8.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='10.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='12.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='14.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='16.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='18.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='-10.0'][.='0']");
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", docValuesField, "facet.range.start", "0", "facet.range.end", "20", "facet.range.gap", "100"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='0.0'][.='10']");
assertFalse(h.getCore().getLatestSchema().getField(nonDocValuesField).hasDocValues());
assertTrue(h.getCore().getLatestSchema().getField(nonDocValuesField).multiValued());
assertTrue(h.getCore().getLatestSchema().getField(nonDocValuesField).getType() instanceof PointField);
// Range Faceting with method = filter should work
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", nonDocValuesField, "facet.range.start", "-10", "facet.range.end", "20", "facet.range.gap", "2", "facet.range.method", "filter"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='0.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='2.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='4.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='6.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='8.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='10.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='12.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='14.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='16.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='18.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='-10.0'][.='0']");
// this should actually use filter method instead of dv
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", nonDocValuesField, "facet.range.start", "-10", "facet.range.end", "20", "facet.range.gap", "2", "facet.range.method", "dv"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='0.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='2.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='4.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='6.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='8.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='10.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='12.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='14.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='16.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='18.0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='-10.0'][.='0']");
}
private void doTestIntPointFieldMultiValuedRangeFacet(String docValuesField, String nonDocValuesField) throws Exception {
for (int i = 0; i < 10; i++) {
assertU(adoc("id", String.valueOf(i), docValuesField, String.valueOf(i), docValuesField, String.valueOf(i + 10),
nonDocValuesField, String.valueOf(i), nonDocValuesField, String.valueOf(i + 10)));
}
assertU(commit());
assertTrue(h.getCore().getLatestSchema().getField(docValuesField).hasDocValues());
assertTrue(h.getCore().getLatestSchema().getField(docValuesField).getType() instanceof PointField);
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", docValuesField, "facet.range.start", "-10", "facet.range.end", "20", "facet.range.gap", "2"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='2'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='4'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='6'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='8'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='10'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='12'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='14'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='16'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='18'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='-10'][.='0']");
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", docValuesField, "facet.range.start", "-10", "facet.range.end", "20", "facet.range.gap", "2", "facet.range.method", "dv"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='2'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='4'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='6'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='8'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='10'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='12'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='14'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='16'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='18'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='-10'][.='0']");
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", docValuesField, "facet.range.start", "0", "facet.range.end", "20", "facet.range.gap", "100"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + docValuesField + "']/lst[@name='counts']/int[@name='0'][.='10']");
assertFalse(h.getCore().getLatestSchema().getField(nonDocValuesField).hasDocValues());
assertTrue(h.getCore().getLatestSchema().getField(nonDocValuesField).getType() instanceof PointField);
// Range Faceting with method = filter should work
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", nonDocValuesField, "facet.range.start", "-10", "facet.range.end", "20", "facet.range.gap", "2", "facet.range.method", "filter"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='2'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='4'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='6'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='8'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='10'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='12'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='14'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='16'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='18'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='-10'][.='0']");
// this should actually use filter method instead of dv
assertQ(req("q", "*:*", "fl", "id", "facet", "true", "facet.range", nonDocValuesField, "facet.range.start", "-10", "facet.range.end", "20", "facet.range.gap", "2", "facet.range.method", "dv"),
"//*[@numFound='10']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='0'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='2'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='4'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='6'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='8'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='10'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='12'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='14'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='16'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='18'][.='2']",
"//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='-10'][.='0']");
}
}

View File

@ -228,7 +228,7 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 {
assertEquals(26, ((TermInSetQuery)q).getTermData().size());
// large numeric filter query should use TermsQuery (for trie fields)
qParser = QParser.getParser("foo_i:(1 2 3 4 5 6 7 8 9 10 20 19 18 17 16 15 14 13 12 11)", req);
qParser = QParser.getParser("foo_ti:(1 2 3 4 5 6 7 8 9 10 20 19 18 17 16 15 14 13 12 11)", req);
qParser.setIsFilter(true); // this may change in the future
q = qParser.getQuery();
assertEquals(20, ((TermInSetQuery)q).getTermData().size());

View File

@ -140,7 +140,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:222", "indent", "true"), "//result[@numFound = '3']");
@ -153,7 +153,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:222", "indent", "true"), "//result[@numFound = '2']");
doc = new SolrInputDocument();
@ -165,7 +165,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:222", "indent", "true"), "//result[@numFound = '1']");
doc = new SolrInputDocument();
@ -175,7 +175,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:111", "indent", "true"), "//result[@numFound = '3']");
}
@ -208,7 +208,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:222", "indent", "true"), "//result[@numFound = '3']");
@ -221,7 +221,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:222", "indent", "true"), "//result[@numFound = '2']");
doc = new SolrInputDocument();
@ -233,7 +233,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:222", "indent", "true"), "//result[@numFound = '1']");
doc = new SolrInputDocument();
@ -243,7 +243,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:111", "indent", "true"), "//result[@numFound = '3']");
}
@ -272,7 +272,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:222", "indent", "true"), "//result[@numFound = '3']");
@ -285,7 +285,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:222", "indent", "true"), "//result[@numFound = '2']");
doc = new SolrInputDocument();
@ -297,7 +297,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:222", "indent", "true"), "//result[@numFound = '1']");
doc = new SolrInputDocument();
@ -307,7 +307,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:111", "indent", "true"), "//result[@numFound = '3']");
}
@ -337,7 +337,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:222", "indent", "true"), "//result[@numFound = '3']");
doc = new SolrInputDocument();
@ -349,7 +349,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:222", "indent", "true"), "//result[@numFound = '2']");
doc = new SolrInputDocument();
@ -361,7 +361,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:222", "indent", "true"), "//result[@numFound = '1']");
doc = new SolrInputDocument();
@ -371,7 +371,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:111", "indent", "true"), "//result[@numFound = '3']");
}
@ -421,7 +421,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:222", "indent", "true"), "//result[@numFound = '3']");
@ -434,7 +434,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:222", "indent", "true"), "//result[@numFound = '2']");
doc = new SolrInputDocument();
@ -446,7 +446,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:222", "indent", "true"), "//result[@numFound = '1']");
doc = new SolrInputDocument();
@ -456,7 +456,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:111", "indent", "true"), "//result[@numFound = '3']");
}
@ -487,7 +487,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:22222222", "indent", "true"), "//result[@numFound = '3']");
@ -500,7 +500,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:22222222", "indent", "true"), "//result[@numFound = '2']");
doc = new SolrInputDocument();
@ -512,7 +512,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:22222222", "indent", "true"), "//result[@numFound = '1']");
doc = new SolrInputDocument();
@ -522,7 +522,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "intRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "intRemove:11111111", "indent", "true"), "//result[@numFound = '3']");
}
@ -708,7 +708,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(commit());
assertQ(req("q", "floatRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "floatRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "floatRemove:\"222.222\"", "indent", "true"), "//result[@numFound = '3']");
@ -722,7 +722,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "floatRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "floatRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "floatRemove:\"222.222\"", "indent", "true"), "//result[@numFound = '2']");
doc = new SolrInputDocument();
@ -734,7 +734,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "floatRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "floatRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "floatRemove:\"222.222\"", "indent", "true"), "//result[@numFound = '1']");
doc = new SolrInputDocument();
@ -744,7 +744,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "floatRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "floatRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "floatRemove:\"111.111\"", "indent", "true"), "//result[@numFound = '3']");
}
@ -776,7 +776,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(commit());
assertQ(req("q", "floatRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "floatRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "floatRemove:\"222.222\"", "indent", "true"), "//result[@numFound = '3']");
@ -790,7 +790,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "floatRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "floatRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "floatRemove:\"222.222\"", "indent", "true"), "//result[@numFound = '2']");
doc = new SolrInputDocument();
@ -802,7 +802,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "floatRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "floatRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "floatRemove:\"222.222\"", "indent", "true"), "//result[@numFound = '1']");
doc = new SolrInputDocument();
@ -812,7 +812,7 @@ public class AtomicUpdatesTest extends SolrTestCaseJ4 {
assertU(adoc(doc));
assertU(commit());
assertQ(req("q", "floatRemove:*", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "floatRemove:[* TO *]", "indent", "true"), "//result[@numFound = '4']");
assertQ(req("q", "floatRemove:\"111.111\"", "indent", "true"), "//result[@numFound = '3']");
}

View File

@ -2474,4 +2474,8 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
protected static void systemClearPropertySolrTestsMergePolicyFactory() {
System.clearProperty(SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICYFACTORY);
}
protected <T> T pickRandom(T... options) {
return options[random().nextInt(options.length)];
}
}