You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2014/06/12 21:40:38 UTC
svn commit: r1602277 [1/3] - in /lucene/dev/trunk: lucene/
lucene/codecs/src/java/org/apache/lucene/codecs/memory/
lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/
lucene/core/src/java/org/apache/lucene/codecs/
lucene/core/src/java/org/apach...
Author: rmuir
Date: Thu Jun 12 19:40:36 2014
New Revision: 1602277
URL: http://svn.apache.org/r1602277
Log:
LUCENE-5748: Add SORTED_NUMERIC docvalues type
Added:
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/SortedNumericDocValuesField.java (with props)
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SingletonSortedNumericDocValues.java (with props)
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedNumericDocValues.java (with props)
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedNumericDocValuesWriter.java (with props)
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/SortedNumericSelector.java (with props)
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/SortedNumericSortField.java (with props)
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSortedNumericSortField.java (with props)
Modified:
lucene/dev/trunk/lucene/CHANGES.txt
lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectDocValuesConsumer.java
lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectDocValuesProducer.java
lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesConsumer.java
lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java
lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java
lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsConsumer.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosWriter.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesConsumer.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesFormat.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesProducer.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsConsumer.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsProducer.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/package.html
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValues.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SimpleMergedSegmentWarmer.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/NumericUtils.java
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene45/TestLucene45DocValuesFormat.java
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOutOfMemory.java
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java
lucene/dev/trunk/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/uninverting/TestUninvertingReader.java
lucene/dev/trunk/lucene/queries/src/test/org/apache/lucene/queries/function/TestDocValuesFieldSources.java
lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java
lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyDocValuesFormat.java
lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java
lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java
lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java
lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java
lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestDocSet.java
Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Thu Jun 12 19:40:36 2014
@@ -132,6 +132,9 @@ New Features
* LUCENE-5743: Add Lucene49NormsFormat, which can compress in some cases
such as very short fields. (Ryan Ernst, Adrien Grand, Robert Muir)
+* LUCENE-5748: Add SORTED_NUMERIC docvalues type, which is efficient
+ for processing numeric fields with multiple values. (Robert Muir)
+
Changes in Backwards Compatibility Policy
* LUCENE-5634: Add reuse argument to IndexableField.tokenStream. This
Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectDocValuesConsumer.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectDocValuesConsumer.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectDocValuesConsumer.java Thu Jun 12 19:40:36 2014
@@ -32,7 +32,10 @@ import org.apache.lucene.util.IOUtils;
import static org.apache.lucene.codecs.memory.DirectDocValuesProducer.VERSION_CURRENT;
import static org.apache.lucene.codecs.memory.DirectDocValuesProducer.BYTES;
import static org.apache.lucene.codecs.memory.DirectDocValuesProducer.SORTED;
+import static org.apache.lucene.codecs.memory.DirectDocValuesProducer.SORTED_NUMERIC;
+import static org.apache.lucene.codecs.memory.DirectDocValuesProducer.SORTED_NUMERIC_SINGLETON;
import static org.apache.lucene.codecs.memory.DirectDocValuesProducer.SORTED_SET;
+import static org.apache.lucene.codecs.memory.DirectDocValuesProducer.SORTED_SET_SINGLETON;
import static org.apache.lucene.codecs.memory.DirectDocValuesProducer.NUMBER;
/**
@@ -238,67 +241,102 @@ class DirectDocValuesConsumer extends Do
// write the values as binary
addBinaryFieldValues(field, values);
}
+
+ @Override
+ public void addSortedNumericField(FieldInfo field, Iterable<Number> docToValueCount, Iterable<Number> values) throws IOException {
+ meta.writeVInt(field.number);
+ if (isSingleValued(docToValueCount)) {
+ meta.writeByte(SORTED_NUMERIC_SINGLETON);
+ addNumericFieldValues(field, singletonView(docToValueCount, values, null));
+ } else {
+ meta.writeByte(SORTED_NUMERIC);
+
+ // First write docToValueCounts, except we "aggregate" the
+ // counts so they turn into addresses, and add a final
+ // value = the total aggregate:
+ addNumericFieldValues(field, countToAddressIterator(docToValueCount));
+
+ // Write values for all docs, appended into one big
+ // numerics:
+ addNumericFieldValues(field, values);
+ }
+ }
// note: this might not be the most efficient... but its fairly simple
@Override
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
meta.writeVInt(field.number);
- meta.writeByte(SORTED_SET);
-
- // First write docToOrdCounts, except we "aggregate" the
- // counts so they turn into addresses, and add a final
- // value = the total aggregate:
- addNumericFieldValues(field, new Iterable<Number>() {
-
- // Just aggregates the count values so they become
- // "addresses", and adds one more value in the end
- // (the final sum):
-
- @Override
- public Iterator<Number> iterator() {
- final Iterator<Number> iter = docToOrdCount.iterator();
-
- return new Iterator<Number>() {
-
- long sum;
- boolean ended;
-
- @Override
- public boolean hasNext() {
- return iter.hasNext() || !ended;
- }
+
+ if (isSingleValued(docToOrdCount)) {
+ meta.writeByte(SORTED_SET_SINGLETON);
+ // Write ordinals for all docs, appended into one big
+ // numerics:
+ addNumericFieldValues(field, singletonView(docToOrdCount, ords, -1L));
+
+ // write the values as binary
+ addBinaryFieldValues(field, values);
+ } else {
+ meta.writeByte(SORTED_SET);
- @Override
- public Number next() {
- long toReturn = sum;
-
- if (iter.hasNext()) {
- Number n = iter.next();
- if (n != null) {
- sum += n.longValue();
- }
- } else if (!ended) {
- ended = true;
- } else {
- assert false;
+ // First write docToOrdCounts, except we "aggregate" the
+ // counts so they turn into addresses, and add a final
+ // value = the total aggregate:
+ addNumericFieldValues(field, countToAddressIterator(docToOrdCount));
+
+ // Write ordinals for all docs, appended into one big
+ // numerics:
+ addNumericFieldValues(field, ords);
+
+ // write the values as binary
+ addBinaryFieldValues(field, values);
+ }
+ }
+
+ /**
+ * Just aggregates the count values so they become
+ * "addresses", and adds one more value in the end
+ * (the final sum)
+ */
+ private Iterable<Number> countToAddressIterator(final Iterable<Number> counts) {
+ return new Iterable<Number>() {
+ @Override
+ public Iterator<Number> iterator() {
+ final Iterator<Number> iter = counts.iterator();
+
+ return new Iterator<Number>() {
+
+ long sum;
+ boolean ended;
+
+ @Override
+ public boolean hasNext() {
+ return iter.hasNext() || !ended;
+ }
+
+ @Override
+ public Number next() {
+ long toReturn = sum;
+
+ if (iter.hasNext()) {
+ Number n = iter.next();
+ if (n != null) {
+ sum += n.longValue();
}
-
- return toReturn;
+ } else if (!ended) {
+ ended = true;
+ } else {
+ assert false;
}
-
- @Override
- public void remove() {
- throw new UnsupportedOperationException();
- }
- };
- }
- });
-
- // Write ordinals for all docs, appended into one big
- // numerics:
- addNumericFieldValues(field, ords);
-
- // write the values as binary
- addBinaryFieldValues(field, values);
+
+ return toReturn;
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+ };
+ }
+ };
}
}
Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectDocValuesProducer.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectDocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectDocValuesProducer.java Thu Jun 12 19:40:36 2014
@@ -33,6 +33,7 @@ import org.apache.lucene.index.NumericDo
import org.apache.lucene.index.RandomAccessOrds;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IndexInput;
@@ -52,6 +53,7 @@ class DirectDocValuesProducer extends Do
private final Map<Integer,BinaryEntry> binaries = new HashMap<>();
private final Map<Integer,SortedEntry> sorteds = new HashMap<>();
private final Map<Integer,SortedSetEntry> sortedSets = new HashMap<>();
+ private final Map<Integer,SortedNumericEntry> sortedNumerics = new HashMap<>();
private final IndexInput data;
// ram instances we have already loaded
@@ -63,6 +65,8 @@ class DirectDocValuesProducer extends Do
new HashMap<>();
private final Map<Integer,SortedSetRawValues> sortedSetInstances =
new HashMap<>();
+ private final Map<Integer,SortedNumericRawValues> sortedNumericInstances =
+ new HashMap<>();
private final Map<Integer,Bits> docsWithFieldInstances = new HashMap<>();
private final int maxDoc;
@@ -73,10 +77,12 @@ class DirectDocValuesProducer extends Do
static final byte BYTES = 1;
static final byte SORTED = 2;
static final byte SORTED_SET = 3;
+ static final byte SORTED_SET_SINGLETON = 4;
+ static final byte SORTED_NUMERIC = 5;
+ static final byte SORTED_NUMERIC_SINGLETON = 6;
- static final int VERSION_START = 0;
- static final int VERSION_CHECKSUM = 1;
- static final int VERSION_CURRENT = VERSION_CHECKSUM;
+ static final int VERSION_START = 2;
+ static final int VERSION_CURRENT = VERSION_START;
DirectDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
maxDoc = state.segmentInfo.getDocCount();
@@ -91,11 +97,7 @@ class DirectDocValuesProducer extends Do
VERSION_CURRENT);
readFields(in);
- if (version >= VERSION_CHECKSUM) {
- CodecUtil.checkFooter(in);
- } else {
- CodecUtil.checkEOF(in);
- }
+ CodecUtil.checkFooter(in);
success = true;
} finally {
if (success) {
@@ -161,13 +163,24 @@ class DirectDocValuesProducer extends Do
return entry;
}
- private SortedSetEntry readSortedSetEntry(IndexInput meta) throws IOException {
+ private SortedSetEntry readSortedSetEntry(IndexInput meta, boolean singleton) throws IOException {
SortedSetEntry entry = new SortedSetEntry();
- entry.docToOrdAddress = readNumericEntry(meta);
+ if (singleton == false) {
+ entry.docToOrdAddress = readNumericEntry(meta);
+ }
entry.ords = readNumericEntry(meta);
entry.values = readBinaryEntry(meta);
return entry;
}
+
+ private SortedNumericEntry readSortedNumericEntry(IndexInput meta, boolean singleton) throws IOException {
+ SortedNumericEntry entry = new SortedNumericEntry();
+ if (singleton == false) {
+ entry.docToAddress = readNumericEntry(meta);
+ }
+ entry.values = readNumericEntry(meta);
+ return entry;
+ }
private void readFields(IndexInput meta) throws IOException {
int fieldNumber = meta.readVInt();
@@ -182,9 +195,19 @@ class DirectDocValuesProducer extends Do
sorteds.put(fieldNumber, entry);
binaries.put(fieldNumber, entry.values);
} else if (fieldType == SORTED_SET) {
- SortedSetEntry entry = readSortedSetEntry(meta);
+ SortedSetEntry entry = readSortedSetEntry(meta, false);
sortedSets.put(fieldNumber, entry);
binaries.put(fieldNumber, entry.values);
+ } else if (fieldType == SORTED_SET_SINGLETON) {
+ SortedSetEntry entry = readSortedSetEntry(meta, true);
+ sortedSets.put(fieldNumber, entry);
+ binaries.put(fieldNumber, entry.values);
+ } else if (fieldType == SORTED_NUMERIC) {
+ SortedNumericEntry entry = readSortedNumericEntry(meta, false);
+ sortedNumerics.put(fieldNumber, entry);
+ } else if (fieldType == SORTED_NUMERIC_SINGLETON) {
+ SortedNumericEntry entry = readSortedNumericEntry(meta, true);
+ sortedNumerics.put(fieldNumber, entry);
} else {
throw new CorruptIndexException("invalid entry type: " + fieldType + ", input=" + meta);
}
@@ -199,9 +222,7 @@ class DirectDocValuesProducer extends Do
@Override
public void checkIntegrity() throws IOException {
- if (version >= VERSION_CHECKSUM) {
- CodecUtil.checksumEntireFile(data);
- }
+ CodecUtil.checksumEntireFile(data);
}
@Override
@@ -337,9 +358,10 @@ class DirectDocValuesProducer extends Do
sortedInstances.put(field.number, instance);
}
}
- final NumericDocValues docToOrd = instance.docToOrd;
- final BinaryDocValues values = getBinary(field);
-
+ return newSortedInstance(instance.docToOrd, getBinary(field), entry.values.count);
+ }
+
+ private SortedDocValues newSortedInstance(final NumericDocValues docToOrd, final BinaryDocValues values, final int count) {
return new SortedDocValues() {
@Override
@@ -354,7 +376,7 @@ class DirectDocValuesProducer extends Do
@Override
public int getValueCount() {
- return entry.values.count;
+ return count;
}
// Leave lookupTerm to super's binary search
@@ -372,6 +394,56 @@ class DirectDocValuesProducer extends Do
}
@Override
+ public synchronized SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+ SortedNumericRawValues instance = sortedNumericInstances.get(field.number);
+ final SortedNumericEntry entry = sortedNumerics.get(field.number);
+ if (instance == null) {
+ // Lazy load
+ instance = loadSortedNumeric(entry);
+ sortedNumericInstances.put(field.number, instance);
+ }
+
+ if (entry.docToAddress == null) {
+ final NumericDocValues single = instance.values;
+ final Bits docsWithField = getMissingBits(field.number, entry.values.missingOffset, entry.values.missingBytes);
+ return DocValues.singleton(single, docsWithField);
+ } else {
+ final NumericDocValues docToAddress = instance.docToAddress;
+ final NumericDocValues values = instance.values;
+
+ return new SortedNumericDocValues() {
+ int valueStart;
+ int valueLimit;
+
+ @Override
+ public void setDocument(int doc) {
+ valueStart = (int) docToAddress.get(doc);
+ valueLimit = (int) docToAddress.get(doc+1);
+ }
+
+ @Override
+ public long valueAt(int index) {
+ return values.get(valueStart + index);
+ }
+
+ @Override
+ public int count() {
+ return valueLimit - valueStart;
+ }
+ };
+ }
+ }
+
+ private SortedNumericRawValues loadSortedNumeric(SortedNumericEntry entry) throws IOException {
+ SortedNumericRawValues instance = new SortedNumericRawValues();
+ if (entry.docToAddress != null) {
+ instance.docToAddress = loadNumeric(entry.docToAddress);
+ }
+ instance.values = loadNumeric(entry.values);
+ return instance;
+ }
+
+ @Override
public synchronized SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
SortedSetRawValues instance = sortedSetInstances.get(field.number);
final SortedSetEntry entry = sortedSets.get(field.number);
@@ -381,60 +453,67 @@ class DirectDocValuesProducer extends Do
sortedSetInstances.put(field.number, instance);
}
- final NumericDocValues docToOrdAddress = instance.docToOrdAddress;
- final NumericDocValues ords = instance.ords;
- final BinaryDocValues values = getBinary(field);
-
- // Must make a new instance since the iterator has state:
- return new RandomAccessOrds() {
- int ordStart;
- int ordUpto;
- int ordLimit;
-
- @Override
- public long nextOrd() {
- if (ordUpto == ordLimit) {
- return NO_MORE_ORDS;
- } else {
- return ords.get(ordUpto++);
- }
- }
+ if (instance.docToOrdAddress == null) {
+ SortedDocValues sorted = newSortedInstance(instance.ords, getBinary(field), entry.values.count);
+ return DocValues.singleton(sorted);
+ } else {
+ final NumericDocValues docToOrdAddress = instance.docToOrdAddress;
+ final NumericDocValues ords = instance.ords;
+ final BinaryDocValues values = getBinary(field);
- @Override
- public void setDocument(int docID) {
- ordStart = ordUpto = (int) docToOrdAddress.get(docID);
- ordLimit = (int) docToOrdAddress.get(docID+1);
- }
-
- @Override
- public BytesRef lookupOrd(long ord) {
- return values.get((int) ord);
- }
-
- @Override
- public long getValueCount() {
- return entry.values.count;
- }
-
- @Override
- public long ordAt(int index) {
- return ords.get(ordStart + index);
- }
-
- @Override
- public int cardinality() {
- return ordLimit - ordStart;
- }
-
- // Leave lookupTerm to super's binary search
-
- // Leave termsEnum to super
- };
+ // Must make a new instance since the iterator has state:
+ return new RandomAccessOrds() {
+ int ordStart;
+ int ordUpto;
+ int ordLimit;
+
+ @Override
+ public long nextOrd() {
+ if (ordUpto == ordLimit) {
+ return NO_MORE_ORDS;
+ } else {
+ return ords.get(ordUpto++);
+ }
+ }
+
+ @Override
+ public void setDocument(int docID) {
+ ordStart = ordUpto = (int) docToOrdAddress.get(docID);
+ ordLimit = (int) docToOrdAddress.get(docID+1);
+ }
+
+ @Override
+ public BytesRef lookupOrd(long ord) {
+ return values.get((int) ord);
+ }
+
+ @Override
+ public long getValueCount() {
+ return entry.values.count;
+ }
+
+ @Override
+ public long ordAt(int index) {
+ return ords.get(ordStart + index);
+ }
+
+ @Override
+ public int cardinality() {
+ return ordLimit - ordStart;
+ }
+
+ // Leave lookupTerm to super's binary search
+
+ // Leave termsEnum to super
+ };
+ }
}
private SortedSetRawValues loadSortedSet(SortedSetEntry entry) throws IOException {
SortedSetRawValues instance = new SortedSetRawValues();
- instance.docToOrdAddress = loadNumeric(entry.docToOrdAddress);
+ if (entry.docToOrdAddress != null) {
+ instance.docToOrdAddress = loadNumeric(entry.docToOrdAddress);
+ }
instance.ords = loadNumeric(entry.ords);
return instance;
}
@@ -467,6 +546,8 @@ class DirectDocValuesProducer extends Do
switch(field.getDocValuesType()) {
case SORTED_SET:
return DocValues.docsWithValue(getSortedSet(field), maxDoc);
+ case SORTED_NUMERIC:
+ return DocValues.docsWithValue(getSortedNumeric(field), maxDoc);
case SORTED:
return DocValues.docsWithValue(getSorted(field), maxDoc);
case BINARY:
@@ -493,6 +574,11 @@ class DirectDocValuesProducer extends Do
static class SortedRawValues {
NumericDocValues docToOrd;
}
+
+ static class SortedNumericRawValues {
+ NumericDocValues docToAddress;
+ NumericDocValues values;
+ }
static class SortedSetRawValues {
NumericDocValues docToOrdAddress;
@@ -531,6 +617,11 @@ class DirectDocValuesProducer extends Do
BinaryEntry values;
}
+ static class SortedNumericEntry {
+ NumericEntry docToAddress;
+ NumericEntry values;
+ }
+
static class FSTEntry {
long offset;
long numOrds;
Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesConsumer.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesConsumer.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesConsumer.java Thu Jun 12 19:40:36 2014
@@ -50,6 +50,10 @@ import static org.apache.lucene.codecs.m
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.BYTES;
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.NUMBER;
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.FST;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.SORTED_SET;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.SORTED_SET_SINGLETON;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.SORTED_NUMERIC;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.SORTED_NUMERIC_SINGLETON;
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.DELTA_COMPRESSED;
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.BLOCK_COMPRESSED;
import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.GCD_COMPRESSED;
@@ -98,10 +102,11 @@ class MemoryDocValuesConsumer extends Do
boolean missing = false;
// TODO: more efficient?
HashSet<Long> uniqueValues = null;
+ long count = 0;
+
if (optimizeStorage) {
uniqueValues = new HashSet<>();
- long count = 0;
long currentBlockMin = Long.MAX_VALUE;
long currentBlockMax = Long.MIN_VALUE;
for (Number nv : values) {
@@ -148,12 +153,12 @@ class MemoryDocValuesConsumer extends Do
currentBlockMin = Long.MAX_VALUE;
}
}
- assert count == maxDoc;
} else {
for (Number nv : values) {
long v = nv.longValue();
maxValue = Math.max(v, maxValue);
minValue = Math.min(v, minValue);
+ count++;
}
}
@@ -171,14 +176,14 @@ class MemoryDocValuesConsumer extends Do
final FormatAndBits deltaBPV = PackedInts.fastestFormatAndBits(maxDoc, deltaRequired, acceptableOverheadRatio);
final FormatAndBits tableBPV;
- if (uniqueValues != null) {
+ if (count < Integer.MAX_VALUE && uniqueValues != null) {
tableBPV = PackedInts.fastestFormatAndBits(maxDoc, PackedInts.bitsRequired(uniqueValues.size()-1), acceptableOverheadRatio);
} else {
tableBPV = null;
}
final FormatAndBits gcdBPV;
- if (gcd != 0 && gcd != 1) {
+ if (count < Integer.MAX_VALUE && gcd != 0 && gcd != 1) {
final long gcdDelta = (maxValue - minValue) / gcd;
final int gcdRequired = gcdDelta < 0 ? 64 : PackedInts.bitsRequired(gcdDelta);
gcdBPV = PackedInts.fastestFormatAndBits(maxDoc, gcdRequired, acceptableOverheadRatio);
@@ -196,6 +201,10 @@ class MemoryDocValuesConsumer extends Do
doBlock = true;
}
}
+ // blockpackedreader allows us to read in huge streams of ints
+ if (count >= Integer.MAX_VALUE) {
+ doBlock = true;
+ }
if (tableBPV != null && (tableBPV.bitsPerValue+tableBPV.bitsPerValue*acceptableOverheadRatio) < deltaBPV.bitsPerValue) {
// small number of unique values
@@ -213,10 +222,11 @@ class MemoryDocValuesConsumer extends Do
}
meta.writeVInt(PackedInts.VERSION_CURRENT);
+ meta.writeLong(count);
data.writeVInt(tableBPV.format.getId());
data.writeVInt(tableBPV.bitsPerValue);
- final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, tableBPV.format, maxDoc, tableBPV.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
+ final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, tableBPV.format, (int)count, tableBPV.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
for(Number nv : values) {
writer.add(encode.get(nv == null ? 0 : nv.longValue()));
}
@@ -224,12 +234,13 @@ class MemoryDocValuesConsumer extends Do
} else if (gcdBPV != null && (gcdBPV.bitsPerValue+gcdBPV.bitsPerValue*acceptableOverheadRatio) < deltaBPV.bitsPerValue) {
meta.writeByte(GCD_COMPRESSED);
meta.writeVInt(PackedInts.VERSION_CURRENT);
+ meta.writeLong(count);
data.writeLong(minValue);
data.writeLong(gcd);
data.writeVInt(gcdBPV.format.getId());
data.writeVInt(gcdBPV.bitsPerValue);
- final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, gcdBPV.format, maxDoc, gcdBPV.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
+ final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, gcdBPV.format, (int)count, gcdBPV.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
for (Number nv : values) {
long value = nv == null ? 0 : nv.longValue();
writer.add((value - minValue) / gcd);
@@ -238,6 +249,7 @@ class MemoryDocValuesConsumer extends Do
} else if (doBlock) {
meta.writeByte(BLOCK_COMPRESSED); // block delta-compressed
meta.writeVInt(PackedInts.VERSION_CURRENT);
+ meta.writeLong(count);
data.writeVInt(BLOCK_SIZE);
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
for (Number nv : values) {
@@ -247,12 +259,13 @@ class MemoryDocValuesConsumer extends Do
} else {
meta.writeByte(DELTA_COMPRESSED); // delta-compressed
meta.writeVInt(PackedInts.VERSION_CURRENT);
+ meta.writeLong(count);
final long minDelta = deltaBPV.bitsPerValue == 64 ? 0 : minValue;
data.writeLong(minDelta);
data.writeVInt(deltaBPV.format.getId());
data.writeVInt(deltaBPV.bitsPerValue);
- final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, deltaBPV.format, maxDoc, deltaBPV.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
+ final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, deltaBPV.format, (int)count, deltaBPV.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
for (Number nv : values) {
long v = nv == null ? 0 : nv.longValue();
writer.add(v - minDelta);
@@ -388,20 +401,58 @@ class MemoryDocValuesConsumer extends Do
// write the values as FST
writeFST(field, values);
}
+
+ @Override
+ public void addSortedNumericField(FieldInfo field, Iterable<Number> docToValueCount, Iterable<Number> values) throws IOException {
+ meta.writeVInt(field.number);
+
+ if (isSingleValued(docToValueCount)) {
+ meta.writeByte(SORTED_NUMERIC_SINGLETON);
+ addNumericField(field, singletonView(docToValueCount, values, null), true);
+ } else {
+ meta.writeByte(SORTED_NUMERIC);
+
+ // write the addresses:
+ meta.writeVInt(PackedInts.VERSION_CURRENT);
+ meta.writeVInt(BLOCK_SIZE);
+ meta.writeLong(data.getFilePointer());
+ final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
+ long addr = 0;
+ writer.add(addr);
+ for (Number v : docToValueCount) {
+ addr += v.longValue();
+ writer.add(addr);
+ }
+ writer.finish();
+ long valueCount = writer.ord();
+ meta.writeLong(valueCount);
+
+ // write the values
+ addNumericField(field, values, true);
+ }
+ }
// note: this might not be the most efficient... but its fairly simple
@Override
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
- // write the ordinals as a binary field
- addBinaryField(field, new Iterable<BytesRef>() {
- @Override
- public Iterator<BytesRef> iterator() {
- return new SortedSetIterator(docToOrdCount.iterator(), ords.iterator());
- }
- });
+ meta.writeVInt(field.number);
+
+ if (isSingleValued(docToOrdCount)) {
+ meta.writeByte(SORTED_SET_SINGLETON);
+ addSortedField(field, values, singletonView(docToOrdCount, ords, -1L));
+ } else {
+ meta.writeByte(SORTED_SET);
+ // write the ordinals as a binary field
+ addBinaryField(field, new Iterable<BytesRef>() {
+ @Override
+ public Iterator<BytesRef> iterator() {
+ return new SortedSetIterator(docToOrdCount.iterator(), ords.iterator());
+ }
+ });
- // write the values as FST
- writeFST(field, values);
+ // write the values as FST
+ writeFST(field, values);
+ }
}
// per-document vint-encoded byte[]
Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesProducer.java Thu Jun 12 19:40:36 2014
@@ -35,6 +35,7 @@ import org.apache.lucene.index.IndexFile
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ByteArrayDataInput;
@@ -45,6 +46,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.LongValues;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.fst.BytesRefFSTEnum;
@@ -63,9 +65,11 @@ import org.apache.lucene.util.packed.Pac
*/
class MemoryDocValuesProducer extends DocValuesProducer {
// metadata maps (just file pointers and minimal stuff)
- private final Map<Integer,NumericEntry> numerics;
- private final Map<Integer,BinaryEntry> binaries;
- private final Map<Integer,FSTEntry> fsts;
+ private final Map<Integer,NumericEntry> numerics = new HashMap<>();
+ private final Map<Integer,BinaryEntry> binaries = new HashMap<>();
+ private final Map<Integer,FSTEntry> fsts = new HashMap<>();
+ private final Map<Integer,SortedSetEntry> sortedSets = new HashMap<>();
+ private final Map<Integer,SortedNumericEntry> sortedNumerics = new HashMap<>();
private final IndexInput data;
// ram instances we have already loaded
@@ -76,6 +80,7 @@ class MemoryDocValuesProducer extends Do
private final Map<Integer,FST<Long>> fstInstances =
new HashMap<>();
private final Map<Integer,Bits> docsWithFieldInstances = new HashMap<>();
+ private final Map<Integer,MonotonicBlockPackedReader> addresses = new HashMap<>();
private final int maxDoc;
private final AtomicLong ramBytesUsed;
@@ -84,6 +89,10 @@ class MemoryDocValuesProducer extends Do
static final byte NUMBER = 0;
static final byte BYTES = 1;
static final byte FST = 2;
+ static final byte SORTED_SET = 4;
+ static final byte SORTED_SET_SINGLETON = 5;
+ static final byte SORTED_NUMERIC = 6;
+ static final byte SORTED_NUMERIC_SINGLETON = 7;
static final int BLOCK_SIZE = 4096;
@@ -92,11 +101,8 @@ class MemoryDocValuesProducer extends Do
static final byte BLOCK_COMPRESSED = 2;
static final byte GCD_COMPRESSED = 3;
- static final int VERSION_START = 0;
- static final int VERSION_GCD_COMPRESSION = 1;
- static final int VERSION_CHECKSUM = 2;
- static final int VERSION_BLOCKDETECTION = 3;
- static final int VERSION_CURRENT = VERSION_BLOCKDETECTION;
+ static final int VERSION_START = 3;
+ static final int VERSION_CURRENT = VERSION_START;
MemoryDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
maxDoc = state.segmentInfo.getDocCount();
@@ -108,15 +114,8 @@ class MemoryDocValuesProducer extends Do
version = CodecUtil.checkHeader(in, metaCodec,
VERSION_START,
VERSION_CURRENT);
- numerics = new HashMap<>();
- binaries = new HashMap<>();
- fsts = new HashMap<>();
readFields(in, state.fieldInfos);
- if (version >= VERSION_CHECKSUM) {
- CodecUtil.checkFooter(in);
- } else {
- CodecUtil.checkEOF(in);
- }
+ CodecUtil.checkFooter(in);
ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
success = true;
} finally {
@@ -146,53 +145,86 @@ class MemoryDocValuesProducer extends Do
}
}
+ private NumericEntry readNumericEntry(IndexInput meta) throws IOException {
+ NumericEntry entry = new NumericEntry();
+ entry.offset = meta.readLong();
+ entry.missingOffset = meta.readLong();
+ if (entry.missingOffset != -1) {
+ entry.missingBytes = meta.readLong();
+ } else {
+ entry.missingBytes = 0;
+ }
+ entry.format = meta.readByte();
+ switch(entry.format) {
+ case DELTA_COMPRESSED:
+ case TABLE_COMPRESSED:
+ case BLOCK_COMPRESSED:
+ case GCD_COMPRESSED:
+ break;
+ default:
+ throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
+ }
+ entry.packedIntsVersion = meta.readVInt();
+ entry.count = meta.readLong();
+ return entry;
+ }
+
+ private BinaryEntry readBinaryEntry(IndexInput meta) throws IOException {
+ BinaryEntry entry = new BinaryEntry();
+ entry.offset = meta.readLong();
+ entry.numBytes = meta.readLong();
+ entry.missingOffset = meta.readLong();
+ if (entry.missingOffset != -1) {
+ entry.missingBytes = meta.readLong();
+ } else {
+ entry.missingBytes = 0;
+ }
+ entry.minLength = meta.readVInt();
+ entry.maxLength = meta.readVInt();
+ if (entry.minLength != entry.maxLength) {
+ entry.packedIntsVersion = meta.readVInt();
+ entry.blockSize = meta.readVInt();
+ }
+ return entry;
+ }
+
+ private FSTEntry readFSTEntry(IndexInput meta) throws IOException {
+ FSTEntry entry = new FSTEntry();
+ entry.offset = meta.readLong();
+ entry.numOrds = meta.readVLong();
+ return entry;
+ }
+
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
int fieldNumber = meta.readVInt();
while (fieldNumber != -1) {
int fieldType = meta.readByte();
if (fieldType == NUMBER) {
- NumericEntry entry = new NumericEntry();
- entry.offset = meta.readLong();
- entry.missingOffset = meta.readLong();
- if (entry.missingOffset != -1) {
- entry.missingBytes = meta.readLong();
- } else {
- entry.missingBytes = 0;
- }
- entry.format = meta.readByte();
- switch(entry.format) {
- case DELTA_COMPRESSED:
- case TABLE_COMPRESSED:
- case BLOCK_COMPRESSED:
- case GCD_COMPRESSED:
- break;
- default:
- throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
- }
- entry.packedIntsVersion = meta.readVInt();
- numerics.put(fieldNumber, entry);
+ numerics.put(fieldNumber, readNumericEntry(meta));
} else if (fieldType == BYTES) {
- BinaryEntry entry = new BinaryEntry();
- entry.offset = meta.readLong();
- entry.numBytes = meta.readLong();
- entry.missingOffset = meta.readLong();
- if (entry.missingOffset != -1) {
- entry.missingBytes = meta.readLong();
- } else {
- entry.missingBytes = 0;
- }
- entry.minLength = meta.readVInt();
- entry.maxLength = meta.readVInt();
- if (entry.minLength != entry.maxLength) {
- entry.packedIntsVersion = meta.readVInt();
- entry.blockSize = meta.readVInt();
- }
- binaries.put(fieldNumber, entry);
+ binaries.put(fieldNumber, readBinaryEntry(meta));
} else if (fieldType == FST) {
- FSTEntry entry = new FSTEntry();
- entry.offset = meta.readLong();
- entry.numOrds = meta.readVLong();
- fsts.put(fieldNumber, entry);
+ fsts.put(fieldNumber,readFSTEntry(meta));
+ } else if (fieldType == SORTED_SET) {
+ SortedSetEntry entry = new SortedSetEntry();
+ entry.singleton = false;
+ sortedSets.put(fieldNumber, entry);
+ } else if (fieldType == SORTED_SET_SINGLETON) {
+ SortedSetEntry entry = new SortedSetEntry();
+ entry.singleton = true;
+ sortedSets.put(fieldNumber, entry);
+ } else if (fieldType == SORTED_NUMERIC) {
+ SortedNumericEntry entry = new SortedNumericEntry();
+ entry.singleton = false;
+ entry.packedIntsVersion = meta.readVInt();
+ entry.blockSize = meta.readVInt();
+ entry.addressOffset = meta.readLong();
+ entry.valueCount = meta.readLong();
+ sortedNumerics.put(fieldNumber, entry);
+ } else if (fieldType == SORTED_NUMERIC_SINGLETON) {
+ SortedNumericEntry entry = new SortedNumericEntry();
+ entry.singleton = true;
+ sortedNumerics.put(fieldNumber, entry);
} else {
throw new CorruptIndexException("invalid entry type: " + fieldType + ", input=" + meta);
}
@@ -217,9 +249,7 @@ class MemoryDocValuesProducer extends Do
@Override
public void checkIntegrity() throws IOException {
- if (version >= VERSION_CHECKSUM) {
- CodecUtil.checksumEntireFile(data);
- }
+ CodecUtil.checksumEntireFile(data);
}
private NumericDocValues loadNumeric(FieldInfo field) throws IOException {
@@ -237,7 +267,7 @@ class MemoryDocValuesProducer extends Do
}
final int formatID = data.readVInt();
final int bitsPerValue = data.readVInt();
- final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), entry.packedIntsVersion, maxDoc, bitsPerValue);
+ final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), entry.packedIntsVersion, (int)entry.count, bitsPerValue);
ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed());
return new NumericDocValues() {
@Override
@@ -249,7 +279,7 @@ class MemoryDocValuesProducer extends Do
final long minDelta = data.readLong();
final int formatIDDelta = data.readVInt();
final int bitsPerValueDelta = data.readVInt();
- final PackedInts.Reader deltaReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatIDDelta), entry.packedIntsVersion, maxDoc, bitsPerValueDelta);
+ final PackedInts.Reader deltaReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatIDDelta), entry.packedIntsVersion, (int)entry.count, bitsPerValueDelta);
ramBytesUsed.addAndGet(deltaReader.ramBytesUsed());
return new NumericDocValues() {
@Override
@@ -259,7 +289,7 @@ class MemoryDocValuesProducer extends Do
};
case BLOCK_COMPRESSED:
final int blockSize = data.readVInt();
- final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, maxDoc, false);
+ final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, entry.count, false);
ramBytesUsed.addAndGet(reader.ramBytesUsed());
return reader;
case GCD_COMPRESSED:
@@ -267,7 +297,7 @@ class MemoryDocValuesProducer extends Do
final long mult = data.readLong();
final int formatIDGCD = data.readVInt();
final int bitsPerValueGCD = data.readVInt();
- final PackedInts.Reader quotientReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatIDGCD), entry.packedIntsVersion, maxDoc, bitsPerValueGCD);
+ final PackedInts.Reader quotientReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatIDGCD), entry.packedIntsVersion, (int)entry.count, bitsPerValueGCD);
ramBytesUsed.addAndGet(quotientReader.ramBytesUsed());
return new NumericDocValues() {
@Override
@@ -414,7 +444,80 @@ class MemoryDocValuesProducer extends Do
}
@Override
+ public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+ SortedNumericEntry entry = sortedNumerics.get(field.number);
+ if (entry.singleton) {
+ NumericDocValues values = getNumeric(field);
+ NumericEntry ne = numerics.get(field.number);
+ Bits docsWithField = getMissingBits(field.number, ne.missingOffset, ne.missingBytes);
+ return DocValues.singleton(values, docsWithField);
+ } else {
+ final NumericDocValues values = getNumeric(field);
+ final MonotonicBlockPackedReader addr;
+ synchronized (this) {
+ MonotonicBlockPackedReader res = addresses.get(field.number);
+ if (res == null) {
+ data.seek(entry.addressOffset);
+ res = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.valueCount, false);
+ addresses.put(field.number, res);
+ }
+ addr = res;
+ }
+ if (values instanceof LongValues) {
+ // probably not the greatest codec choice for this situation, but we support it
+ final LongValues longValues = (LongValues) values;
+ return new SortedNumericDocValues() {
+ long startOffset;
+ long endOffset;
+
+ @Override
+ public void setDocument(int doc) {
+ startOffset = (int) addr.get(doc);
+ endOffset = (int) addr.get(doc+1L);
+ }
+
+ @Override
+ public long valueAt(int index) {
+ return longValues.get(startOffset + index);
+ }
+
+ @Override
+ public int count() {
+ return (int) (endOffset - startOffset);
+ }
+ };
+ } else {
+ return new SortedNumericDocValues() {
+ int startOffset;
+ int endOffset;
+
+ @Override
+ public void setDocument(int doc) {
+ startOffset = (int) addr.get(doc);
+ endOffset = (int) addr.get(doc+1);
+ }
+
+ @Override
+ public long valueAt(int index) {
+ return values.get(startOffset + index);
+ }
+
+ @Override
+ public int count() {
+ return (endOffset - startOffset);
+ }
+ };
+ }
+ }
+ }
+
+ @Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+ SortedSetEntry sortedSetEntry = sortedSets.get(field.number);
+ if (sortedSetEntry.singleton) {
+ return DocValues.singleton(getSorted(field));
+ }
+
final FSTEntry entry = fsts.get(field.number);
if (entry.numOrds == 0) {
return DocValues.emptySortedSet(); // empty FST!
@@ -530,6 +633,8 @@ class MemoryDocValuesProducer extends Do
switch(field.getDocValuesType()) {
case SORTED_SET:
return DocValues.docsWithValue(getSortedSet(field), maxDoc);
+ case SORTED_NUMERIC:
+ return DocValues.docsWithValue(getSortedNumeric(field), maxDoc);
case SORTED:
return DocValues.docsWithValue(getSorted(field), maxDoc);
case BINARY:
@@ -550,6 +655,7 @@ class MemoryDocValuesProducer extends Do
static class NumericEntry {
long offset;
+ long count;
long missingOffset;
long missingBytes;
byte format;
@@ -571,6 +677,18 @@ class MemoryDocValuesProducer extends Do
long offset;
long numOrds;
}
+
+ static class SortedSetEntry {
+ boolean singleton;
+ }
+
+ static class SortedNumericEntry {
+ boolean singleton;
+ long addressOffset;
+ int packedIntsVersion;
+ int blockSize;
+ long valueCount;
+ }
static class BytesAndAddresses {
PagedBytes.Reader reader;
Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java Thu Jun 12 19:40:36 2014
@@ -114,6 +114,9 @@ import org.apache.lucene.index.SegmentWr
* this is a comma-separated list, and its padded with spaces to be fixed width. so trim() and split() it.
* and beware the empty string!
* an ord's value can be retrieved by seeking to startOffset + (9+pattern.length+maxlength)*ord
+ *
+ * for sorted numerics, its encoded (not very creatively) as a comma-separated list of strings the same as binary.
+ * beware the empty string!
*
* the reader can just scan this file when it opens, skipping over the data blocks
* and saving the offset/etc for each field.
Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java Thu Jun 12 19:40:36 2014
@@ -48,6 +48,7 @@ import org.apache.lucene.index.IndexFile
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.store.BufferedChecksumIndexInput;
import org.apache.lucene.store.ChecksumIndexInput;
@@ -347,6 +348,38 @@ class SimpleTextDocValuesReader extends
}
};
}
+
+ @Override
+ public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+ final BinaryDocValues binary = getBinary(field);
+ return new SortedNumericDocValues() {
+ long values[];
+
+ @Override
+ public void setDocument(int doc) {
+ String csv = binary.get(doc).utf8ToString();
+ if (csv.length() == 0) {
+ values = new long[0];
+ } else {
+ String s[] = csv.split(",");
+ values = new long[s.length];
+ for (int i = 0; i < values.length; i++) {
+ values[i] = Long.parseLong(s[i]);
+ }
+ }
+ }
+
+ @Override
+ public long valueAt(int index) {
+ return values[index];
+ }
+
+ @Override
+ public int count() {
+ return values.length;
+ }
+ };
+ }
@Override
public SortedSetDocValues getSortedSet(FieldInfo fieldInfo) throws IOException {
@@ -431,6 +464,8 @@ class SimpleTextDocValuesReader extends
switch (field.getDocValuesType()) {
case SORTED_SET:
return DocValues.docsWithValue(getSortedSet(field), maxDoc);
+ case SORTED_NUMERIC:
+ return DocValues.docsWithValue(getSortedNumeric(field), maxDoc);
case SORTED:
return DocValues.docsWithValue(getSorted(field), maxDoc);
case BINARY:
Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java Thu Jun 12 19:40:36 2014
@@ -136,6 +136,10 @@ class SimpleTextDocValuesWriter extends
public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
assert fieldSeen(field.name);
assert field.getDocValuesType() == DocValuesType.BINARY;
+ doAddBinary(field, values);
+ }
+
+ private void doAddBinary(FieldInfo field, Iterable<BytesRef> values) throws IOException {
int maxLength = 0;
for(BytesRef value : values) {
final int length = value == null ? 0 : value.length;
@@ -268,6 +272,48 @@ class SimpleTextDocValuesWriter extends
}
@Override
+ public void addSortedNumericField(FieldInfo field, final Iterable<Number> docToValueCount, final Iterable<Number> values) throws IOException {
+ assert fieldSeen(field.name);
+ assert field.getDocValuesType() == DocValuesType.SORTED_NUMERIC;
+ doAddBinary(field, new Iterable<BytesRef>() {
+ @Override
+ public Iterator<BytesRef> iterator() {
+ final StringBuilder builder = new StringBuilder();
+ final BytesRef scratch = new BytesRef();
+ final Iterator<Number> counts = docToValueCount.iterator();
+ final Iterator<Number> numbers = values.iterator();
+
+ return new Iterator<BytesRef>() {
+
+ @Override
+ public boolean hasNext() {
+ return counts.hasNext();
+ }
+
+ @Override
+ public BytesRef next() {
+ builder.setLength(0);
+ long count = counts.next().longValue();
+ for (int i = 0; i < count; i++) {
+ if (i > 0) {
+ builder.append(',');
+ }
+ builder.append(Long.toString(numbers.next().longValue()));
+ }
+ scratch.copyChars(builder);
+ return scratch;
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+ };
+ }
+ });
+ }
+
+ @Override
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
assert fieldSeen(field.name);
assert field.getDocValuesType() == DocValuesType.SORTED_SET;
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java Thu Jun 12 19:40:36 2014
@@ -32,6 +32,7 @@ import org.apache.lucene.index.MultiDocV
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.ArrayUtil;
@@ -93,6 +94,16 @@ public abstract class DocValuesConsumer
* @throws IOException if an I/O error occurred.
*/
public abstract void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException;
+
+ /**
+ * Writes pre-sorted numeric docvalues for a field
+ * @param field field information
+ * @param docToValueCount Iterable of the number of values for each document. A zero
+ * count indicates a missing value.
+ * @param values Iterable of numeric values in sorted order (not deduplicated).
+ * @throws IOException if an I/O error occurred.
+ */
+ public abstract void addSortedNumericField(FieldInfo field, Iterable<Number> docToValueCount, Iterable<Number> values) throws IOException;
/**
* Writes pre-sorted set docvalues for a field
@@ -265,6 +276,156 @@ public abstract class DocValuesConsumer
});
}
+ /**
+ * Merges the sorted docvalues from <code>toMerge</code>.
+ * <p>
+ * The default implementation calls {@link #addSortedNumericField}, passing
+ * iterables that filter deleted documents.
+ */
+ public void mergeSortedNumericField(FieldInfo fieldInfo, final MergeState mergeState, List<SortedNumericDocValues> toMerge) throws IOException {
+ final AtomicReader readers[] = mergeState.readers.toArray(new AtomicReader[toMerge.size()]);
+ final SortedNumericDocValues dvs[] = toMerge.toArray(new SortedNumericDocValues[toMerge.size()]);
+
+ // step 3: add field
+ addSortedNumericField(fieldInfo,
+ // doc -> value count
+ new Iterable<Number>() {
+ @Override
+ public Iterator<Number> iterator() {
+ return new Iterator<Number>() {
+ int readerUpto = -1;
+ int docIDUpto;
+ int nextValue;
+ AtomicReader currentReader;
+ Bits currentLiveDocs;
+ boolean nextIsSet;
+
+ @Override
+ public boolean hasNext() {
+ return nextIsSet || setNext();
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Number next() {
+ if (!hasNext()) {
+ throw new NoSuchElementException();
+ }
+ assert nextIsSet;
+ nextIsSet = false;
+ return nextValue;
+ }
+
+ private boolean setNext() {
+ while (true) {
+ if (readerUpto == readers.length) {
+ return false;
+ }
+
+ if (currentReader == null || docIDUpto == currentReader.maxDoc()) {
+ readerUpto++;
+ if (readerUpto < readers.length) {
+ currentReader = readers[readerUpto];
+ currentLiveDocs = currentReader.getLiveDocs();
+ }
+ docIDUpto = 0;
+ continue;
+ }
+
+ if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
+ nextIsSet = true;
+ SortedNumericDocValues dv = dvs[readerUpto];
+ dv.setDocument(docIDUpto);
+ nextValue = dv.count();
+ docIDUpto++;
+ return true;
+ }
+
+ docIDUpto++;
+ }
+ }
+ };
+ }
+ },
+ // values
+ new Iterable<Number>() {
+ @Override
+ public Iterator<Number> iterator() {
+ return new Iterator<Number>() {
+ int readerUpto = -1;
+ int docIDUpto;
+ long nextValue;
+ AtomicReader currentReader;
+ Bits currentLiveDocs;
+ boolean nextIsSet;
+ int valueUpto;
+ int valueLength;
+
+ @Override
+ public boolean hasNext() {
+ return nextIsSet || setNext();
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Number next() {
+ if (!hasNext()) {
+ throw new NoSuchElementException();
+ }
+ assert nextIsSet;
+ nextIsSet = false;
+ return nextValue;
+ }
+
+ private boolean setNext() {
+ while (true) {
+ if (readerUpto == readers.length) {
+ return false;
+ }
+
+ if (valueUpto < valueLength) {
+ nextValue = dvs[readerUpto].valueAt(valueUpto);
+ valueUpto++;
+ nextIsSet = true;
+ return true;
+ }
+
+ if (currentReader == null || docIDUpto == currentReader.maxDoc()) {
+ readerUpto++;
+ if (readerUpto < readers.length) {
+ currentReader = readers[readerUpto];
+ currentLiveDocs = currentReader.getLiveDocs();
+ }
+ docIDUpto = 0;
+ continue;
+ }
+
+ if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
+ assert docIDUpto < currentReader.maxDoc();
+ SortedNumericDocValues dv = dvs[readerUpto];
+ dv.setDocument(docIDUpto);
+ valueUpto = 0;
+ valueLength = dv.count();
+ docIDUpto++;
+ continue;
+ }
+
+ docIDUpto++;
+ }
+ }
+ };
+ }
+ }
+ );
+ }
/**
* Merges the sorted docvalues from <code>toMerge</code>.
@@ -640,4 +801,49 @@ public abstract class DocValuesConsumer
}
}
}
+
+ /** Helper: returns true if the given docToValue count contains only at most one value */
+ public static boolean isSingleValued(Iterable<Number> docToValueCount) {
+ for (Number count : docToValueCount) {
+ if (count.longValue() > 1) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ /** Helper: returns single-valued view, using {@code missingValue} when count is zero */
+ public static Iterable<Number> singletonView(final Iterable<Number> docToValueCount, final Iterable<Number> values, final Number missingValue) {
+ assert isSingleValued(docToValueCount);
+ return new Iterable<Number>() {
+
+ @Override
+ public Iterator<Number> iterator() {
+ final Iterator<Number> countIterator = docToValueCount.iterator();
+ final Iterator<Number> valuesIterator = values.iterator();
+ return new Iterator<Number>() {
+
+ @Override
+ public boolean hasNext() {
+ return countIterator.hasNext();
+ }
+
+ @Override
+ public Number next() {
+ int count = countIterator.next().intValue();
+ if (count == 0) {
+ return missingValue;
+ } else {
+ return valuesIterator.next();
+ }
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+ };
+ }
+ };
+ }
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java Thu Jun 12 19:40:36 2014
@@ -24,6 +24,7 @@ import org.apache.lucene.index.BinaryDoc
import org.apache.lucene.index.FieldInfo;
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.util.Accountable;
import org.apache.lucene.util.Bits;
@@ -54,6 +55,11 @@ public abstract class DocValuesProducer
* used by a single thread. */
public abstract SortedDocValues getSorted(FieldInfo field) throws IOException;
+ /** Returns {@link SortedNumericDocValues} for this field.
+ * The returned instance need not be thread-safe: it will only be
+ * used by a single thread. */
+ public abstract SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException;
+
/** Returns {@link SortedSetDocValues} for this field.
* The returned instance need not be thread-safe: it will only be
* used by a single thread. */
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java Thu Jun 12 19:40:36 2014
@@ -32,6 +32,7 @@ import org.apache.lucene.index.IndexFile
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
@@ -628,6 +629,11 @@ final class Lucene40DocValuesReader exte
}
};
}
+
+ @Override
+ public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+ throw new IllegalStateException("Lucene 4.0 does not support SortedNumeric: how did you pull this off?");
+ }
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java Thu Jun 12 19:40:36 2014
@@ -35,6 +35,7 @@ import org.apache.lucene.index.IndexFile
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ByteArrayDataInput;
@@ -490,6 +491,11 @@ class Lucene42DocValuesProducer extends
return new Bits.MatchAllBits(maxDoc);
}
}
+
+ @Override
+ public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+ throw new IllegalStateException("Lucene 4.2 does not support SortedNumeric: how did you pull this off?");
+ }
@Override
public void close() throws IOException {
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsConsumer.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsConsumer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsConsumer.java Thu Jun 12 19:40:36 2014
@@ -206,6 +206,11 @@ class Lucene42NormsConsumer extends DocV
}
@Override
+ public void addSortedNumericField(FieldInfo field, Iterable<Number> docToValueCount, Iterable<Number> values) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
throw new UnsupportedOperationException();
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java Thu Jun 12 19:40:36 2014
@@ -21,7 +21,6 @@ import java.io.Closeable; // javadocs
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
-import java.util.Iterator;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesConsumer;
@@ -43,7 +42,6 @@ class Lucene45DocValuesConsumer extends
static final int BLOCK_SIZE = 16384;
static final int ADDRESS_INTERVAL = 16;
- static final Number MISSING_ORD = Long.valueOf(-1);
/** Compressed using packed blocks of ints. */
public static final int DELTA_COMPRESSED = 0;
@@ -349,14 +347,10 @@ class Lucene45DocValuesConsumer extends
addTermsDict(field, values);
addNumericField(field, docToOrd, false);
}
-
- private static boolean isSingleValued(Iterable<Number> docToOrdCount) {
- for (Number ordCount : docToOrdCount) {
- if (ordCount.longValue() > 1) {
- return false;
- }
- }
- return true;
+
+ @Override
+ public void addSortedNumericField(FieldInfo field, Iterable<Number> docToValueCount, Iterable<Number> values) throws IOException {
+ throw new UnsupportedOperationException("Lucene 4.5 does not support SORTED_NUMERIC");
}
@Override
@@ -367,40 +361,7 @@ class Lucene45DocValuesConsumer extends
if (isSingleValued(docToOrdCount)) {
meta.writeVInt(SORTED_SET_SINGLE_VALUED_SORTED);
// The field is single-valued, we can encode it as SORTED
- addSortedField(field, values, new Iterable<Number>() {
-
- @Override
- public Iterator<Number> iterator() {
- final Iterator<Number> docToOrdCountIt = docToOrdCount.iterator();
- final Iterator<Number> ordsIt = ords.iterator();
- return new Iterator<Number>() {
-
- @Override
- public boolean hasNext() {
- assert ordsIt.hasNext() ? docToOrdCountIt.hasNext() : true;
- return docToOrdCountIt.hasNext();
- }
-
- @Override
- public Number next() {
- final Number ordCount = docToOrdCountIt.next();
- if (ordCount.longValue() == 0) {
- return MISSING_ORD;
- } else {
- assert ordCount.longValue() == 1;
- return ordsIt.next();
- }
- }
-
- @Override
- public void remove() {
- throw new UnsupportedOperationException();
- }
-
- };
- }
-
- });
+ addSortedField(field, values, singletonView(docToOrdCount, ords, -1L));
return;
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java Thu Jun 12 19:40:36 2014
@@ -47,6 +47,7 @@ import org.apache.lucene.index.NumericDo
import org.apache.lucene.index.RandomAccessOrds;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.TermsEnum.SeekStatus;
@@ -541,6 +542,11 @@ public class Lucene45DocValuesProducer e
}
return ordIndex;
}
+
+ @Override
+ public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+ throw new IllegalStateException("Lucene 4.5 does not support SortedNumeric: how did you pull this off?");
+ }
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java Thu Jun 12 19:40:36 2014
@@ -115,7 +115,8 @@ public final class Lucene46FieldInfosFor
static final String CODEC_NAME = "Lucene46FieldInfos";
static final int FORMAT_START = 0;
static final int FORMAT_CHECKSUM = 1;
- static final int FORMAT_CURRENT = FORMAT_CHECKSUM;
+ static final int FORMAT_SORTED_NUMERIC = 2;
+ static final int FORMAT_CURRENT = FORMAT_SORTED_NUMERIC;
// Field flags
static final byte IS_INDEXED = 0x1;
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java Thu Jun 12 19:40:36 2014
@@ -120,6 +120,8 @@ final class Lucene46FieldInfosReader ext
return DocValuesType.SORTED;
} else if (b == 4) {
return DocValuesType.SORTED_SET;
+ } else if (b == 5) {
+ return DocValuesType.SORTED_NUMERIC;
} else {
throw new CorruptIndexException("invalid docvalues byte: " + b + " (resource=" + input + ")");
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosWriter.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosWriter.java Thu Jun 12 19:40:36 2014
@@ -103,6 +103,8 @@ final class Lucene46FieldInfosWriter ext
return 3;
} else if (type == DocValuesType.SORTED_SET) {
return 4;
+ } else if (type == DocValuesType.SORTED_NUMERIC) {
+ return 5;
} else {
throw new AssertionError();
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesConsumer.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesConsumer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesConsumer.java Thu Jun 12 19:40:36 2014
@@ -22,7 +22,6 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
-import java.util.Iterator;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesConsumer;
@@ -44,7 +43,6 @@ public class Lucene49DocValuesConsumer e
static final int BLOCK_SIZE = 16384;
static final int ADDRESS_INTERVAL = 16;
- static final Number MISSING_ORD = Long.valueOf(-1);
/** Compressed using packed blocks of ints. */
public static final int DELTA_COMPRESSED = 0;
@@ -64,10 +62,10 @@ public class Lucene49DocValuesConsumer e
/** Standard storage for sorted set values with 1 level of indirection:
* docId -> address -> ord. */
- public static final int SORTED_SET_WITH_ADDRESSES = 0;
+ public static final int SORTED_WITH_ADDRESSES = 0;
/** Single-valued sorted set values, encoded as sorted values, so no level
* of indirection: docId -> ord. */
- public static final int SORTED_SET_SINGLE_VALUED_SORTED = 1;
+ public static final int SORTED_SINGLE_VALUED = 1;
IndexOutput data, meta;
final int maxDoc;
@@ -366,13 +364,21 @@ public class Lucene49DocValuesConsumer e
addNumericField(field, docToOrd, false);
}
- private static boolean isSingleValued(Iterable<Number> docToOrdCount) {
- for (Number ordCount : docToOrdCount) {
- if (ordCount.longValue() > 1) {
- return false;
- }
+ @Override
+ public void addSortedNumericField(FieldInfo field, final Iterable<Number> docToValueCount, final Iterable<Number> values) throws IOException {
+ meta.writeVInt(field.number);
+ meta.writeByte(Lucene49DocValuesFormat.SORTED_NUMERIC);
+ if (isSingleValued(docToValueCount)) {
+ meta.writeVInt(SORTED_SINGLE_VALUED);
+ // The field is single-valued, we can encode it as NUMERIC
+ addNumericField(field, singletonView(docToValueCount, values, null));
+ } else {
+ meta.writeVInt(SORTED_WITH_ADDRESSES);
+ // write the stream of values as a numeric field
+ addNumericField(field, values, true);
+ // write the doc -> ord count as a absolute index to the stream
+ addAddresses(field, docToValueCount);
}
- return true;
}
@Override
@@ -381,55 +387,26 @@ public class Lucene49DocValuesConsumer e
meta.writeByte(Lucene49DocValuesFormat.SORTED_SET);
if (isSingleValued(docToOrdCount)) {
- meta.writeVInt(SORTED_SET_SINGLE_VALUED_SORTED);
+ meta.writeVInt(SORTED_SINGLE_VALUED);
// The field is single-valued, we can encode it as SORTED
- addSortedField(field, values, new Iterable<Number>() {
-
- @Override
- public Iterator<Number> iterator() {
- final Iterator<Number> docToOrdCountIt = docToOrdCount.iterator();
- final Iterator<Number> ordsIt = ords.iterator();
- return new Iterator<Number>() {
-
- @Override
- public boolean hasNext() {
- assert ordsIt.hasNext() ? docToOrdCountIt.hasNext() : true;
- return docToOrdCountIt.hasNext();
- }
-
- @Override
- public Number next() {
- final Number ordCount = docToOrdCountIt.next();
- if (ordCount.longValue() == 0) {
- return MISSING_ORD;
- } else {
- assert ordCount.longValue() == 1;
- return ordsIt.next();
- }
- }
+ addSortedField(field, values, singletonView(docToOrdCount, ords, -1L));
+ } else {
+ meta.writeVInt(SORTED_WITH_ADDRESSES);
- @Override
- public void remove() {
- throw new UnsupportedOperationException();
- }
+ // write the ord -> byte[] as a binary field
+ addTermsDict(field, values);
- };
- }
+ // write the stream of ords as a numeric field
+ // NOTE: we could return an iterator that delta-encodes these within a doc
+ addNumericField(field, ords, false);
- });
- return;
+ // write the doc -> ord count as a absolute index to the stream
+ addAddresses(field, docToOrdCount);
}
-
- meta.writeVInt(SORTED_SET_WITH_ADDRESSES);
-
- // write the ord -> byte[] as a binary field
- addTermsDict(field, values);
-
- // write the stream of ords as a numeric field
- // NOTE: we could return an iterator that delta-encodes these within a doc
- addNumericField(field, ords, false);
-
- // write the doc -> ord count as a absolute index to the stream
+ }
+
+ // writes addressing information as MONOTONIC_COMPRESSED integer
+ private void addAddresses(FieldInfo field, Iterable<Number> values) throws IOException {
meta.writeVInt(field.number);
meta.writeByte(Lucene49DocValuesFormat.NUMERIC);
meta.writeVInt(MONOTONIC_COMPRESSED);
@@ -442,7 +419,7 @@ public class Lucene49DocValuesConsumer e
final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
long addr = 0;
writer.add(addr);
- for (Number v : docToOrdCount) {
+ for (Number v : values) {
addr += v.longValue();
writer.add(addr);
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesFormat.java?rev=1602277&r1=1602276&r2=1602277&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesFormat.java Thu Jun 12 19:40:36 2014
@@ -35,7 +35,7 @@ import org.apache.lucene.util.packed.Mon
/**
* Lucene 4.9 DocValues format.
* <p>
- * Encodes the four per-document value types (Numeric,Binary,Sorted,SortedSet) with these strategies:
+ * Encodes the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) with these strategies:
* <p>
* {@link DocValuesType#NUMERIC NUMERIC}:
* <ul>
@@ -75,6 +75,12 @@ import org.apache.lucene.util.packed.Mon
* above.
* </ul>
* <p>
+ * {@link DocValuesType#SORTED_NUMERIC SORTED_NUMERIC}:
+ * <ul>
+ * <li>SortedNumeric: a value list and per-document index into this list are written using the numeric
+ * strategies above.
+ * </ul>
+ * <p>
* Files:
* <ol>
* <li><tt>.dvd</tt>: DocValues data</li>
@@ -87,7 +93,7 @@ import org.apache.lucene.util.packed.Mon
* DocValues data (.dvd)</p>
* <p>DocValues metadata (.dvm) --> Header,<Entry><sup>NumFields</sup>,Footer</p>
* <ul>
- * <li>Entry --> NumericEntry | BinaryEntry | SortedEntry | SortedSetEntry</li>
+ * <li>Entry --> NumericEntry | BinaryEntry | SortedEntry | SortedSetEntry | SortedNumericEntry</li>
* <li>NumericEntry --> GCDNumericEntry | TableNumericEntry | DeltaNumericEntry</li>
* <li>GCDNumericEntry --> NumericHeader,MinValue,GCD,BitsPerValue</li>
* <li>TableNumericEntry --> NumericHeader,TableSize,{@link DataOutput#writeLong Int64}<sup>TableSize</sup>,BitsPerValue</li>
@@ -101,6 +107,7 @@ import org.apache.lucene.util.packed.Mon
* <li>BinaryHeader --> FieldNumber,EntryType,BinaryType,MissingOffset,MinLength,MaxLength,DataOffset</li>
* <li>SortedEntry --> FieldNumber,EntryType,BinaryEntry,NumericEntry</li>
* <li>SortedSetEntry --> EntryType,BinaryEntry,NumericEntry,NumericEntry</li>
+ * <li>SortedNumericEntry --> EntryType,NumericEntry,NumericEntry</li>
* <li>FieldNumber,PackedVersion,MinLength,MaxLength,BlockSize,ValueCount --> {@link DataOutput#writeVInt VInt}</li>
* <li>EntryType,CompressionType --> {@link DataOutput#writeByte Byte}</li>
* <li>Header --> {@link CodecUtil#writeHeader CodecHeader}</li>
@@ -112,6 +119,8 @@ import org.apache.lucene.util.packed.Mon
* and an ordinary NumericEntry for the document-to-ord metadata.</p>
* <p>SortedSet fields have three entries: a BinaryEntry with the value metadata,
* and two NumericEntries for the document-to-ord-index and ordinal list metadata.</p>
+ * <p>SortedNumeric fields have two entries: A NumericEntry with the value metadata,
+ * and a numeric entry with the document-to-value index.</p>
* <p>FieldNumber of -1 indicates the end of metadata.</p>
* <p>EntryType is a 0 (NumericEntry) or 1 (BinaryEntry)</p>
* <p>DataOffset is the pointer to the start of the data in the DocValues data (.dvd)</p>
@@ -182,4 +191,5 @@ public final class Lucene49DocValuesForm
static final byte BINARY = 1;
static final byte SORTED = 2;
static final byte SORTED_SET = 3;
+ static final byte SORTED_NUMERIC = 4;
}