mirror of https://github.com/apache/lucene.git
make DV strongly typed; fix solr
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4547@1412906 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
156e5bfd80
commit
041ff1140f
|
@ -89,7 +89,8 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
DocFieldProcessorPerField field = fieldHash[i];
|
||||
while(field != null) {
|
||||
// nocommit maybe we should sort by .... somethign?
|
||||
// field name? field number? else this is hash order!!
|
||||
// field name? field number? else this is hash
|
||||
// order!!
|
||||
if (field.bytesDVWriter != null || field.numberDVWriter != null || field.sortedBytesDVWriter != null) {
|
||||
|
||||
if (dvConsumer == null) {
|
||||
|
|
|
@ -161,11 +161,13 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
|
||||
private final Map<Integer,String> numberToName;
|
||||
private final Map<String,Integer> nameToNumber;
|
||||
private final Map<String,DocValues.Type> docValuesType;
|
||||
private int lowestUnassignedFieldNumber = -1;
|
||||
|
||||
FieldNumbers() {
|
||||
this.nameToNumber = new HashMap<String, Integer>();
|
||||
this.numberToName = new HashMap<Integer, String>();
|
||||
this.docValuesType = new HashMap<String,DocValues.Type>();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -174,7 +176,15 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
* number assigned if possible otherwise the first unassigned field number
|
||||
* is used as the field number.
|
||||
*/
|
||||
synchronized int addOrGet(String fieldName, int preferredFieldNumber) {
|
||||
synchronized int addOrGet(String fieldName, int preferredFieldNumber, DocValues.Type dvType) {
|
||||
if (dvType != null) {
|
||||
DocValues.Type currentDVType = docValuesType.get(fieldName);
|
||||
if (currentDVType == null) {
|
||||
docValuesType.put(fieldName, dvType);
|
||||
} else if (currentDVType != null && currentDVType != dvType) {
|
||||
throw new IllegalArgumentException("cannot change DocValues type from " + currentDVType + " to " + dvType + " for field \"" + fieldName + "\"");
|
||||
}
|
||||
}
|
||||
Integer fieldNumber = nameToNumber.get(fieldName);
|
||||
if (fieldNumber == null) {
|
||||
final Integer preferredBoxed = Integer.valueOf(preferredFieldNumber);
|
||||
|
@ -200,21 +210,30 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
/**
|
||||
* Sets the given field number and name if not yet set.
|
||||
*/
|
||||
synchronized void setIfNotSet(int fieldNumber, String fieldName) {
|
||||
synchronized void setIfNotSet(int fieldNumber, String fieldName, DocValues.Type dvType) {
|
||||
final Integer boxedFieldNumber = Integer.valueOf(fieldNumber);
|
||||
if (!numberToName.containsKey(boxedFieldNumber)
|
||||
&& !nameToNumber.containsKey(fieldName)) {
|
||||
&& !nameToNumber.containsKey(fieldName)
|
||||
&& !docValuesType.containsKey(fieldName)) {
|
||||
numberToName.put(boxedFieldNumber, fieldName);
|
||||
nameToNumber.put(fieldName, boxedFieldNumber);
|
||||
docValuesType.put(fieldName, dvType);
|
||||
} else {
|
||||
assert containsConsistent(boxedFieldNumber, fieldName);
|
||||
assert containsConsistent(boxedFieldNumber, fieldName, dvType);
|
||||
}
|
||||
}
|
||||
|
||||
// used by assert
|
||||
synchronized boolean containsConsistent(Integer number, String name) {
|
||||
synchronized boolean containsConsistent(Integer number, String name, DocValues.Type dvType) {
|
||||
return name.equals(numberToName.get(number))
|
||||
&& number.equals(nameToNumber.get(name));
|
||||
&& number.equals(nameToNumber.get(name)) &&
|
||||
(dvType == null || docValuesType.get(name) == null || dvType == docValuesType.get(name));
|
||||
}
|
||||
|
||||
synchronized void clear() {
|
||||
numberToName.clear();
|
||||
nameToNumber.clear();
|
||||
docValuesType.clear();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -247,7 +266,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
*/
|
||||
private void putInternal(FieldInfo fi) {
|
||||
assert !byName.containsKey(fi.name);
|
||||
assert globalFieldNumbers.containsConsistent(Integer.valueOf(fi.number), fi.name);
|
||||
assert globalFieldNumbers.containsConsistent(Integer.valueOf(fi.number), fi.name, fi.getDocValuesType());
|
||||
byName.put(fi.name, fi);
|
||||
}
|
||||
|
||||
|
@ -282,7 +301,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
// be updated by maybe FreqProxTermsWriterPerField:
|
||||
return addOrUpdateInternal(name, -1, fieldType.indexed(), false,
|
||||
fieldType.omitNorms(), false,
|
||||
fieldType.indexOptions(), null, null);
|
||||
fieldType.indexOptions(), fieldType.docValueType(), null);
|
||||
}
|
||||
|
||||
private FieldInfo addOrUpdateInternal(String name, int preferredFieldNumber, boolean isIndexed,
|
||||
|
@ -291,9 +310,17 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
FieldInfo fi = fieldInfo(name);
|
||||
if (fi == null) {
|
||||
// get a global number for this field
|
||||
final int fieldNumber = globalFieldNumbers.addOrGet(name, preferredFieldNumber);
|
||||
final int fieldNumber = globalFieldNumbers.addOrGet(name, preferredFieldNumber, docValues);
|
||||
fi = addInternal(name, fieldNumber, isIndexed, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, normType);
|
||||
} else {
|
||||
if (docValues != null) {
|
||||
DocValues.Type currentDVType = fi.getDocValuesType();
|
||||
if (currentDVType == null) {
|
||||
fi.setDocValuesType(docValues);
|
||||
} else if (currentDVType != docValues) {
|
||||
throw new IllegalArgumentException("cannot change DocValues type from " + currentDVType + " to " + docValues + " for field \"" + name + "\"");
|
||||
}
|
||||
}
|
||||
fi.update(isIndexed, storeTermVector, omitNorms, storePayloads, indexOptions);
|
||||
if (docValues != null) {
|
||||
fi.setDocValuesType(docValues);
|
||||
|
@ -315,7 +342,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
|||
private FieldInfo addInternal(String name, int fieldNumber, boolean isIndexed,
|
||||
boolean storeTermVector, boolean omitNorms, boolean storePayloads,
|
||||
IndexOptions indexOptions, DocValues.Type docValuesType, DocValues.Type normType) {
|
||||
globalFieldNumbers.setIfNotSet(fieldNumber, name);
|
||||
globalFieldNumbers.setIfNotSet(fieldNumber, name, docValuesType);
|
||||
final FieldInfo fi = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, omitNorms, storePayloads, indexOptions, docValuesType, normType, null);
|
||||
putInternal(fi);
|
||||
return fi;
|
||||
|
|
|
@ -729,25 +729,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
* If this {@link SegmentInfos} has no global field number map the returned instance is empty
|
||||
*/
|
||||
private FieldNumbers getFieldNumberMap() throws IOException {
|
||||
final FieldNumbers map = new FieldNumbers();
|
||||
final FieldNumbers map = new FieldNumbers();
|
||||
|
||||
SegmentInfoPerCommit biggest = null;
|
||||
for(SegmentInfoPerCommit info : segmentInfos) {
|
||||
if (biggest == null || (info.info.getDocCount()-info.getDelCount()) > (biggest.info.getDocCount()-biggest.getDelCount())) {
|
||||
biggest = info;
|
||||
for(FieldInfo fi : getFieldInfos(info.info)) {
|
||||
map.addOrGet(fi.name, fi.number, fi.getDocValuesType());
|
||||
}
|
||||
}
|
||||
|
||||
if (biggest != null) {
|
||||
for(FieldInfo fi : getFieldInfos(biggest.info)) {
|
||||
map.addOrGet(fi.name, fi.number);
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: we could also pull DV type of each field here,
|
||||
// and use that to make sure new segment(s) don't change
|
||||
// the type...
|
||||
|
||||
return map;
|
||||
}
|
||||
|
||||
|
@ -1921,7 +1910,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
infoStream.message("IW", "rollback");
|
||||
}
|
||||
|
||||
|
||||
try {
|
||||
synchronized(this) {
|
||||
finishMerges(false);
|
||||
|
@ -2026,6 +2014,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
deleter.checkpoint(segmentInfos, false);
|
||||
deleter.refresh();
|
||||
|
||||
globalFieldNumberMap.clear();
|
||||
|
||||
// Don't bother saving any changes in our segmentInfos
|
||||
readerPool.dropAll(false);
|
||||
|
||||
|
@ -2269,7 +2259,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
|
||||
IOContext context = new IOContext(new MergeInfo(info.info.getDocCount(), info.info.sizeInBytes(), true, -1));
|
||||
|
||||
|
||||
for(FieldInfo fi : getFieldInfos(info.info)) {
|
||||
globalFieldNumberMap.addOrGet(fi.name, fi.number, fi.getDocValuesType());
|
||||
}
|
||||
infos.add(copySegmentAsIs(info, newSegName, context));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,6 +30,8 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.packed.PackedInts.Reader;
|
||||
|
||||
// nocommit if types are inconsistent, refuse to build
|
||||
|
||||
/**
|
||||
* A wrapper for CompositeIndexReader providing access to per segment
|
||||
* {@link DocValues}
|
||||
|
|
|
@ -1913,6 +1913,7 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
}
|
||||
|
||||
// nocommit why do we have this AND DTI.binarySearch?
|
||||
final protected static int binarySearch(BytesRef br, DocTermsIndex a, BytesRef key) {
|
||||
return binarySearch(br, a, key, 0, a.numOrd()-1);
|
||||
}
|
||||
|
|
|
@ -25,10 +25,12 @@ import java.util.EnumSet;
|
|||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.ByteDocValuesField;
|
||||
|
@ -47,9 +49,9 @@ import org.apache.lucene.document.StringField;
|
|||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.DocValues.SortedSource;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.SourceCache.DirectSourceCache;
|
||||
import org.apache.lucene.index.DocValues.SourceCache;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.DocValues.SourceCache.DirectSourceCache;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
|
@ -79,6 +81,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
* Simple test case to show how to use the API
|
||||
*/
|
||||
public void testDocValuesSimple() throws IOException {
|
||||
assumeTrue("requires simple dv", _TestUtil.canUseSimpleDV());
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, writerConfig(false));
|
||||
for (int i = 0; i < 5; i++) {
|
||||
|
@ -133,6 +136,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testAddIndexes() throws IOException {
|
||||
assumeTrue("requires simple dv", _TestUtil.canUseSimpleDV());
|
||||
Directory d1 = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), d1);
|
||||
Document doc = new Document();
|
||||
|
@ -816,8 +820,8 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
Document doc = new Document();
|
||||
// Index doc values are single-valued so we should not
|
||||
// be able to add same field more than once:
|
||||
Field f;
|
||||
doc.add(f = new PackedLongDocValuesField("field", 17));
|
||||
Field f = new PackedLongDocValuesField("field", 17);
|
||||
doc.add(f);
|
||||
doc.add(new SortedBytesDocValuesField("field", new BytesRef("hello")));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
|
@ -825,15 +829,16 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
}
|
||||
|
||||
doc = new Document();
|
||||
doc.add(f);
|
||||
w.addDocument(doc);
|
||||
w.forceMerge(1);
|
||||
DirectoryReader r = w.getReader();
|
||||
if (_TestUtil.canUseSimpleDV()) {
|
||||
DirectoryReader r = w.getReader();
|
||||
assertEquals(17, getOnlySegmentReader(r).getNumericDocValues("field").get(0));
|
||||
r.close();
|
||||
}
|
||||
w.close();
|
||||
assertEquals(17, getOnlySegmentReader(r).getNumericDocValues("field").get(0));
|
||||
r.close();
|
||||
d.close();
|
||||
}
|
||||
|
||||
|
@ -929,6 +934,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testWithThreads() throws Exception {
|
||||
assumeTrue("requires simple dv", _TestUtil.canUseSimpleDV());
|
||||
Random random = random();
|
||||
final int NUM_DOCS = atLeast(100);
|
||||
final Directory dir = newDirectory();
|
||||
|
@ -1110,57 +1116,227 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
|||
writer.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
// Same field in one document as different types:
|
||||
public void testMixedTypesSameDocument() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())));
|
||||
Document doc = new Document();
|
||||
doc.add(new IntDocValuesField("foo", 0));
|
||||
doc.add(new SortedBytesDocValuesField("foo", new BytesRef("hello")));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
}
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
// Two documents with same field as different types:
|
||||
public void testMixedTypesDifferentDocuments() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())));
|
||||
Document doc = new Document();
|
||||
doc.add(new IntDocValuesField("foo", 0));
|
||||
w.addDocument(doc);
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new SortedBytesDocValuesField("foo", new BytesRef("hello")));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
}
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
// Two documents across segments
|
||||
public void testMixedTypesDifferentSegments() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())));
|
||||
Document doc = new Document();
|
||||
doc.add(new IntDocValuesField("foo", 0));
|
||||
w.addDocument(doc);
|
||||
w.commit();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new SortedBytesDocValuesField("foo", new BytesRef("hello")));
|
||||
try {
|
||||
w.addDocument(doc);
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
}
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
// Add inconsistent document after deleteAll
|
||||
public void testMixedTypesAfterDeleteAll() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())));
|
||||
Document doc = new Document();
|
||||
doc.add(new IntDocValuesField("foo", 0));
|
||||
w.addDocument(doc);
|
||||
w.deleteAll();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new SortedBytesDocValuesField("foo", new BytesRef("hello")));
|
||||
w.addDocument(doc);
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
// Add inconsistent document after reopening IW w/ create
|
||||
public void testMixedTypesAfterReopenCreate() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())));
|
||||
Document doc = new Document();
|
||||
doc.add(new IntDocValuesField("foo", 0));
|
||||
w.addDocument(doc);
|
||||
w.close();
|
||||
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||
iwc.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
|
||||
w = new IndexWriter(dir, iwc);
|
||||
doc = new Document();
|
||||
doc.add(new SortedBytesDocValuesField("foo", new BytesRef("hello")));
|
||||
w.addDocument(doc);
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
// Two documents with same field as different types, added
|
||||
// from separate threads:
|
||||
public void testMixedTypesDifferentThreads() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
final IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())));
|
||||
|
||||
final CountDownLatch startingGun = new CountDownLatch(1);
|
||||
final AtomicBoolean hitExc = new AtomicBoolean();
|
||||
Thread[] threads = new Thread[3];
|
||||
for(int i=0;i<3;i++) {
|
||||
Field field;
|
||||
if (i == 0) {
|
||||
field = new SortedBytesDocValuesField("foo", new BytesRef("hello"));
|
||||
} else if (i == 1) {
|
||||
field = new IntDocValuesField("foo", 0);
|
||||
} else {
|
||||
field = new DerefBytesDocValuesField("foo", new BytesRef("bazz"), true);
|
||||
}
|
||||
final Document doc = new Document();
|
||||
doc.add(field);
|
||||
|
||||
threads[i] = new Thread() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
startingGun.await();
|
||||
w.addDocument(doc);
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
hitExc.set(true);
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
};
|
||||
threads[i].start();
|
||||
}
|
||||
|
||||
startingGun.countDown();
|
||||
|
||||
for(Thread t : threads) {
|
||||
t.join();
|
||||
}
|
||||
assertTrue(hitExc.get());
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
// Adding documents via addIndexes
|
||||
public void testMixedTypesViaAddIndexes() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())));
|
||||
Document doc = new Document();
|
||||
doc.add(new IntDocValuesField("foo", 0));
|
||||
w.addDocument(doc);
|
||||
|
||||
// Make 2nd index w/ inconsistent field
|
||||
Directory dir2 = newDirectory();
|
||||
IndexWriter w2 = new IndexWriter(dir2, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())));
|
||||
doc = new Document();
|
||||
doc.add(new SortedBytesDocValuesField("foo", new BytesRef("hello")));
|
||||
w2.addDocument(doc);
|
||||
w2.close();
|
||||
|
||||
try {
|
||||
w.addIndexes(new Directory[] {dir2});
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
}
|
||||
|
||||
IndexReader r = DirectoryReader.open(dir2);
|
||||
try {
|
||||
w.addIndexes(new IndexReader[] {r});
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
}
|
||||
|
||||
r.close();
|
||||
dir2.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public static class NotCachingSourceCache extends SourceCache {
|
||||
public static class NotCachingSourceCache extends SourceCache {
|
||||
|
||||
@Override
|
||||
public Source load(DocValues values) throws IOException {
|
||||
return values.loadSource();
|
||||
}
|
||||
@Override
|
||||
public Source load(DocValues values) throws IOException {
|
||||
return values.loadSource();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source loadDirect(DocValues values) throws IOException {
|
||||
return values.loadDirectSource();
|
||||
}
|
||||
@Override
|
||||
public Source loadDirect(DocValues values) throws IOException {
|
||||
return values.loadDirectSource();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void invalidate(DocValues values) {}
|
||||
}
|
||||
@Override
|
||||
public void invalidate(DocValues values) {}
|
||||
}
|
||||
|
||||
public NumericDocValues numeric(AtomicReader reader, String field) throws IOException {
|
||||
NumericDocValues docValues = reader.getNumericDocValues(field);
|
||||
if(random().nextBoolean()) {
|
||||
return docValues.newRAMInstance();
|
||||
}
|
||||
return docValues;
|
||||
}
|
||||
public NumericDocValues numeric(AtomicReader reader, String field) throws IOException {
|
||||
NumericDocValues docValues = reader.getNumericDocValues(field);
|
||||
if(random().nextBoolean()) {
|
||||
return docValues.newRAMInstance();
|
||||
}
|
||||
return docValues;
|
||||
}
|
||||
|
||||
public NumericDocValues numeric(DirectoryReader reader, String field) throws IOException {
|
||||
return numeric(getOnlySegmentReader(reader), field);
|
||||
}
|
||||
public BinaryDocValues binary(DirectoryReader reader, String field) throws IOException {
|
||||
return binary(getOnlySegmentReader(reader), field);
|
||||
}
|
||||
public SortedDocValues sorted(DirectoryReader reader, String field) throws IOException {
|
||||
return sorted(getOnlySegmentReader(reader), field);
|
||||
}
|
||||
public NumericDocValues numeric(DirectoryReader reader, String field) throws IOException {
|
||||
return numeric(getOnlySegmentReader(reader), field);
|
||||
}
|
||||
public BinaryDocValues binary(DirectoryReader reader, String field) throws IOException {
|
||||
return binary(getOnlySegmentReader(reader), field);
|
||||
}
|
||||
public SortedDocValues sorted(DirectoryReader reader, String field) throws IOException {
|
||||
return sorted(getOnlySegmentReader(reader), field);
|
||||
}
|
||||
|
||||
public BinaryDocValues binary(AtomicReader reader, String field) throws IOException {
|
||||
BinaryDocValues docValues = reader.getBinaryDocValues(field);
|
||||
if(random().nextBoolean()) {
|
||||
return docValues.newRAMInstance();
|
||||
}
|
||||
return docValues;
|
||||
}
|
||||
public SortedDocValues sorted(AtomicReader reader, String field) throws IOException {
|
||||
SortedDocValues docValues = reader.getSortedDocValues(field);
|
||||
if(random().nextBoolean()) {
|
||||
return docValues.newRAMInstance();
|
||||
}
|
||||
return docValues;
|
||||
}
|
||||
public BinaryDocValues binary(AtomicReader reader, String field) throws IOException {
|
||||
BinaryDocValues docValues = reader.getBinaryDocValues(field);
|
||||
if(random().nextBoolean()) {
|
||||
return docValues.newRAMInstance();
|
||||
}
|
||||
return docValues;
|
||||
}
|
||||
public SortedDocValues sorted(AtomicReader reader, String field) throws IOException {
|
||||
SortedDocValues docValues = reader.getSortedDocValues(field);
|
||||
if(random().nextBoolean()) {
|
||||
return docValues.newRAMInstance();
|
||||
}
|
||||
return docValues;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -1,317 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.ByteDocValuesField;
|
||||
import org.apache.lucene.document.DerefBytesDocValuesField;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.DoubleDocValuesField;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FloatDocValuesField;
|
||||
import org.apache.lucene.document.IntDocValuesField;
|
||||
import org.apache.lucene.document.LongDocValuesField;
|
||||
import org.apache.lucene.document.PackedLongDocValuesField;
|
||||
import org.apache.lucene.document.ShortDocValuesField;
|
||||
import org.apache.lucene.document.SortedBytesDocValuesField;
|
||||
import org.apache.lucene.document.StraightBytesDocValuesField;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.junit.Ignore;
|
||||
|
||||
/**
|
||||
* Tests compatibility of {@link DocValues.Type} during indexing
|
||||
*/
|
||||
public class TestDocValuesTypeCompatibility extends LuceneTestCase {
|
||||
|
||||
public void testAddCompatibleIntTypes() throws IOException {
|
||||
int numIter = atLeast(10);
|
||||
for (int i = 0; i < numIter; i++) {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT,
|
||||
new MockAnalyzer(random()));
|
||||
int numDocs = atLeast(100);
|
||||
|
||||
iwc.setMaxBufferedDocs(2 * numDocs); // make sure we hit the same DWPT
|
||||
// here
|
||||
iwc.setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
iwc.setRAMPerThreadHardLimitMB(2000);
|
||||
IndexWriter writer = new IndexWriter(dir, iwc);
|
||||
Type[] types = new Type[] {Type.VAR_INTS, Type.FIXED_INTS_16,
|
||||
Type.FIXED_INTS_64, Type.FIXED_INTS_16, Type.FIXED_INTS_8};
|
||||
Type maxType = types[random().nextInt(types.length)];
|
||||
for (int j = 0; j < numDocs; j++) {
|
||||
addDoc(writer, getRandomIntsField(maxType, j == 0));
|
||||
}
|
||||
writer.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@SuppressWarnings("fallthrough")
|
||||
public Field getRandomIntsField(Type maxType, boolean force) {
|
||||
switch (maxType) {
|
||||
|
||||
case VAR_INTS:
|
||||
if (random().nextInt(5) == 0 || force) {
|
||||
return new PackedLongDocValuesField("f", 1);
|
||||
}
|
||||
case FIXED_INTS_64:
|
||||
if (random().nextInt(4) == 0 || force) {
|
||||
return new LongDocValuesField("f", 1);
|
||||
}
|
||||
case FIXED_INTS_32:
|
||||
if (random().nextInt(3) == 0 || force) {
|
||||
return new IntDocValuesField("f", 1);
|
||||
}
|
||||
case FIXED_INTS_16:
|
||||
if (random().nextInt(2) == 0 || force) {
|
||||
return new ShortDocValuesField("f", (short) 1);
|
||||
}
|
||||
case FIXED_INTS_8:
|
||||
return new ByteDocValuesField("f", (byte) 1);
|
||||
|
||||
default:
|
||||
throw new IllegalArgumentException();
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
public void testAddCompatibleDoubleTypes() throws IOException {
|
||||
int numIter = atLeast(10);
|
||||
for (int i = 0; i < numIter; i++) {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT,
|
||||
new MockAnalyzer(random()));
|
||||
int numDocs = atLeast(100);
|
||||
|
||||
iwc.setMaxBufferedDocs(2 * numDocs); // make sure we hit the same DWPT
|
||||
// here
|
||||
iwc.setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
iwc.setRAMPerThreadHardLimitMB(2000);
|
||||
IndexWriter writer = new IndexWriter(dir, iwc);
|
||||
|
||||
Type[] types = new Type[] {Type.FLOAT_64, Type.FLOAT_32};
|
||||
Type maxType = types[random().nextInt(types.length)];
|
||||
for (int j = 0; j < numDocs; j++) {
|
||||
addDoc(writer, getRandomFloatField(maxType, j == 0));
|
||||
}
|
||||
writer.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
}
|
||||
@SuppressWarnings("fallthrough")
|
||||
public Field getRandomFloatField(Type maxType, boolean force) {
|
||||
switch (maxType) {
|
||||
|
||||
case FLOAT_64:
|
||||
if (random().nextInt(5) == 0 || force) {
|
||||
return new PackedLongDocValuesField("f", 1);
|
||||
}
|
||||
case FIXED_INTS_32:
|
||||
if (random().nextInt(4) == 0 || force) {
|
||||
return new LongDocValuesField("f", 1);
|
||||
}
|
||||
case FLOAT_32:
|
||||
if (random().nextInt(3) == 0 || force) {
|
||||
return new IntDocValuesField("f", 1);
|
||||
}
|
||||
case FIXED_INTS_16:
|
||||
if (random().nextInt(2) == 0 || force) {
|
||||
return new ShortDocValuesField("f", (short) 1);
|
||||
}
|
||||
case FIXED_INTS_8:
|
||||
return new ByteDocValuesField("f", (byte) 1);
|
||||
|
||||
default:
|
||||
throw new IllegalArgumentException();
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
public void testAddCompatibleDoubleTypes2() throws IOException {
|
||||
int numIter = atLeast(10);
|
||||
for (int i = 0; i < numIter; i++) {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT,
|
||||
new MockAnalyzer(random()));
|
||||
int numDocs = atLeast(100);
|
||||
|
||||
iwc.setMaxBufferedDocs(2 * numDocs); // make sure we hit the same DWPT
|
||||
// here
|
||||
iwc.setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
iwc.setRAMPerThreadHardLimitMB(2000);
|
||||
IndexWriter writer = new IndexWriter(dir, iwc);
|
||||
Field[] fields = new Field[] {
|
||||
new DoubleDocValuesField("f", 1.0), new IntDocValuesField("f", 1),
|
||||
new ShortDocValuesField("f", (short) 1),
|
||||
new ByteDocValuesField("f", (byte) 1)};
|
||||
int base = random().nextInt(fields.length - 1);
|
||||
|
||||
addDoc(writer, fields[base]);
|
||||
|
||||
for (int j = 0; j < numDocs; j++) {
|
||||
int f = base + random().nextInt(fields.length - base);
|
||||
addDoc(writer, fields[f]);
|
||||
}
|
||||
writer.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// nocommit remove this test? simple dv doesn't let you
|
||||
// change b/w sorted & binary?
|
||||
@Ignore
|
||||
public void testAddCompatibleByteTypes() throws IOException {
|
||||
int numIter = atLeast(10);
|
||||
for (int i = 0; i < numIter; i++) {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT,
|
||||
new MockAnalyzer(random()));
|
||||
int numDocs = atLeast(100);
|
||||
|
||||
iwc.setMaxBufferedDocs(2 * numDocs); // make sure we hit the same DWPT
|
||||
// here
|
||||
iwc.setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
iwc.setRAMPerThreadHardLimitMB(2000);
|
||||
IndexWriter writer = new IndexWriter(dir, iwc);
|
||||
boolean mustBeFixed = random().nextBoolean();
|
||||
int maxSize = 2 + random().nextInt(15);
|
||||
Field bytesField = getRandomBytesField(mustBeFixed, maxSize,
|
||||
true);
|
||||
addDoc(writer, bytesField);
|
||||
for (int j = 0; j < numDocs; j++) {
|
||||
bytesField = getRandomBytesField(mustBeFixed, maxSize, false);
|
||||
addDoc(writer, bytesField);
|
||||
|
||||
}
|
||||
writer.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
||||
public Field getRandomBytesField(boolean mustBeFixed, int maxSize,
|
||||
boolean mustBeVariableIfNotFixed) {
|
||||
int size = mustBeFixed ? maxSize : random().nextInt(maxSize) + 1;
|
||||
StringBuilder s = new StringBuilder();
|
||||
for (int i = 0; i < size; i++) {
|
||||
s.append("a");
|
||||
}
|
||||
BytesRef bytesRef = new BytesRef(s);
|
||||
boolean fixed = mustBeFixed ? true : mustBeVariableIfNotFixed ? false
|
||||
: random().nextBoolean();
|
||||
switch (random().nextInt(3)) {
|
||||
case 0:
|
||||
return new SortedBytesDocValuesField("f", bytesRef, fixed);
|
||||
case 1:
|
||||
return new DerefBytesDocValuesField("f", bytesRef, fixed);
|
||||
default:
|
||||
return new StraightBytesDocValuesField("f", bytesRef, fixed);
|
||||
}
|
||||
}
|
||||
|
||||
public void testIncompatibleTypesBytes() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT,
|
||||
new MockAnalyzer(random()));
|
||||
int numDocs = atLeast(100);
|
||||
|
||||
iwc.setMaxBufferedDocs(numDocs); // make sure we hit the same DWPT
|
||||
iwc.setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
|
||||
iwc.setRAMPerThreadHardLimitMB(2000);
|
||||
IndexWriter writer = new IndexWriter(dir, iwc);
|
||||
|
||||
int numDocsIndexed = 0;
|
||||
for (int j = 1; j < numDocs; j++) {
|
||||
try {
|
||||
addDoc(writer, getRandomIndexableDVField());
|
||||
numDocsIndexed++;
|
||||
} catch (IllegalArgumentException e) {
|
||||
assertTrue(e.getMessage().startsWith("Incompatible DocValues type:"));
|
||||
}
|
||||
}
|
||||
|
||||
writer.commit();
|
||||
DirectoryReader open = DirectoryReader.open(dir);
|
||||
assertEquals(numDocsIndexed, open.numDocs());
|
||||
open.close();
|
||||
writer.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
private void addDoc(IndexWriter writer, Field... fields)
|
||||
throws IOException {
|
||||
Document doc = new Document();
|
||||
for (Field indexableField : fields) {
|
||||
doc.add(indexableField);
|
||||
}
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
|
||||
public Field getRandomIndexableDVField() {
|
||||
int size = random().nextInt(100) + 1;
|
||||
StringBuilder s = new StringBuilder();
|
||||
for (int i = 0; i < size; i++) {
|
||||
s.append("a");
|
||||
}
|
||||
BytesRef bytesRef = new BytesRef(s);
|
||||
|
||||
Type[] values = Type.values();
|
||||
Type t = values[random().nextInt(values.length)];
|
||||
switch (t) {
|
||||
case BYTES_FIXED_DEREF:
|
||||
return new DerefBytesDocValuesField("f", bytesRef, true);
|
||||
case BYTES_FIXED_SORTED:
|
||||
return new SortedBytesDocValuesField("f", bytesRef, true);
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
return new StraightBytesDocValuesField("f", bytesRef, true);
|
||||
case BYTES_VAR_DEREF:
|
||||
return new DerefBytesDocValuesField("f", bytesRef, false);
|
||||
case BYTES_VAR_SORTED:
|
||||
return new SortedBytesDocValuesField("f", bytesRef, false);
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
return new StraightBytesDocValuesField("f", bytesRef, false);
|
||||
case FIXED_INTS_16:
|
||||
return new ShortDocValuesField("f", (short) 1);
|
||||
case FIXED_INTS_32:
|
||||
return new IntDocValuesField("f", 1);
|
||||
case FIXED_INTS_64:
|
||||
return new LongDocValuesField("f", 1);
|
||||
case FIXED_INTS_8:
|
||||
return new ByteDocValuesField("f", (byte) 1);
|
||||
case FLOAT_32:
|
||||
return new FloatDocValuesField("f", 1.0f);
|
||||
case FLOAT_64:
|
||||
return new DoubleDocValuesField("f", 1.0f);
|
||||
case VAR_INTS:
|
||||
return new PackedLongDocValuesField("f", 1);
|
||||
default:
|
||||
throw new IllegalArgumentException();
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -48,11 +48,12 @@ import org.junit.BeforeClass;
|
|||
|
||||
public class TestFieldsReader extends LuceneTestCase {
|
||||
private static Directory dir;
|
||||
private static Document testDoc = new Document();
|
||||
private static Document testDoc;
|
||||
private static FieldInfos.Builder fieldInfos = null;
|
||||
|
||||
@BeforeClass
|
||||
public static void beforeClass() throws Exception {
|
||||
testDoc = new Document();
|
||||
fieldInfos = new FieldInfos.Builder();
|
||||
DocHelper.setupDoc(testDoc);
|
||||
for (IndexableField field : testDoc.getFields()) {
|
||||
|
|
|
@ -1,385 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.ByteDocValuesField;
|
||||
import org.apache.lucene.document.DerefBytesDocValuesField;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.DoubleDocValuesField;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FloatDocValuesField;
|
||||
import org.apache.lucene.document.IntDocValuesField;
|
||||
import org.apache.lucene.document.LongDocValuesField;
|
||||
import org.apache.lucene.document.PackedLongDocValuesField;
|
||||
import org.apache.lucene.document.ShortDocValuesField;
|
||||
import org.apache.lucene.document.SortedBytesDocValuesField;
|
||||
import org.apache.lucene.document.StraightBytesDocValuesField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.store.ByteArrayDataOutput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
public class TestTypePromotion extends LuceneTestCase {
|
||||
|
||||
private static EnumSet<Type> INTEGERS = EnumSet.of(Type.VAR_INTS,
|
||||
Type.FIXED_INTS_16, Type.FIXED_INTS_32,
|
||||
Type.FIXED_INTS_64, Type.FIXED_INTS_8);
|
||||
|
||||
private static EnumSet<Type> FLOATS = EnumSet.of(Type.FLOAT_32,
|
||||
Type.FLOAT_64, Type.FIXED_INTS_8);
|
||||
|
||||
private static EnumSet<Type> UNSORTED_BYTES = EnumSet.of(
|
||||
Type.BYTES_FIXED_DEREF, Type.BYTES_FIXED_STRAIGHT,
|
||||
Type.BYTES_VAR_STRAIGHT, Type.BYTES_VAR_DEREF);
|
||||
|
||||
private static EnumSet<Type> SORTED_BYTES = EnumSet.of(
|
||||
Type.BYTES_FIXED_SORTED, Type.BYTES_VAR_SORTED);
|
||||
|
||||
public Type randomValueType(EnumSet<Type> typeEnum, Random random) {
|
||||
Type[] array = typeEnum.toArray(new Type[0]);
|
||||
return array[random.nextInt(array.length)];
|
||||
}
|
||||
|
||||
private static enum TestType {
|
||||
Int, Float, Byte
|
||||
}
|
||||
|
||||
private void runTest(EnumSet<Type> types, TestType type) throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir,
|
||||
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())));
|
||||
int num_1 = atLeast(200);
|
||||
int num_2 = atLeast(200);
|
||||
int num_3 = atLeast(200);
|
||||
long[] values = new long[num_1 + num_2 + num_3];
|
||||
Type[] sourceType = new Type[num_1 + num_2 + num_3];
|
||||
index(writer,
|
||||
randomValueType(types, random()), values, sourceType, 0, num_1);
|
||||
writer.commit();
|
||||
|
||||
index(writer,
|
||||
randomValueType(types, random()), values, sourceType, num_1, num_2);
|
||||
writer.commit();
|
||||
|
||||
if (random().nextInt(4) == 0) {
|
||||
// once in a while use addIndexes
|
||||
writer.forceMerge(1);
|
||||
|
||||
Directory dir_2 = newDirectory() ;
|
||||
IndexWriter writer_2 = new IndexWriter(dir_2,
|
||||
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())));
|
||||
index(writer_2,
|
||||
randomValueType(types, random()), values, sourceType, num_1 + num_2, num_3);
|
||||
writer_2.commit();
|
||||
writer_2.close();
|
||||
if (rarely()) {
|
||||
writer.addIndexes(dir_2);
|
||||
} else {
|
||||
// do a real merge here
|
||||
IndexReader open = maybeWrapReader(DirectoryReader.open(dir_2));
|
||||
writer.addIndexes(open);
|
||||
open.close();
|
||||
}
|
||||
dir_2.close();
|
||||
} else {
|
||||
index(writer,
|
||||
randomValueType(types, random()), values, sourceType, num_1 + num_2, num_3);
|
||||
}
|
||||
|
||||
writer.forceMerge(1);
|
||||
writer.close();
|
||||
assertValues(type, dir, values, sourceType);
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
||||
private void assertValues(TestType type, Directory dir, long[] values, Type[] sourceType)
|
||||
throws IOException {
|
||||
DirectoryReader reader = DirectoryReader.open(dir);
|
||||
assertEquals(1, reader.leaves().size());
|
||||
IndexReaderContext topReaderContext = reader.getContext();
|
||||
List<AtomicReaderContext> leaves = topReaderContext.leaves();
|
||||
assertEquals(1, leaves.size());
|
||||
DocValues docValues = leaves.get(0).reader().docValues("promote");
|
||||
Source directSource = docValues.getDirectSource();
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
int id = Integer.parseInt(reader.document(i).get("id"));
|
||||
String msg = "id: " + id + " doc: " + i;
|
||||
switch (type) {
|
||||
case Byte:
|
||||
BytesRef bytes = directSource.getBytes(i, new BytesRef());
|
||||
long value = 0;
|
||||
switch(bytes.length) {
|
||||
case 1:
|
||||
value = bytes.bytes[bytes.offset];
|
||||
break;
|
||||
case 2:
|
||||
value = ((bytes.bytes[bytes.offset] & 0xFF) << 8) | (bytes.bytes[bytes.offset+1] & 0xFF);
|
||||
break;
|
||||
case 4:
|
||||
value = ((bytes.bytes[bytes.offset] & 0xFF) << 24) | ((bytes.bytes[bytes.offset+1] & 0xFF) << 16)
|
||||
| ((bytes.bytes[bytes.offset+2] & 0xFF) << 8) | (bytes.bytes[bytes.offset+3] & 0xFF);
|
||||
break;
|
||||
case 8:
|
||||
value = (((long)(bytes.bytes[bytes.offset] & 0xff) << 56) | ((long)(bytes.bytes[bytes.offset+1] & 0xff) << 48) |
|
||||
((long)(bytes.bytes[bytes.offset+2] & 0xff) << 40) | ((long)(bytes.bytes[bytes.offset+3] & 0xff) << 32) |
|
||||
((long)(bytes.bytes[bytes.offset+4] & 0xff) << 24) | ((long)(bytes.bytes[bytes.offset+5] & 0xff) << 16) |
|
||||
((long)(bytes.bytes[bytes.offset+6] & 0xff) << 8) | ((long)(bytes.bytes[bytes.offset+7] & 0xff)));
|
||||
break;
|
||||
|
||||
default:
|
||||
fail(msg + " bytessize: " + bytes.length);
|
||||
}
|
||||
|
||||
assertEquals(msg + " byteSize: " + bytes.length, values[id], value);
|
||||
break;
|
||||
case Float:
|
||||
if (sourceType[id] == Type.FLOAT_32
|
||||
|| sourceType[id] == Type.FLOAT_64) {
|
||||
assertEquals(msg, values[id],
|
||||
Double.doubleToRawLongBits(directSource.getFloat(i)));
|
||||
} else {
|
||||
assertEquals(msg, values[id], directSource.getFloat(i), 0.0d);
|
||||
}
|
||||
break;
|
||||
case Int:
|
||||
assertEquals(msg, values[id], directSource.getInt(i));
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
|
||||
}
|
||||
docValues.close();
|
||||
reader.close();
|
||||
}
|
||||
|
||||
public void index(IndexWriter writer,
|
||||
Type valueType, long[] values, Type[] sourceTypes, int offset, int num)
|
||||
throws IOException {
|
||||
final Field valField;
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: add docs " + offset + "-" + (offset+num) + " valType=" + valueType);
|
||||
}
|
||||
|
||||
switch(valueType) {
|
||||
case VAR_INTS:
|
||||
valField = new PackedLongDocValuesField("promote", (long) 0);
|
||||
break;
|
||||
case FIXED_INTS_8:
|
||||
valField = new ByteDocValuesField("promote", (byte) 0);
|
||||
break;
|
||||
case FIXED_INTS_16:
|
||||
valField = new ShortDocValuesField("promote", (short) 0);
|
||||
break;
|
||||
case FIXED_INTS_32:
|
||||
valField = new IntDocValuesField("promote", 0);
|
||||
break;
|
||||
case FIXED_INTS_64:
|
||||
valField = new LongDocValuesField("promote", (byte) 0);
|
||||
break;
|
||||
case FLOAT_32:
|
||||
valField = new FloatDocValuesField("promote", 0f);
|
||||
break;
|
||||
case FLOAT_64:
|
||||
valField = new DoubleDocValuesField("promote", 0d);
|
||||
break;
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
valField = new StraightBytesDocValuesField("promote", new BytesRef(), true);
|
||||
break;
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
valField = new StraightBytesDocValuesField("promote", new BytesRef(), false);
|
||||
break;
|
||||
case BYTES_FIXED_DEREF:
|
||||
valField = new DerefBytesDocValuesField("promote", new BytesRef(), true);
|
||||
break;
|
||||
case BYTES_VAR_DEREF:
|
||||
valField = new DerefBytesDocValuesField("promote", new BytesRef(), false);
|
||||
break;
|
||||
case BYTES_FIXED_SORTED:
|
||||
valField = new SortedBytesDocValuesField("promote", new BytesRef(), true);
|
||||
break;
|
||||
case BYTES_VAR_SORTED:
|
||||
valField = new SortedBytesDocValuesField("promote", new BytesRef(), false);
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("unknown Type: " + valueType);
|
||||
}
|
||||
|
||||
for (int i = offset; i < offset + num; i++) {
|
||||
Document doc = new Document();
|
||||
doc.add(new TextField("id", i + "", Field.Store.YES));
|
||||
sourceTypes[i] = valueType;
|
||||
switch (valueType) {
|
||||
case VAR_INTS:
|
||||
// TODO: can we do nextLong()?
|
||||
values[i] = random().nextInt();
|
||||
valField.setLongValue(values[i]);
|
||||
break;
|
||||
case FIXED_INTS_16:
|
||||
// TODO: negatives too?
|
||||
values[i] = random().nextInt(Short.MAX_VALUE);
|
||||
valField.setShortValue((short) values[i]);
|
||||
break;
|
||||
case FIXED_INTS_32:
|
||||
values[i] = random().nextInt();
|
||||
valField.setIntValue((int) values[i]);
|
||||
break;
|
||||
case FIXED_INTS_64:
|
||||
values[i] = random().nextLong();
|
||||
valField.setLongValue(values[i]);
|
||||
break;
|
||||
case FLOAT_64:
|
||||
final double nextDouble = random().nextDouble();
|
||||
values[i] = Double.doubleToRawLongBits(nextDouble);
|
||||
valField.setDoubleValue(nextDouble);
|
||||
break;
|
||||
case FLOAT_32:
|
||||
final float nextFloat = random().nextFloat();
|
||||
values[i] = Double.doubleToRawLongBits(nextFloat);
|
||||
valField.setFloatValue(nextFloat);
|
||||
break;
|
||||
case FIXED_INTS_8:
|
||||
values[i] = (byte) i;
|
||||
valField.setByteValue((byte)values[i]);
|
||||
break;
|
||||
case BYTES_FIXED_DEREF:
|
||||
case BYTES_FIXED_SORTED:
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
values[i] = random().nextLong();
|
||||
byte bytes[] = new byte[8];
|
||||
ByteArrayDataOutput out = new ByteArrayDataOutput(bytes, 0, 8);
|
||||
out.writeLong(values[i]);
|
||||
valField.setBytesValue(new BytesRef(bytes));
|
||||
break;
|
||||
case BYTES_VAR_DEREF:
|
||||
case BYTES_VAR_SORTED:
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
byte lbytes[] = new byte[8];
|
||||
ByteArrayDataOutput lout = new ByteArrayDataOutput(lbytes, 0, 8);
|
||||
final int len;
|
||||
if (random().nextBoolean()) {
|
||||
values[i] = random().nextInt();
|
||||
lout.writeInt((int)values[i]);
|
||||
len = 4;
|
||||
} else {
|
||||
values[i] = random().nextLong();
|
||||
lout.writeLong(values[i]);
|
||||
len = 8;
|
||||
}
|
||||
valField.setBytesValue(new BytesRef(lbytes, 0, len));
|
||||
break;
|
||||
|
||||
default:
|
||||
fail("unexpected value " + valueType);
|
||||
}
|
||||
if (VERBOSE) {
|
||||
System.out.println(" doc " + i + " has val=" + valField);
|
||||
}
|
||||
doc.add(valField);
|
||||
writer.addDocument(doc);
|
||||
if (random().nextInt(10) == 0) {
|
||||
writer.commit();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testPromoteBytes() throws IOException {
|
||||
runTest(UNSORTED_BYTES, TestType.Byte);
|
||||
}
|
||||
|
||||
public void testSortedPromoteBytes() throws IOException {
|
||||
runTest(SORTED_BYTES, TestType.Byte);
|
||||
}
|
||||
|
||||
public void testPromoteInteger() throws IOException {
|
||||
runTest(INTEGERS, TestType.Int);
|
||||
}
|
||||
|
||||
public void testPromotFloatingPoint() throws IOException {
|
||||
runTest(FLOATS, TestType.Float);
|
||||
}
|
||||
|
||||
public void testMergeIncompatibleTypes() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig writerConfig = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||
writerConfig.setMergePolicy(NoMergePolicy.NO_COMPOUND_FILES); // no merges until we are done with adding values
|
||||
IndexWriter writer = new IndexWriter(dir, writerConfig);
|
||||
int num_1 = atLeast(200);
|
||||
int num_2 = atLeast(200);
|
||||
long[] values = new long[num_1 + num_2];
|
||||
Type[] sourceType = new Type[num_1 + num_2];
|
||||
index(writer,
|
||||
randomValueType(INTEGERS, random()), values, sourceType, 0, num_1);
|
||||
writer.commit();
|
||||
|
||||
if (random().nextInt(4) == 0) {
|
||||
// once in a while use addIndexes
|
||||
Directory dir_2 = newDirectory() ;
|
||||
IndexWriter writer_2 = new IndexWriter(dir_2,
|
||||
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())));
|
||||
index(writer_2,
|
||||
randomValueType(random().nextBoolean() ? UNSORTED_BYTES : SORTED_BYTES, random()), values, sourceType, num_1, num_2);
|
||||
writer_2.commit();
|
||||
writer_2.close();
|
||||
if (random().nextBoolean()) {
|
||||
writer.addIndexes(dir_2);
|
||||
} else {
|
||||
// do a real merge here
|
||||
IndexReader open = DirectoryReader.open(dir_2);
|
||||
writer.addIndexes(open);
|
||||
open.close();
|
||||
}
|
||||
dir_2.close();
|
||||
} else {
|
||||
index(writer,
|
||||
randomValueType(random().nextBoolean() ? UNSORTED_BYTES : SORTED_BYTES, random()), values, sourceType, num_1, num_2);
|
||||
writer.commit();
|
||||
}
|
||||
writer.close();
|
||||
writerConfig = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||
if (writerConfig.getMergePolicy() instanceof NoMergePolicy) {
|
||||
writerConfig.setMergePolicy(newLogMergePolicy()); // make sure we merge to one segment (merge everything together)
|
||||
}
|
||||
writer = new IndexWriter(dir, writerConfig);
|
||||
// now merge
|
||||
writer.forceMerge(1);
|
||||
writer.close();
|
||||
DirectoryReader reader = DirectoryReader.open(dir);
|
||||
assertEquals(1, reader.leaves().size());
|
||||
IndexReaderContext topReaderContext = reader.getContext();
|
||||
List<AtomicReaderContext> leaves = topReaderContext.leaves();
|
||||
DocValues docValues = leaves.get(0).reader().docValues("promote");
|
||||
assertNotNull(docValues);
|
||||
assertValues(TestType.Byte, dir, values, sourceType);
|
||||
assertEquals(Type.BYTES_VAR_STRAIGHT, docValues.getType());
|
||||
reader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
}
|
|
@ -96,18 +96,18 @@ public class TestBasics extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
static Analyzer simplePayloadAnalyzer = new Analyzer() {
|
||||
static Analyzer simplePayloadAnalyzer;
|
||||
|
||||
@Override
|
||||
public TokenStreamComponents createComponents(String fieldName, Reader reader) {
|
||||
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.SIMPLE, true);
|
||||
return new TokenStreamComponents(tokenizer, new SimplePayloadFilter(tokenizer));
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
@BeforeClass
|
||||
public static void beforeClass() throws Exception {
|
||||
simplePayloadAnalyzer = new Analyzer() {
|
||||
@Override
|
||||
public TokenStreamComponents createComponents(String fieldName, Reader reader) {
|
||||
Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.SIMPLE, true);
|
||||
return new TokenStreamComponents(tokenizer, new SimplePayloadFilter(tokenizer));
|
||||
}
|
||||
};
|
||||
|
||||
directory = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), directory,
|
||||
newIndexWriterConfig(TEST_VERSION_CURRENT, simplePayloadAnalyzer)
|
||||
|
|
|
@ -298,11 +298,13 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
|
|||
if (fields[i].getType() == SortField.Type.SCORE) {
|
||||
scores[i] = scorer.score();
|
||||
} else {
|
||||
sortValues[i] = sortsIndex[i].getTerm(doc, new BytesRef());
|
||||
sortOrds[i] = sortsIndex[i].getOrd(doc);
|
||||
sortValues[i] = new BytesRef();
|
||||
if (sortOrds[i] != -1) {
|
||||
sortValues[i] = sortsIndex[i].getTerm(doc, sortValues[i]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public int compare(int compIDX, int doc) throws IOException {
|
||||
|
@ -317,7 +319,12 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
|
|||
} else {
|
||||
if (sortOrds[compIDX] < 0) {
|
||||
// The current segment doesn't contain the sort value we encountered before. Therefore the ord is negative.
|
||||
return sortValues[compIDX].compareTo(sortsIndex[compIDX].getTerm(doc, scratchBytesRef));
|
||||
if (sortsIndex[compIDX].getOrd(doc) == -1) {
|
||||
scratchBytesRef.length = 0;
|
||||
} else {
|
||||
sortsIndex[compIDX].getTerm(doc, scratchBytesRef);
|
||||
}
|
||||
return sortValues[compIDX].compareTo(scratchBytesRef);
|
||||
} else {
|
||||
return sortOrds[compIDX] - sortsIndex[compIDX].getOrd(doc);
|
||||
}
|
||||
|
@ -329,15 +336,17 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
|
|||
if (fields[i].getType() == SortField.Type.SCORE) {
|
||||
scores[i] = scorer.score();
|
||||
} else {
|
||||
sortValues[i] = sortsIndex[i].getTerm(doc, sortValues[i]);
|
||||
sortOrds[i] = sortsIndex[i].getOrd(doc);
|
||||
if (sortOrds[i] == -1) {
|
||||
sortValues[i].length = 0;
|
||||
} else {
|
||||
sortValues[i] = sortsIndex[i].getTerm(doc, sortValues[i]);
|
||||
}
|
||||
}
|
||||
}
|
||||
this.doc = doc + readerContext.docBase;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
@ -412,7 +421,7 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
|
|||
|
||||
for (int i = 0; i < sortsIndex.length; i++) {
|
||||
int sortOrd;
|
||||
if (collectedGroup.sortValues[i] == null) {
|
||||
if (collectedGroup.sortOrds[i] == -1) {
|
||||
sortOrd = -1;
|
||||
} else {
|
||||
sortOrd = sortsIndex[i].binarySearchLookup(collectedGroup.sortValues[i], scratchBytesRef);
|
||||
|
@ -433,15 +442,23 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
|
|||
sortValues = new BytesRef[sortsIndex.length];
|
||||
sortOrds = new int[sortsIndex.length];
|
||||
for (int i = 0; i < sortsIndex.length; i++) {
|
||||
sortValues[i] = sortsIndex[i].getTerm(doc, new BytesRef());
|
||||
sortOrds[i] = sortsIndex[i].getOrd(doc);
|
||||
sortValues[i] = new BytesRef();
|
||||
if (sortOrds[i] != -1) {
|
||||
sortsIndex[i].getTerm(doc, sortValues[i]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public int compare(int compIDX, int doc) throws IOException {
|
||||
if (sortOrds[compIDX] < 0) {
|
||||
// The current segment doesn't contain the sort value we encountered before. Therefore the ord is negative.
|
||||
return sortValues[compIDX].compareTo(sortsIndex[compIDX].getTerm(doc, scratchBytesRef));
|
||||
if (sortsIndex[compIDX].getOrd(doc) == -1) {
|
||||
scratchBytesRef.length = 0;
|
||||
} else {
|
||||
sortsIndex[compIDX].getTerm(doc, scratchBytesRef);
|
||||
}
|
||||
return sortValues[compIDX].compareTo(scratchBytesRef);
|
||||
} else {
|
||||
return sortOrds[compIDX] - sortsIndex[compIDX].getOrd(doc);
|
||||
}
|
||||
|
@ -449,8 +466,12 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
|
|||
|
||||
public void updateDocHead(int doc) throws IOException {
|
||||
for (int i = 0; i < sortsIndex.length; i++) {
|
||||
sortValues[i] = sortsIndex[i].getTerm(doc, sortValues[i]);
|
||||
sortOrds[i] = sortsIndex[i].getOrd(doc);
|
||||
if (sortOrds[i] == -1) {
|
||||
sortValues[i].length = 0;
|
||||
} else {
|
||||
sortValues[i] = sortsIndex[i].getTerm(doc, sortValues[i]);
|
||||
}
|
||||
}
|
||||
this.doc = doc + readerContext.docBase;
|
||||
}
|
||||
|
|
|
@ -79,7 +79,7 @@ public class ReverseOrdFieldSource extends ValueSource {
|
|||
return new IntDocValues(this) {
|
||||
@Override
|
||||
public int intVal(int doc) {
|
||||
return (end - sindex.getOrd(doc+off));
|
||||
return (end - sindex.getOrd(doc+off) - 1);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -133,7 +133,7 @@ public class TestOrdValues extends FunctionTestSetup {
|
|||
String id = s.getIndexReader().document(sd[i].doc).get(ID_FIELD);
|
||||
log("-------- " + i + ". Explain doc " + id);
|
||||
log(s.explain(q, sd[i].doc));
|
||||
float expectedScore = N_DOCS - i;
|
||||
float expectedScore = N_DOCS - i - 1;
|
||||
assertEquals("score of result " + i + " shuould be " + expectedScore + " != " + score, expectedScore, score, TEST_SCORE_TOLERANCE_DELTA);
|
||||
String expectedId = inOrder
|
||||
? id2String(N_DOCS - i) // in-order ==> larger values first
|
||||
|
|
|
@ -77,15 +77,14 @@ import org.apache.lucene.index.Terms;
|
|||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.TieredMergePolicy;
|
||||
import org.apache.lucene.search.FieldDoc;
|
||||
import org.apache.lucene.search.FilteredQuery;
|
||||
import org.apache.lucene.search.FilteredQuery.FilterStrategy;
|
||||
import org.apache.lucene.search.FilteredQuery;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.junit.Assert;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.RandomizedContext;
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomInts;
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
||||
|
@ -674,6 +673,11 @@ public class _TestUtil {
|
|||
}
|
||||
}
|
||||
|
||||
// nocommit remove this once all codecs support simple dv!!
|
||||
public static boolean canUseSimpleDV() {
|
||||
return Codec.getDefault().simpleDocValuesFormat() != null;
|
||||
}
|
||||
|
||||
public static boolean anyFilesExceptWriteLock(Directory dir) throws IOException {
|
||||
String[] files = dir.listAll();
|
||||
if (files.length > 1 || (files.length == 1 && !files[0].equals("write.lock"))) {
|
||||
|
|
|
@ -62,7 +62,7 @@ public class FieldFacetStats {
|
|||
this.facet_sf = facet_sf;
|
||||
this.numStatsTerms = numStatsTerms;
|
||||
|
||||
startTermIndex = 1;
|
||||
startTermIndex = 0;
|
||||
endTermIndex = si.numOrd();
|
||||
nTerms = endTermIndex - startTermIndex;
|
||||
|
||||
|
@ -79,7 +79,7 @@ public class FieldFacetStats {
|
|||
|
||||
BytesRef getTermText(int docID, BytesRef ret) {
|
||||
final int ord = si.getOrd(docID);
|
||||
if (ord == 0) {
|
||||
if (ord == -1) {
|
||||
return null;
|
||||
} else {
|
||||
return si.lookup(ord, ret);
|
||||
|
|
|
@ -261,7 +261,7 @@ class SimpleStats {
|
|||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
"Stats can only facet on single-valued fields, not: " + facetField );
|
||||
}
|
||||
|
||||
|
||||
try {
|
||||
facetTermsIndex = FieldCache.DEFAULT.getTermsIndex(searcher.getAtomicReader(), facetField);
|
||||
}
|
||||
|
@ -276,11 +276,19 @@ class SimpleStats {
|
|||
DocIterator iter = docs.iterator();
|
||||
while (iter.hasNext()) {
|
||||
int docID = iter.nextDoc();
|
||||
BytesRef raw = si.lookup(si.getOrd(docID), tempBR);
|
||||
if( raw.length > 0 ) {
|
||||
allstats.accumulate(raw);
|
||||
} else {
|
||||
int docOrd = si.getOrd(docID);
|
||||
BytesRef raw;
|
||||
if (docOrd == -1) {
|
||||
allstats.missing();
|
||||
tempBR.length = 0;
|
||||
raw = tempBR;
|
||||
} else {
|
||||
raw = si.lookup(docOrd, tempBR);
|
||||
if( raw.length > 0 ) {
|
||||
allstats.accumulate(raw);
|
||||
} else {
|
||||
allstats.missing();
|
||||
}
|
||||
}
|
||||
|
||||
// now update the facets
|
||||
|
|
|
@ -144,10 +144,10 @@ class PerSegmentSingleValuedFaceting {
|
|||
|
||||
|
||||
if (seg.startTermIndex < seg.endTermIndex) {
|
||||
if (seg.startTermIndex==0) {
|
||||
if (seg.startTermIndex==-1) {
|
||||
hasMissingCount=true;
|
||||
missingCount += seg.counts[0];
|
||||
seg.pos = 1;
|
||||
seg.pos = 0;
|
||||
} else {
|
||||
seg.pos = seg.startTermIndex;
|
||||
}
|
||||
|
@ -247,7 +247,7 @@ class PerSegmentSingleValuedFaceting {
|
|||
assert endTermIndex < 0;
|
||||
endTermIndex = -endTermIndex-1;
|
||||
} else {
|
||||
startTermIndex=0;
|
||||
startTermIndex=-1;
|
||||
endTermIndex=si.numOrd();
|
||||
}
|
||||
|
||||
|
@ -261,68 +261,19 @@ class PerSegmentSingleValuedFaceting {
|
|||
|
||||
|
||||
////
|
||||
PackedInts.Reader ordReader = si.getDocToOrd();
|
||||
int doc;
|
||||
|
||||
final Object arr;
|
||||
if (ordReader.hasArray()) {
|
||||
arr = ordReader.getArray();
|
||||
} else {
|
||||
arr = null;
|
||||
}
|
||||
|
||||
if (arr instanceof int[]) {
|
||||
int[] ords = (int[]) arr;
|
||||
if (prefix==null) {
|
||||
while ((doc = iter.nextDoc()) < DocIdSetIterator.NO_MORE_DOCS) {
|
||||
counts[ords[doc]]++;
|
||||
}
|
||||
} else {
|
||||
while ((doc = iter.nextDoc()) < DocIdSetIterator.NO_MORE_DOCS) {
|
||||
int term = ords[doc];
|
||||
int arrIdx = term-startTermIndex;
|
||||
if (arrIdx>=0 && arrIdx<nTerms) counts[arrIdx]++;
|
||||
}
|
||||
}
|
||||
} else if (arr instanceof short[]) {
|
||||
short[] ords = (short[]) arr;
|
||||
if (prefix==null) {
|
||||
while ((doc = iter.nextDoc()) < DocIdSetIterator.NO_MORE_DOCS) {
|
||||
counts[ords[doc] & 0xffff]++;
|
||||
}
|
||||
} else {
|
||||
while ((doc = iter.nextDoc()) < DocIdSetIterator.NO_MORE_DOCS) {
|
||||
int term = ords[doc] & 0xffff;
|
||||
int arrIdx = term-startTermIndex;
|
||||
if (arrIdx>=0 && arrIdx<nTerms) counts[arrIdx]++;
|
||||
}
|
||||
}
|
||||
} else if (arr instanceof byte[]) {
|
||||
byte[] ords = (byte[]) arr;
|
||||
if (prefix==null) {
|
||||
while ((doc = iter.nextDoc()) < DocIdSetIterator.NO_MORE_DOCS) {
|
||||
counts[ords[doc] & 0xff]++;
|
||||
}
|
||||
} else {
|
||||
while ((doc = iter.nextDoc()) < DocIdSetIterator.NO_MORE_DOCS) {
|
||||
int term = ords[doc] & 0xff;
|
||||
int arrIdx = term-startTermIndex;
|
||||
if (arrIdx>=0 && arrIdx<nTerms) counts[arrIdx]++;
|
||||
}
|
||||
if (prefix==null) {
|
||||
// specialized version when collecting counts for all terms
|
||||
while ((doc = iter.nextDoc()) < DocIdSetIterator.NO_MORE_DOCS) {
|
||||
counts[1+si.getOrd(doc)]++;
|
||||
}
|
||||
} else {
|
||||
if (prefix==null) {
|
||||
// specialized version when collecting counts for all terms
|
||||
while ((doc = iter.nextDoc()) < DocIdSetIterator.NO_MORE_DOCS) {
|
||||
counts[si.getOrd(doc)]++;
|
||||
}
|
||||
} else {
|
||||
// version that adjusts term numbers because we aren't collecting the full range
|
||||
while ((doc = iter.nextDoc()) < DocIdSetIterator.NO_MORE_DOCS) {
|
||||
int term = si.getOrd(doc);
|
||||
int arrIdx = term-startTermIndex;
|
||||
if (arrIdx>=0 && arrIdx<nTerms) counts[arrIdx]++;
|
||||
}
|
||||
// version that adjusts term numbers because we aren't collecting the full range
|
||||
while ((doc = iter.nextDoc()) < DocIdSetIterator.NO_MORE_DOCS) {
|
||||
int term = si.getOrd(doc);
|
||||
int arrIdx = term-startTermIndex;
|
||||
if (arrIdx>=0 && arrIdx<nTerms) counts[arrIdx]++;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -500,7 +500,7 @@ public class SimpleFacets {
|
|||
assert endTermIndex < 0;
|
||||
endTermIndex = -endTermIndex-1;
|
||||
} else {
|
||||
startTermIndex=0;
|
||||
startTermIndex=-1;
|
||||
endTermIndex=si.numOrd();
|
||||
}
|
||||
|
||||
|
@ -515,62 +515,13 @@ public class SimpleFacets {
|
|||
|
||||
DocIterator iter = docs.iterator();
|
||||
|
||||
PackedInts.Reader ordReader = si.getDocToOrd();
|
||||
final Object arr;
|
||||
if (ordReader.hasArray()) {
|
||||
arr = ordReader.getArray();
|
||||
} else {
|
||||
arr = null;
|
||||
while (iter.hasNext()) {
|
||||
int term = si.getOrd(iter.nextDoc());
|
||||
int arrIdx = term-startTermIndex;
|
||||
if (arrIdx>=0 && arrIdx<nTerms) counts[arrIdx]++;
|
||||
}
|
||||
|
||||
if (arr instanceof int[]) {
|
||||
int[] ords = (int[]) arr;
|
||||
if (prefix==null) {
|
||||
while (iter.hasNext()) {
|
||||
counts[ords[iter.nextDoc()]]++;
|
||||
}
|
||||
} else {
|
||||
while (iter.hasNext()) {
|
||||
int term = ords[iter.nextDoc()];
|
||||
int arrIdx = term-startTermIndex;
|
||||
if (arrIdx>=0 && arrIdx<nTerms) counts[arrIdx]++;
|
||||
}
|
||||
}
|
||||
} else if (arr instanceof short[]) {
|
||||
short[] ords = (short[]) arr;
|
||||
if (prefix==null) {
|
||||
while (iter.hasNext()) {
|
||||
counts[ords[iter.nextDoc()] & 0xffff]++;
|
||||
}
|
||||
} else {
|
||||
while (iter.hasNext()) {
|
||||
int term = ords[iter.nextDoc()] & 0xffff;
|
||||
int arrIdx = term-startTermIndex;
|
||||
if (arrIdx>=0 && arrIdx<nTerms) counts[arrIdx]++;
|
||||
}
|
||||
}
|
||||
} else if (arr instanceof byte[]) {
|
||||
byte[] ords = (byte[]) arr;
|
||||
if (prefix==null) {
|
||||
while (iter.hasNext()) {
|
||||
counts[ords[iter.nextDoc()] & 0xff]++;
|
||||
}
|
||||
} else {
|
||||
while (iter.hasNext()) {
|
||||
int term = ords[iter.nextDoc()] & 0xff;
|
||||
int arrIdx = term-startTermIndex;
|
||||
if (arrIdx>=0 && arrIdx<nTerms) counts[arrIdx]++;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
while (iter.hasNext()) {
|
||||
int term = si.getOrd(iter.nextDoc());
|
||||
int arrIdx = term-startTermIndex;
|
||||
if (arrIdx>=0 && arrIdx<nTerms) counts[arrIdx]++;
|
||||
}
|
||||
}
|
||||
|
||||
if (startTermIndex == 0) {
|
||||
if (startTermIndex == -1) {
|
||||
missingCount = counts[0];
|
||||
}
|
||||
|
||||
|
@ -586,7 +537,7 @@ public class SimpleFacets {
|
|||
LongPriorityQueue queue = new LongPriorityQueue(Math.min(maxsize,1000), maxsize, Long.MIN_VALUE);
|
||||
|
||||
int min=mincount-1; // the smallest value in the top 'N' values
|
||||
for (int i=(startTermIndex==0)?1:0; i<nTerms; i++) {
|
||||
for (int i=(startTermIndex==-1)?1:0; i<nTerms; i++) {
|
||||
int c = counts[i];
|
||||
if (c>min) {
|
||||
// NOTE: we use c>min rather than c>=min as an optimization because we are going in
|
||||
|
@ -619,7 +570,7 @@ public class SimpleFacets {
|
|||
|
||||
} else {
|
||||
// add results in index order
|
||||
int i=(startTermIndex==0)?1:0;
|
||||
int i=(startTermIndex==-1)?1:0;
|
||||
if (mincount<=0) {
|
||||
// if mincount<=0, then we won't discard any terms and we know exactly
|
||||
// where to start.
|
||||
|
|
|
@ -173,7 +173,7 @@ class BoolFieldSource extends ValueSource {
|
|||
int nord = sindex.numOrd();
|
||||
BytesRef br = new BytesRef();
|
||||
int tord = -1;
|
||||
for (int i=1; i<nord; i++) {
|
||||
for (int i=0; i<nord; i++) {
|
||||
sindex.lookup(i, br);
|
||||
if (br.length==1 && br.bytes[br.offset]=='T') {
|
||||
tord = i;
|
||||
|
@ -191,7 +191,7 @@ class BoolFieldSource extends ValueSource {
|
|||
|
||||
@Override
|
||||
public boolean exists(int doc) {
|
||||
return sindex.getOrd(doc) != 0;
|
||||
return sindex.getOrd(doc) != -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -208,7 +208,7 @@ class BoolFieldSource extends ValueSource {
|
|||
public void fillValue(int doc) {
|
||||
int ord = sindex.getOrd(doc);
|
||||
mval.value = (ord == trueOrd);
|
||||
mval.exists = (ord != 0);
|
||||
mval.exists = (ord != -1);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -497,7 +497,7 @@ class DateFieldSource extends FieldCacheSource {
|
|||
@Override
|
||||
public String strVal(int doc) {
|
||||
int ord=termsIndex.getOrd(doc);
|
||||
if (ord == 0) {
|
||||
if (ord == -1) {
|
||||
return null;
|
||||
} else {
|
||||
final BytesRef br = termsIndex.lookup(ord, spare);
|
||||
|
@ -508,7 +508,7 @@ class DateFieldSource extends FieldCacheSource {
|
|||
@Override
|
||||
public Object objectVal(int doc) {
|
||||
int ord=termsIndex.getOrd(doc);
|
||||
if (ord == 0) {
|
||||
if (ord == -1) {
|
||||
return null;
|
||||
} else {
|
||||
final BytesRef br = termsIndex.lookup(ord, new BytesRef());
|
||||
|
|
|
@ -149,7 +149,7 @@ class SortableDoubleFieldSource extends FieldCacheSource {
|
|||
@Override
|
||||
public double doubleVal(int doc) {
|
||||
int ord=termsIndex.getOrd(doc);
|
||||
return ord==0 ? def : NumberUtils.SortableStr2double(termsIndex.lookup(ord, spare));
|
||||
return ord==-1 ? def : NumberUtils.SortableStr2double(termsIndex.lookup(ord, spare));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -160,7 +160,7 @@ class SortableDoubleFieldSource extends FieldCacheSource {
|
|||
@Override
|
||||
public Object objectVal(int doc) {
|
||||
int ord=termsIndex.getOrd(doc);
|
||||
return ord==0 ? null : NumberUtils.SortableStr2double(termsIndex.lookup(ord, spare));
|
||||
return ord==-1 ? null : NumberUtils.SortableStr2double(termsIndex.lookup(ord, spare));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -181,7 +181,7 @@ class SortableDoubleFieldSource extends FieldCacheSource {
|
|||
@Override
|
||||
public void fillValue(int doc) {
|
||||
int ord=termsIndex.getOrd(doc);
|
||||
if (ord == 0) {
|
||||
if (ord == -1) {
|
||||
mval.value = def;
|
||||
mval.exists = false;
|
||||
} else {
|
||||
|
|
|
@ -137,7 +137,7 @@ class SortableFloatFieldSource extends FieldCacheSource {
|
|||
@Override
|
||||
public float floatVal(int doc) {
|
||||
int ord=termsIndex.getOrd(doc);
|
||||
return ord==0 ? def : NumberUtils.SortableStr2float(termsIndex.lookup(ord, spare));
|
||||
return ord==-1 ? def : NumberUtils.SortableStr2float(termsIndex.lookup(ord, spare));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -168,7 +168,7 @@ class SortableFloatFieldSource extends FieldCacheSource {
|
|||
@Override
|
||||
public Object objectVal(int doc) {
|
||||
int ord=termsIndex.getOrd(doc);
|
||||
return ord==0 ? null : NumberUtils.SortableStr2float(termsIndex.lookup(ord, spare));
|
||||
return ord==-1 ? null : NumberUtils.SortableStr2float(termsIndex.lookup(ord, spare));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -184,7 +184,7 @@ class SortableFloatFieldSource extends FieldCacheSource {
|
|||
@Override
|
||||
public void fillValue(int doc) {
|
||||
int ord=termsIndex.getOrd(doc);
|
||||
if (ord == 0) {
|
||||
if (ord == -1) {
|
||||
mval.value = def;
|
||||
mval.exists = false;
|
||||
} else {
|
||||
|
|
|
@ -144,7 +144,7 @@ class SortableIntFieldSource extends FieldCacheSource {
|
|||
@Override
|
||||
public int intVal(int doc) {
|
||||
int ord=termsIndex.getOrd(doc);
|
||||
return ord==0 ? def : NumberUtils.SortableStr2int(termsIndex.lookup(ord, spare),0,3);
|
||||
return ord==-1 ? def : NumberUtils.SortableStr2int(termsIndex.lookup(ord, spare),0,3);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -170,7 +170,7 @@ class SortableIntFieldSource extends FieldCacheSource {
|
|||
@Override
|
||||
public Object objectVal(int doc) {
|
||||
int ord=termsIndex.getOrd(doc);
|
||||
return ord==0 ? null : NumberUtils.SortableStr2int(termsIndex.lookup(ord, spare));
|
||||
return ord==-1 ? null : NumberUtils.SortableStr2int(termsIndex.lookup(ord, spare));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -186,7 +186,7 @@ class SortableIntFieldSource extends FieldCacheSource {
|
|||
@Override
|
||||
public void fillValue(int doc) {
|
||||
int ord=termsIndex.getOrd(doc);
|
||||
if (ord == 0) {
|
||||
if (ord == -1) {
|
||||
mval.value = def;
|
||||
mval.exists = false;
|
||||
} else {
|
||||
|
|
|
@ -147,7 +147,7 @@ class SortableLongFieldSource extends FieldCacheSource {
|
|||
@Override
|
||||
public long longVal(int doc) {
|
||||
int ord=termsIndex.getOrd(doc);
|
||||
return ord==0 ? def : NumberUtils.SortableStr2long(termsIndex.lookup(ord, spare),0,5);
|
||||
return ord==-1 ? def : NumberUtils.SortableStr2long(termsIndex.lookup(ord, spare),0,5);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -163,7 +163,7 @@ class SortableLongFieldSource extends FieldCacheSource {
|
|||
@Override
|
||||
public Object objectVal(int doc) {
|
||||
int ord=termsIndex.getOrd(doc);
|
||||
return ord==0 ? null : NumberUtils.SortableStr2long(termsIndex.lookup(ord, spare));
|
||||
return ord==-1 ? null : NumberUtils.SortableStr2long(termsIndex.lookup(ord, spare));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -184,7 +184,7 @@ class SortableLongFieldSource extends FieldCacheSource {
|
|||
@Override
|
||||
public void fillValue(int doc) {
|
||||
int ord=termsIndex.getOrd(doc);
|
||||
if (ord == 0) {
|
||||
if (ord == -1) {
|
||||
mval.value = def;
|
||||
mval.exists = false;
|
||||
} else {
|
||||
|
|
|
@ -56,7 +56,7 @@ public class MissingStringLastComparatorSource extends FieldComparatorSource {
|
|||
// Copied from Lucene's TermOrdValComparator and modified since the Lucene version couldn't
|
||||
// be extended.
|
||||
class TermOrdValComparator_SML extends FieldComparator<Comparable> {
|
||||
private static final int NULL_ORD = Integer.MAX_VALUE;
|
||||
private static final int NULL_ORD = Integer.MAX_VALUE-1;
|
||||
|
||||
private final int[] ords;
|
||||
private final BytesRef[] values;
|
||||
|
@ -196,7 +196,7 @@ class TermOrdValComparator_SML extends FieldComparator<Comparable> {
|
|||
bottomSameReader = true;
|
||||
} else {
|
||||
if (bottomValue == null) {
|
||||
// 0 ord is null for all segments
|
||||
// -1 ord is null for all segments
|
||||
assert ords[bottomSlot] == NULL_ORD;
|
||||
bottomOrd = NULL_ORD;
|
||||
bottomSameReader = true;
|
||||
|
@ -237,211 +237,54 @@ class TermOrdValComparator_SML extends FieldComparator<Comparable> {
|
|||
}
|
||||
}
|
||||
|
||||
// Used per-segment when bit width of doc->ord is 8:
|
||||
private static final class ByteOrdComparator extends PerSegmentComparator {
|
||||
private final byte[] readerOrds;
|
||||
|
||||
public ByteOrdComparator(byte[] readerOrds, TermOrdValComparator_SML parent) {
|
||||
super(parent);
|
||||
this.readerOrds = readerOrds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareBottom(int doc) {
|
||||
assert bottomSlot != -1;
|
||||
int order = readerOrds[doc]&0xFF;
|
||||
if (order == 0) order = NULL_ORD;
|
||||
if (bottomSameReader) {
|
||||
// ord is precisely comparable, even in the equal case
|
||||
return bottomOrd - order;
|
||||
} else {
|
||||
// ord is only approx comparable: if they are not
|
||||
// equal, we can use that; if they are equal, we
|
||||
// must fallback to compare by value
|
||||
final int cmp = bottomOrd - order;
|
||||
if (cmp != 0) {
|
||||
return cmp;
|
||||
}
|
||||
|
||||
// take care of the case where both vals are null
|
||||
if (order == NULL_ORD) return 0;
|
||||
|
||||
// and at this point we know that neither value is null, so safe to compare
|
||||
termsIndex.lookup(order, tempBR);
|
||||
return bottomValue.compareTo(tempBR);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copy(int slot, int doc) {
|
||||
int ord = readerOrds[doc]&0xFF;
|
||||
if (ord == 0) {
|
||||
ords[slot] = NULL_ORD;
|
||||
values[slot] = null;
|
||||
} else {
|
||||
ords[slot] = ord;
|
||||
assert ord > 0;
|
||||
if (values[slot] == null) {
|
||||
values[slot] = new BytesRef();
|
||||
}
|
||||
termsIndex.lookup(ord, values[slot]);
|
||||
}
|
||||
readerGen[slot] = currentReaderGen;
|
||||
}
|
||||
}
|
||||
|
||||
// Used per-segment when bit width of doc->ord is 16:
|
||||
private static final class ShortOrdComparator extends PerSegmentComparator {
|
||||
private final short[] readerOrds;
|
||||
|
||||
public ShortOrdComparator(short[] readerOrds, TermOrdValComparator_SML parent) {
|
||||
super(parent);
|
||||
this.readerOrds = readerOrds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareBottom(int doc) {
|
||||
assert bottomSlot != -1;
|
||||
int order = readerOrds[doc]&0xFFFF;
|
||||
if (order == 0) order = NULL_ORD;
|
||||
if (bottomSameReader) {
|
||||
// ord is precisely comparable, even in the equal case
|
||||
return bottomOrd - order;
|
||||
} else {
|
||||
// ord is only approx comparable: if they are not
|
||||
// equal, we can use that; if they are equal, we
|
||||
// must fallback to compare by value
|
||||
final int cmp = bottomOrd - order;
|
||||
if (cmp != 0) {
|
||||
return cmp;
|
||||
}
|
||||
|
||||
// take care of the case where both vals are null
|
||||
if (order == NULL_ORD) return 0;
|
||||
|
||||
// and at this point we know that neither value is null, so safe to compare
|
||||
termsIndex.lookup(order, tempBR);
|
||||
return bottomValue.compareTo(tempBR);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copy(int slot, int doc) {
|
||||
int ord = readerOrds[doc]&0xFFFF;
|
||||
if (ord == 0) {
|
||||
ords[slot] = NULL_ORD;
|
||||
values[slot] = null;
|
||||
} else {
|
||||
ords[slot] = ord;
|
||||
assert ord > 0;
|
||||
if (values[slot] == null) {
|
||||
values[slot] = new BytesRef();
|
||||
}
|
||||
termsIndex.lookup(ord, values[slot]);
|
||||
}
|
||||
readerGen[slot] = currentReaderGen;
|
||||
}
|
||||
}
|
||||
|
||||
// Used per-segment when bit width of doc->ord is 32:
|
||||
private static final class IntOrdComparator extends PerSegmentComparator {
|
||||
private final int[] readerOrds;
|
||||
|
||||
public IntOrdComparator(int[] readerOrds, TermOrdValComparator_SML parent) {
|
||||
super(parent);
|
||||
this.readerOrds = readerOrds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareBottom(int doc) {
|
||||
assert bottomSlot != -1;
|
||||
int order = readerOrds[doc];
|
||||
if (order == 0) order = NULL_ORD;
|
||||
if (bottomSameReader) {
|
||||
// ord is precisely comparable, even in the equal case
|
||||
return bottomOrd - order;
|
||||
} else {
|
||||
// ord is only approx comparable: if they are not
|
||||
// equal, we can use that; if they are equal, we
|
||||
// must fallback to compare by value
|
||||
final int cmp = bottomOrd - order;
|
||||
if (cmp != 0) {
|
||||
return cmp;
|
||||
}
|
||||
|
||||
// take care of the case where both vals are null
|
||||
if (order == NULL_ORD) return 0;
|
||||
|
||||
// and at this point we know that neither value is null, so safe to compare
|
||||
termsIndex.lookup(order, tempBR);
|
||||
return bottomValue.compareTo(tempBR);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copy(int slot, int doc) {
|
||||
int ord = readerOrds[doc];
|
||||
if (ord == 0) {
|
||||
ords[slot] = NULL_ORD;
|
||||
values[slot] = null;
|
||||
} else {
|
||||
ords[slot] = ord;
|
||||
assert ord > 0;
|
||||
if (values[slot] == null) {
|
||||
values[slot] = new BytesRef();
|
||||
}
|
||||
termsIndex.lookup(ord, values[slot]);
|
||||
}
|
||||
readerGen[slot] = currentReaderGen;
|
||||
}
|
||||
}
|
||||
|
||||
// Used per-segment when bit width is not a native array
|
||||
// size (8, 16, 32):
|
||||
private static final class AnyOrdComparator extends PerSegmentComparator {
|
||||
private final PackedInts.Reader readerOrds;
|
||||
|
||||
public AnyOrdComparator(PackedInts.Reader readerOrds, TermOrdValComparator_SML parent) {
|
||||
public AnyOrdComparator(TermOrdValComparator_SML parent) {
|
||||
super(parent);
|
||||
this.readerOrds = readerOrds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareBottom(int doc) {
|
||||
assert bottomSlot != -1;
|
||||
int order = (int) readerOrds.get(doc);
|
||||
if (order == 0) order = NULL_ORD;
|
||||
int order = termsIndex.getOrd(doc);
|
||||
if (order == -1) order = NULL_ORD;
|
||||
if (bottomSameReader) {
|
||||
// ord is precisely comparable, even in the equal case
|
||||
// ord is precisely comparable, even in the equal
|
||||
// case
|
||||
return bottomOrd - order;
|
||||
} else {
|
||||
// ord is only approx comparable: if they are not
|
||||
// equal, we can use that; if they are equal, we
|
||||
// must fallback to compare by value
|
||||
|
||||
final int cmp = bottomOrd - order;
|
||||
if (cmp != 0) {
|
||||
return cmp;
|
||||
}
|
||||
|
||||
// take care of the case where both vals are null
|
||||
if (order == NULL_ORD) return 0;
|
||||
if (order == NULL_ORD) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
// and at this point we know that neither value is null, so safe to compare
|
||||
termsIndex.lookup(order, tempBR);
|
||||
return bottomValue.compareTo(tempBR);
|
||||
if (order == NULL_ORD) {
|
||||
return bottomValue.compareTo(parent.NULL_VAL);
|
||||
} else {
|
||||
termsIndex.lookup(order, tempBR);
|
||||
return bottomValue.compareTo(tempBR);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copy(int slot, int doc) {
|
||||
int ord = (int) readerOrds.get(doc);
|
||||
if (ord == 0) {
|
||||
int ord = termsIndex.getOrd(doc);
|
||||
if (ord == -1) {
|
||||
ords[slot] = NULL_ORD;
|
||||
values[slot] = null;
|
||||
} else {
|
||||
ords[slot] = ord;
|
||||
assert ord > 0;
|
||||
assert ord >= 0;
|
||||
if (values[slot] == null) {
|
||||
values[slot] = new BytesRef();
|
||||
}
|
||||
|
@ -453,22 +296,7 @@ class TermOrdValComparator_SML extends FieldComparator<Comparable> {
|
|||
|
||||
public static FieldComparator createComparator(AtomicReader reader, TermOrdValComparator_SML parent) throws IOException {
|
||||
parent.termsIndex = FieldCache.DEFAULT.getTermsIndex(reader, parent.field);
|
||||
final PackedInts.Reader docToOrd = parent.termsIndex.getDocToOrd();
|
||||
PerSegmentComparator perSegComp = null;
|
||||
if (docToOrd.hasArray()) {
|
||||
final Object arr = docToOrd.getArray();
|
||||
if (arr instanceof byte[]) {
|
||||
perSegComp = new ByteOrdComparator((byte[]) arr, parent);
|
||||
} else if (arr instanceof short[]) {
|
||||
perSegComp = new ShortOrdComparator((short[]) arr, parent);
|
||||
} else if (arr instanceof int[]) {
|
||||
perSegComp = new IntOrdComparator((int[]) arr, parent);
|
||||
}
|
||||
}
|
||||
|
||||
if (perSegComp == null) {
|
||||
perSegComp = new AnyOrdComparator(docToOrd, parent);
|
||||
}
|
||||
PerSegmentComparator perSegComp = new AnyOrdComparator(parent);
|
||||
|
||||
if (perSegComp.bottomSlot != -1) {
|
||||
perSegComp.setBottom(perSegComp.bottomSlot);
|
||||
|
|
|
@ -355,10 +355,10 @@ public class TestFunctionQuery extends SolrTestCaseJ4 {
|
|||
|
||||
// test that ord and rord are working on a global index basis, not just
|
||||
// at the segment level (since Lucene 2.9 has switched to per-segment searching)
|
||||
assertQ(req("fl","*,score","q", "{!func}ord(id)", "fq","id:6"), "//float[@name='score']='6.0'");
|
||||
assertQ(req("fl","*,score","q", "{!func}top(ord(id))", "fq","id:6"), "//float[@name='score']='6.0'");
|
||||
assertQ(req("fl","*,score","q", "{!func}rord(id)", "fq","id:1"),"//float[@name='score']='6.0'");
|
||||
assertQ(req("fl","*,score","q", "{!func}top(rord(id))", "fq","id:1"),"//float[@name='score']='6.0'");
|
||||
assertQ(req("fl","*,score","q", "{!func}ord(id)", "fq","id:6"), "//float[@name='score']='5.0'");
|
||||
assertQ(req("fl","*,score","q", "{!func}top(ord(id))", "fq","id:6"), "//float[@name='score']='5.0'");
|
||||
assertQ(req("fl","*,score","q", "{!func}rord(id)", "fq","id:1"),"//float[@name='score']='5.0'");
|
||||
assertQ(req("fl","*,score","q", "{!func}top(rord(id))", "fq","id:1"),"//float[@name='score']='5.0'");
|
||||
|
||||
|
||||
// test that we can subtract dates to millisecond precision
|
||||
|
|
Loading…
Reference in New Issue