You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/06/16 21:26:23 UTC
svn commit: r1602968 [2/5] - in /lucene/dev/branches/lucene5752: ./
dev-tools/ dev-tools/idea/.idea/ lucene/ lucene/analysis/
lucene/analysis/common/
lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/
lucene/codecs/ lucene/codecs/src/...
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java Mon Jun 16 19:26:19 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/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java Mon Jun 16 19:26:19 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/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListReader.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListReader.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListReader.java Mon Jun 16 19:26:19 2014
@@ -241,7 +241,7 @@ public abstract class MultiLevelSkipList
// clone this stream, it is already at the start of the current level
skipStream[i] = skipStream[0].clone();
if (inputIsBuffered && length < BufferedIndexInput.BUFFER_SIZE) {
- ((BufferedIndexInput) skipStream[i]).setBufferSize((int) length);
+ ((BufferedIndexInput) skipStream[i]).setBufferSize(Math.max(BufferedIndexInput.MIN_BUFFER_SIZE, (int) length));
}
// move base stream beyond the current level
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java Mon Jun 16 19:26:19 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/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java Mon Jun 16 19:26:19 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/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java Mon Jun 16 19:26:19 2014
@@ -67,7 +67,7 @@ public class Lucene42NormsFormat extends
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
- return new Lucene42NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
+ throw new UnsupportedOperationException("this codec can only be used for reading");
}
@Override
@@ -75,8 +75,8 @@ public class Lucene42NormsFormat extends
return new Lucene42DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
}
- private static final String DATA_CODEC = "Lucene41NormsData";
- private static final String DATA_EXTENSION = "nvd";
- private static final String METADATA_CODEC = "Lucene41NormsMetadata";
- private static final String METADATA_EXTENSION = "nvm";
+ static final String DATA_CODEC = "Lucene41NormsData";
+ static final String DATA_EXTENSION = "nvd";
+ static final String METADATA_CODEC = "Lucene41NormsMetadata";
+ static final String METADATA_EXTENSION = "nvm";
}
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java Mon Jun 16 19:26:19 2014
@@ -21,13 +21,13 @@ 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;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.BytesRef;
@@ -43,7 +43,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;
@@ -90,6 +89,7 @@ class Lucene45DocValuesConsumer extends
@Override
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
+ checkCanWrite(field);
addNumericField(field, values, true);
}
@@ -147,7 +147,7 @@ class Lucene45DocValuesConsumer extends
final int format;
if (uniqueValues != null
- && (delta < 0L || PackedInts.bitsRequired(uniqueValues.size() - 1) < PackedInts.bitsRequired(delta))
+ && (PackedInts.bitsRequired(uniqueValues.size() - 1) < PackedInts.unsignedBitsRequired(delta))
&& count <= Integer.MAX_VALUE) {
format = TABLE_COMPRESSED;
} else if (gcd != 0 && gcd != 1) {
@@ -230,6 +230,7 @@ class Lucene45DocValuesConsumer extends
@Override
public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
+ checkCanWrite(field);
// write the byte[] data
meta.writeVInt(field.number);
meta.writeByte(Lucene45DocValuesFormat.BINARY);
@@ -344,63 +345,28 @@ class Lucene45DocValuesConsumer extends
@Override
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
+ checkCanWrite(field);
meta.writeVInt(field.number);
meta.writeByte(Lucene45DocValuesFormat.SORTED);
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
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
+ checkCanWrite(field);
meta.writeVInt(field.number);
meta.writeByte(Lucene45DocValuesFormat.SORTED_SET);
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;
}
@@ -453,4 +419,14 @@ class Lucene45DocValuesConsumer extends
meta = data = null;
}
}
+
+ void checkCanWrite(FieldInfo field) {
+ if ((field.getDocValuesType() == DocValuesType.NUMERIC ||
+ field.getDocValuesType() == DocValuesType.BINARY) &&
+ field.getDocValuesGen() != -1) {
+ // ok
+ } else {
+ throw new UnsupportedOperationException("this codec can only be used for reading");
+ }
+ }
}
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java Mon Jun 16 19:26:19 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;
@@ -63,7 +64,7 @@ import org.apache.lucene.util.packed.Mon
import org.apache.lucene.util.packed.PackedInts;
/** reader for {@link Lucene45DocValuesFormat} */
-public class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
+class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
private final Map<Integer,NumericEntry> numerics;
private final Map<Integer,BinaryEntry> binaries;
private final Map<Integer,SortedSetEntry> sortedSets;
@@ -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/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java Mon Jun 16 19:26:19 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/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java Mon Jun 16 19:26:19 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/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosWriter.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosWriter.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosWriter.java Mon Jun 16 19:26:19 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/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesConsumer.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesConsumer.java Mon Jun 16 19:26:19 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;
@@ -40,11 +39,10 @@ import org.apache.lucene.util.packed.Mon
import org.apache.lucene.util.packed.PackedInts;
/** writer for {@link Lucene49DocValuesFormat} */
-public class Lucene49DocValuesConsumer extends DocValuesConsumer implements Closeable {
+class Lucene49DocValuesConsumer extends DocValuesConsumer implements Closeable {
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;
@@ -150,14 +148,17 @@ public class Lucene49DocValuesConsumer e
}
final long delta = maxValue - minValue;
- final int deltaBitsRequired = delta < 0 ? 64 : DirectWriter.bitsRequired(delta);
+ final int deltaBitsRequired = DirectWriter.unsignedBitsRequired(delta);
+ final int tableBitsRequired = uniqueValues == null
+ ? Integer.MAX_VALUE
+ : DirectWriter.bitsRequired(uniqueValues.size() - 1);
final int format;
- if (uniqueValues != null && DirectWriter.bitsRequired(uniqueValues.size() - 1) < deltaBitsRequired) {
+ if (uniqueValues != null && tableBitsRequired < deltaBitsRequired) {
format = TABLE_COMPRESSED;
} else if (gcd != 0 && gcd != 1) {
final long gcdDelta = (maxValue - minValue) / gcd;
- final long gcdBitsRequired = gcdDelta < 0 ? 64 : DirectWriter.bitsRequired(gcdDelta);
+ final long gcdBitsRequired = DirectWriter.unsignedBitsRequired(gcdDelta);
format = gcdBitsRequired < deltaBitsRequired ? GCD_COMPRESSED : DELTA_COMPRESSED;
} else {
format = DELTA_COMPRESSED;
@@ -179,7 +180,7 @@ public class Lucene49DocValuesConsumer e
meta.writeLong(minValue);
meta.writeLong(gcd);
final long maxDelta = (maxValue - minValue) / gcd;
- final int bits = maxDelta < 0 ? 64 : DirectWriter.bitsRequired(maxDelta);
+ final int bits = DirectWriter.unsignedBitsRequired(maxDelta);
meta.writeVInt(bits);
final DirectWriter quotientWriter = DirectWriter.getInstance(data, count, bits);
for (Number nv : values) {
@@ -208,9 +209,8 @@ public class Lucene49DocValuesConsumer e
meta.writeLong(decode[i]);
encode.put(decode[i], i);
}
- final int bitsRequired = DirectWriter.bitsRequired(uniqueValues.size() - 1);
- meta.writeVInt(bitsRequired);
- final DirectWriter ordsWriter = DirectWriter.getInstance(data, count, bitsRequired);
+ meta.writeVInt(tableBitsRequired);
+ final DirectWriter ordsWriter = DirectWriter.getInstance(data, count, tableBitsRequired);
for (Number nv : values) {
ordsWriter.add(encode.get(nv == null ? 0 : nv.longValue()));
}
@@ -301,7 +301,7 @@ public class Lucene49DocValuesConsumer e
}
/** expert: writes a value dictionary for a sorted/sortedset field */
- protected void addTermsDict(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
+ private void addTermsDict(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
// first check if its a "fixed-length" terms dict
int minLength = Integer.MAX_VALUE;
int maxLength = Integer.MIN_VALUE;
@@ -366,13 +366,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 +389,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 +421,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/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesFormat.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesFormat.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesFormat.java Mon Jun 16 19:26:19 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;
}
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesProducer.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesProducer.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesProducer.java Mon Jun 16 19:26:19 2014
@@ -23,8 +23,8 @@ import static org.apache.lucene.codecs.l
import static org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer.DELTA_COMPRESSED;
import static org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer.GCD_COMPRESSED;
import static org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer.MONOTONIC_COMPRESSED;
-import static org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer.SORTED_SET_SINGLE_VALUED_SORTED;
-import static org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer.SORTED_SET_WITH_ADDRESSES;
+import static org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer.SORTED_SINGLE_VALUED;
+import static org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer.SORTED_WITH_ADDRESSES;
import static org.apache.lucene.codecs.lucene49.Lucene49DocValuesConsumer.TABLE_COMPRESSED;
import java.io.Closeable; // javadocs
@@ -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;
@@ -62,10 +63,11 @@ import org.apache.lucene.util.packed.Dir
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
/** reader for {@link Lucene49DocValuesFormat} */
-public class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
+class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
private final Map<Integer,NumericEntry> numerics;
private final Map<Integer,BinaryEntry> binaries;
private final Map<Integer,SortedSetEntry> sortedSets;
+ private final Map<Integer,SortedSetEntry> sortedNumerics;
private final Map<Integer,NumericEntry> ords;
private final Map<Integer,NumericEntry> ordIndexes;
private final AtomicLong ramBytesUsed;
@@ -78,7 +80,7 @@ public class Lucene49DocValuesProducer e
private final Map<Integer,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
/** expert: instantiates a new reader */
- protected Lucene49DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+ Lucene49DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
// read in the entries from the metadata file.
ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
@@ -93,6 +95,7 @@ public class Lucene49DocValuesProducer e
ordIndexes = new HashMap<>();
binaries = new HashMap<>();
sortedSets = new HashMap<>();
+ sortedNumerics = new HashMap<>();
readFields(in, state.fieldInfos);
CodecUtil.checkFooter(in);
@@ -196,9 +199,9 @@ public class Lucene49DocValuesProducer e
} else if (type == Lucene49DocValuesFormat.SORTED_SET) {
SortedSetEntry ss = readSortedSetEntry(meta);
sortedSets.put(fieldNumber, ss);
- if (ss.format == SORTED_SET_WITH_ADDRESSES) {
+ if (ss.format == SORTED_WITH_ADDRESSES) {
readSortedSetFieldWithAddresses(fieldNumber, meta, infos);
- } else if (ss.format == SORTED_SET_SINGLE_VALUED_SORTED) {
+ } else if (ss.format == SORTED_SINGLE_VALUED) {
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
}
@@ -209,6 +212,28 @@ public class Lucene49DocValuesProducer e
} else {
throw new AssertionError();
}
+ } else if (type == Lucene49DocValuesFormat.SORTED_NUMERIC) {
+ SortedSetEntry ss = readSortedSetEntry(meta);
+ sortedNumerics.put(fieldNumber, ss);
+ if (meta.readVInt() != fieldNumber) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+ }
+ if (meta.readByte() != Lucene49DocValuesFormat.NUMERIC) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+ }
+ numerics.put(fieldNumber, readNumericEntry(meta));
+ if (ss.format == SORTED_WITH_ADDRESSES) {
+ if (meta.readVInt() != fieldNumber) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+ }
+ if (meta.readByte() != Lucene49DocValuesFormat.NUMERIC) {
+ throw new CorruptIndexException("sortednumeric entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+ }
+ NumericEntry ordIndex = readNumericEntry(meta);
+ ordIndexes.put(fieldNumber, ordIndex);
+ } else if (ss.format != SORTED_SINGLE_VALUED) {
+ throw new AssertionError();
+ }
} else {
throw new CorruptIndexException("invalid type: " + type + ", resource=" + meta);
}
@@ -285,7 +310,7 @@ public class Lucene49DocValuesProducer e
SortedSetEntry readSortedSetEntry(IndexInput meta) throws IOException {
SortedSetEntry entry = new SortedSetEntry();
entry.format = meta.readVInt();
- if (entry.format != SORTED_SET_SINGLE_VALUED_SORTED && entry.format != SORTED_SET_WITH_ADDRESSES) {
+ if (entry.format != SORTED_SINGLE_VALUED && entry.format != SORTED_WITH_ADDRESSES) {
throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
}
return entry;
@@ -309,7 +334,6 @@ public class Lucene49DocValuesProducer e
LongValues getNumeric(NumericEntry entry) throws IOException {
RandomAccessInput slice = this.data.randomAccessSlice(entry.offset, entry.endOffset - entry.offset);
-
switch (entry.format) {
case DELTA_COMPRESSED:
final long delta = entry.minValue;
@@ -384,9 +408,8 @@ public class Lucene49DocValuesProducer e
};
}
- /** returns an address instance for variable-length binary values.
- * @lucene.internal */
- protected MonotonicBlockPackedReader getAddressInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
+ /** returns an address instance for variable-length binary values. */
+ private MonotonicBlockPackedReader getAddressInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
final MonotonicBlockPackedReader addresses;
synchronized (addressInstances) {
MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
@@ -426,9 +449,8 @@ public class Lucene49DocValuesProducer e
};
}
- /** returns an address instance for prefix-compressed binary values.
- * @lucene.internal */
- protected MonotonicBlockPackedReader getIntervalInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
+ /** returns an address instance for prefix-compressed binary values. */
+ private MonotonicBlockPackedReader getIntervalInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
final MonotonicBlockPackedReader addresses;
final long interval = bytes.addressInterval;
synchronized (addressInstances) {
@@ -503,9 +525,8 @@ public class Lucene49DocValuesProducer e
};
}
- /** returns an address instance for sortedset ordinal lists
- * @lucene.internal */
- protected MonotonicBlockPackedReader getOrdIndexInstance(IndexInput data, FieldInfo field, NumericEntry entry) throws IOException {
+ /** returns an address instance for sortedset ordinal lists */
+ private MonotonicBlockPackedReader getOrdIndexInstance(IndexInput data, FieldInfo field, NumericEntry entry) throws IOException {
final MonotonicBlockPackedReader ordIndex;
synchronized (ordIndexInstances) {
MonotonicBlockPackedReader ordIndexInstance = ordIndexInstances.get(field.number);
@@ -519,14 +540,51 @@ public class Lucene49DocValuesProducer e
}
return ordIndex;
}
+
+ @Override
+ public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+ SortedSetEntry ss = sortedNumerics.get(field.number);
+ NumericEntry numericEntry = numerics.get(field.number);
+ final LongValues values = getNumeric(numericEntry);
+ if (ss.format == SORTED_SINGLE_VALUED) {
+ final Bits docsWithField = getMissingBits(numericEntry.missingOffset);
+ return DocValues.singleton(values, docsWithField);
+ } else if (ss.format == SORTED_WITH_ADDRESSES) {
+ final IndexInput data = this.data.clone();
+ final MonotonicBlockPackedReader ordIndex = getOrdIndexInstance(data, field, ordIndexes.get(field.number));
+
+ return new SortedNumericDocValues() {
+ long startOffset;
+ long endOffset;
+
+ @Override
+ public void setDocument(int doc) {
+ startOffset = ordIndex.get(doc);
+ endOffset = ordIndex.get(doc+1L);
+ }
+
+ @Override
+ public long valueAt(int index) {
+ return values.get(startOffset + index);
+ }
+
+ @Override
+ public int count() {
+ return (int) (endOffset - startOffset);
+ }
+ };
+ } else {
+ throw new AssertionError();
+ }
+ }
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
SortedSetEntry ss = sortedSets.get(field.number);
- if (ss.format == SORTED_SET_SINGLE_VALUED_SORTED) {
+ if (ss.format == SORTED_SINGLE_VALUED) {
final SortedDocValues values = getSorted(field);
return DocValues.singleton(values);
- } else if (ss.format != SORTED_SET_WITH_ADDRESSES) {
+ } else if (ss.format != SORTED_WITH_ADDRESSES) {
throw new AssertionError();
}
@@ -629,6 +687,8 @@ public class Lucene49DocValuesProducer e
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:
@@ -648,7 +708,7 @@ public class Lucene49DocValuesProducer e
}
/** metadata entry for a numeric docvalues field */
- protected static class NumericEntry {
+ static class NumericEntry {
private NumericEntry() {}
/** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
long missingOffset;
@@ -673,7 +733,7 @@ public class Lucene49DocValuesProducer e
}
/** metadata entry for a binary docvalues field */
- protected static class BinaryEntry {
+ static class BinaryEntry {
private BinaryEntry() {}
/** offset to the bitset representing docsWithField, or -1 if no documents have missing values */
long missingOffset;
@@ -696,7 +756,7 @@ public class Lucene49DocValuesProducer e
}
/** metadata entry for a sorted-set docvalues field */
- protected static class SortedSetEntry {
+ static class SortedSetEntry {
private SortedSetEntry() {}
int format;
}
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsConsumer.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsConsumer.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsConsumer.java Mon Jun 16 19:26:19 2014
@@ -205,4 +205,9 @@ class Lucene49NormsConsumer extends DocV
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
throw new UnsupportedOperationException();
}
+
+ @Override
+ public void addSortedNumericField(FieldInfo field, Iterable<Number> docToValueCount, Iterable<Number> values) throws IOException {
+ throw new UnsupportedOperationException();
+ }
}
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsFormat.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsFormat.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsFormat.java Mon Jun 16 19:26:19 2014
@@ -97,7 +97,7 @@ import org.apache.lucene.util.packed.Pac
* </ol>
* @lucene.experimental
*/
-public class Lucene49NormsFormat extends NormsFormat {
+public final class Lucene49NormsFormat extends NormsFormat {
/** Sole Constructor */
public Lucene49NormsFormat() {}
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsProducer.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsProducer.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/Lucene49NormsProducer.java Mon Jun 16 19:26:19 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.ChecksumIndexInput;
import org.apache.lucene.store.IndexInput;
@@ -217,6 +218,11 @@ class Lucene49NormsProducer extends DocV
}
@Override
+ public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+ throw new IllegalStateException();
+ }
+
+ @Override
public Bits getDocsWithField(FieldInfo field) throws IOException {
throw new IllegalStateException();
}
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/package.html?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/package.html (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/lucene49/package.html Mon Jun 16 19:26:19 2014
@@ -386,6 +386,9 @@ allow updating NumericDocValues fields.<
<li>In version 4.8, checksum footers were added to the end of each index file
for improved data integrity. Specifically, the last 8 bytes of every index file
contain the zlib-crc32 checksum of the file.</li>
+<li>In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric)
+that is suitable for faceting/sorting/analytics.
+</li>
</ul>
<a name="Limitations" id="Limitations"></a>
<h2>Limitations</h2>
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java Mon Jun 16 19:26:19 2014
@@ -35,6 +35,7 @@ import org.apache.lucene.index.NumericDo
import org.apache.lucene.index.SegmentReadState;
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.util.Bits;
import org.apache.lucene.util.BytesRef;
@@ -117,6 +118,11 @@ public abstract class PerFieldDocValuesF
}
@Override
+ public void addSortedNumericField(FieldInfo field, Iterable<Number> docToValueCount, Iterable<Number> values) throws IOException {
+ getInstance(field).addSortedNumericField(field, docToValueCount, values);
+ }
+
+ @Override
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
getInstance(field).addSortedSetField(field, values, docToOrdCount, ords);
}
@@ -280,6 +286,12 @@ public abstract class PerFieldDocValuesF
}
@Override
+ public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+ DocValuesProducer producer = fields.get(field.name);
+ return producer == null ? null : producer.getSortedNumeric(field);
+ }
+
+ @Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
DocValuesProducer producer = fields.get(field.name);
return producer == null ? null : producer.getSortedSet(field);
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java Mon Jun 16 19:26:19 2014
@@ -275,6 +275,12 @@ public abstract class AtomicReader exten
* this field. The returned instance should only be
* used by a single thread. */
public abstract SortedDocValues getSortedDocValues(String field) throws IOException;
+
+ /** Returns {@link SortedNumericDocValues} for this field, or
+ * null if no {@link SortedNumericDocValues} were indexed for
+ * this field. The returned instance should only be
+ * used by a single thread. */
+ public abstract SortedNumericDocValues getSortedNumericDocValues(String field) throws IOException;
/** Returns {@link SortedSetDocValues} for this field, or
* null if no {@link SortedSetDocValues} were indexed for
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java Mon Jun 16 19:26:19 2014
@@ -32,6 +32,7 @@ import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.InfoStream;
@@ -50,7 +51,7 @@ import org.apache.lucene.util.InfoStream
* track which BufferedDeletes packets to apply to any given
* segment. */
-class BufferedUpdatesStream {
+class BufferedUpdatesStream implements Accountable {
// TODO: maybe linked list?
private final List<FrozenBufferedUpdates> updates = new ArrayList<>();
@@ -111,7 +112,8 @@ class BufferedUpdatesStream {
return numTerms.get();
}
- public long bytesUsed() {
+ @Override
+ public long ramBytesUsed() {
return bytesUsed.get();
}
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Mon Jun 16 19:26:19 2014
@@ -30,7 +30,6 @@ import java.util.Map;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
import org.apache.lucene.codecs.blocktree.FieldReader;
import org.apache.lucene.codecs.blocktree.Stats;
import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
@@ -299,6 +298,9 @@ public class CheckIndex {
/** Total number of sorted fields */
public long totalSortedFields;
+ /** Total number of sortednumeric fields */
+ public long totalSortedNumericFields;
+
/** Total number of sortedset fields */
public long totalSortedSetFields;
@@ -1380,6 +1382,7 @@ public class CheckIndex {
+ status.totalBinaryFields + " BINARY; "
+ status.totalNumericFields + " NUMERIC; "
+ status.totalSortedFields + " SORTED; "
+ + status.totalSortedNumericFields + " SORTED_NUMERIC; "
+ status.totalSortedSetFields + " SORTED_SET]");
} catch (Throwable e) {
msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
@@ -1510,6 +1513,30 @@ public class CheckIndex {
lastValue = BytesRef.deepCopyOf(term);
}
}
+
+ private static void checkSortedNumericDocValues(String fieldName, AtomicReader reader, SortedNumericDocValues ndv, Bits docsWithField) {
+ for (int i = 0; i < reader.maxDoc(); i++) {
+ ndv.setDocument(i);
+ int count = ndv.count();
+ if (docsWithField.get(i)) {
+ if (count == 0) {
+ throw new RuntimeException("dv for field: " + fieldName + " is not marked missing but has zero count for doc: " + i);
+ }
+ long previous = Long.MIN_VALUE;
+ for (int j = 0; j < count; j++) {
+ long value = ndv.valueAt(j);
+ if (value < previous) {
+ throw new RuntimeException("values out of order: " + value + " < " + previous + " for doc: " + i);
+ }
+ previous = value;
+ }
+ } else {
+ if (count != 0) {
+ throw new RuntimeException("dv for field: " + fieldName + " is marked missing but has count=" + count + " for doc: " + i);
+ }
+ }
+ }
+ }
private static void checkNumericDocValues(String fieldName, AtomicReader reader, NumericDocValues ndv, Bits docsWithField) {
for (int i = 0; i < reader.maxDoc(); i++) {
@@ -1533,15 +1560,27 @@ public class CheckIndex {
checkSortedDocValues(fi.name, reader, reader.getSortedDocValues(fi.name), docsWithField);
if (reader.getBinaryDocValues(fi.name) != null ||
reader.getNumericDocValues(fi.name) != null ||
+ reader.getSortedNumericDocValues(fi.name) != null ||
reader.getSortedSetDocValues(fi.name) != null) {
throw new RuntimeException(fi.name + " returns multiple docvalues types!");
}
break;
+ case SORTED_NUMERIC:
+ status.totalSortedNumericFields++;
+ checkSortedNumericDocValues(fi.name, reader, reader.getSortedNumericDocValues(fi.name), docsWithField);
+ if (reader.getBinaryDocValues(fi.name) != null ||
+ reader.getNumericDocValues(fi.name) != null ||
+ reader.getSortedSetDocValues(fi.name) != null ||
+ reader.getSortedDocValues(fi.name) != null) {
+ throw new RuntimeException(fi.name + " returns multiple docvalues types!");
+ }
+ break;
case SORTED_SET:
status.totalSortedSetFields++;
checkSortedSetDocValues(fi.name, reader, reader.getSortedSetDocValues(fi.name), docsWithField);
if (reader.getBinaryDocValues(fi.name) != null ||
reader.getNumericDocValues(fi.name) != null ||
+ reader.getSortedNumericDocValues(fi.name) != null ||
reader.getSortedDocValues(fi.name) != null) {
throw new RuntimeException(fi.name + " returns multiple docvalues types!");
}
@@ -1551,6 +1590,7 @@ public class CheckIndex {
checkBinaryDocValues(fi.name, reader, reader.getBinaryDocValues(fi.name), docsWithField);
if (reader.getNumericDocValues(fi.name) != null ||
reader.getSortedDocValues(fi.name) != null ||
+ reader.getSortedNumericDocValues(fi.name) != null ||
reader.getSortedSetDocValues(fi.name) != null) {
throw new RuntimeException(fi.name + " returns multiple docvalues types!");
}
@@ -1560,6 +1600,7 @@ public class CheckIndex {
checkNumericDocValues(fi.name, reader, reader.getNumericDocValues(fi.name), docsWithField);
if (reader.getBinaryDocValues(fi.name) != null ||
reader.getSortedDocValues(fi.name) != null ||
+ reader.getSortedNumericDocValues(fi.name) != null ||
reader.getSortedSetDocValues(fi.name) != null) {
throw new RuntimeException(fi.name + " returns multiple docvalues types!");
}
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java Mon Jun 16 19:26:19 2014
@@ -64,9 +64,6 @@ final class DefaultIndexingChain extends
private int totalFieldCount;
private long nextFieldGen;
- // Lazy init:
- private NumericDocValuesWriter norms;
-
// Holds fields seen in each document
private PerField[] fields = new PerField[1];
@@ -426,6 +423,13 @@ final class DefaultIndexingChain extends
}
((SortedDocValuesWriter) fp.docValuesWriter).addValue(docID, field.binaryValue());
break;
+
+ case SORTED_NUMERIC:
+ if (fp.docValuesWriter == null) {
+ fp.docValuesWriter = new SortedNumericDocValuesWriter(fp.fieldInfo, bytesUsed);
+ }
+ ((SortedNumericDocValuesWriter) fp.docValuesWriter).addValue(docID, field.numericValue().longValue());
+ break;
case SORTED_SET:
if (fp.docValuesWriter == null) {
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DocValues.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DocValues.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DocValues.java Mon Jun 16 19:26:19 2014
@@ -77,6 +77,26 @@ public final class DocValues {
}
};
}
+
+ /**
+ * An empty SortedNumericDocValues which returns zero values for every document
+ */
+ public static final SortedNumericDocValues emptySortedNumeric() {
+ return new SortedNumericDocValues() {
+ @Override
+ public void setDocument(int doc) {}
+
+ @Override
+ public long valueAt(int index) {
+ throw new IndexOutOfBoundsException();
+ }
+
+ @Override
+ public int count() {
+ return 0;
+ }
+ };
+ }
/**
* An empty SortedDocValues which returns {@link SortedSetDocValues#NO_MORE_ORDS} for every document
@@ -122,7 +142,7 @@ public final class DocValues {
/**
* Returns a single-valued view of the SortedSetDocValues, if it was previously
- * wrapped with {@link #singleton}, or null.
+ * wrapped with {@link #singleton(SortedDocValues)}, or null.
*/
public static SortedDocValues unwrapSingleton(SortedSetDocValues dv) {
if (dv instanceof SingletonSortedSetDocValues) {
@@ -132,6 +152,38 @@ public final class DocValues {
}
}
+ /**
+ * Returns a single-valued view of the SortedNumericDocValues, if it was previously
+ * wrapped with {@link #singleton(NumericDocValues, Bits)}, or null.
+ * @see #unwrapSingletonBits(SortedNumericDocValues)
+ */
+ public static NumericDocValues unwrapSingleton(SortedNumericDocValues dv) {
+ if (dv instanceof SingletonSortedNumericDocValues) {
+ return ((SingletonSortedNumericDocValues)dv).getNumericDocValues();
+ } else {
+ return null;
+ }
+ }
+
+ /**
+ * Returns the documents with a value for the SortedNumericDocValues, if it was previously
+ * wrapped with {@link #singleton(NumericDocValues, Bits)}, or null.
+ */
+ public static Bits unwrapSingletonBits(SortedNumericDocValues dv) {
+ if (dv instanceof SingletonSortedNumericDocValues) {
+ return ((SingletonSortedNumericDocValues)dv).getDocsWithField();
+ } else {
+ return null;
+ }
+ }
+
+ /**
+ * Returns a multi-valued view over the provided NumericDocValues
+ */
+ public static SortedNumericDocValues singleton(NumericDocValues dv, Bits docsWithField) {
+ return new SingletonSortedNumericDocValues(dv, docsWithField);
+ }
+
/**
* Returns a Bits representing all documents from <code>dv</code> that have a value.
*/
@@ -167,6 +219,24 @@ public final class DocValues {
};
}
+ /**
+ * Returns a Bits representing all documents from <code>dv</code> that have a value.
+ */
+ public static Bits docsWithValue(final SortedNumericDocValues dv, final int maxDoc) {
+ return new Bits() {
+ @Override
+ public boolean get(int index) {
+ dv.setDocument(index);
+ return dv.count() != 0;
+ }
+
+ @Override
+ public int length() {
+ return maxDoc;
+ }
+ };
+ }
+
// some helpers, for transition from fieldcache apis.
// as opposed to the AtomicReader apis (which must be strict for consistency), these are lenient
@@ -209,6 +279,22 @@ public final class DocValues {
}
/**
+ * Returns SortedNumericDocValues for the reader, or {@link #emptySortedNumeric} if it has none.
+ */
+ public static SortedNumericDocValues getSortedNumeric(AtomicReader in, String field) throws IOException {
+ SortedNumericDocValues dv = in.getSortedNumericDocValues(field);
+ if (dv == null) {
+ NumericDocValues single = in.getNumericDocValues(field);
+ if (single == null) {
+ return emptySortedNumeric();
+ }
+ Bits bits = in.getDocsWithField(field);
+ return singleton(single, bits);
+ }
+ return dv;
+ }
+
+ /**
* Returns SortedSetDocValues for the reader, or {@link #emptySortedSet} if it has none.
*/
public static SortedSetDocValues getSortedSet(AtomicReader in, String field) throws IOException {
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java Mon Jun 16 19:26:19 2014
@@ -34,6 +34,7 @@ import org.apache.lucene.index.IndexWrit
import org.apache.lucene.search.Query;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.InfoStream;
/**
@@ -94,7 +95,7 @@ import org.apache.lucene.util.InfoStream
* or none") added to the index.
*/
-final class DocumentsWriter implements Closeable {
+final class DocumentsWriter implements Closeable, Accountable {
private final Directory directory;
private volatile boolean closed;
@@ -662,7 +663,12 @@ final class DocumentsWriter implements C
private void putEvent(Event event) {
events.add(event);
}
-
+
+ @Override
+ public long ramBytesUsed() {
+ return flushControl.ramBytesUsed();
+ }
+
static final class ApplyDeletesEvent implements Event {
static final Event INSTANCE = new ApplyDeletesEvent();
private int instCount = 0;
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java Mon Jun 16 19:26:19 2014
@@ -24,6 +24,7 @@ import java.util.concurrent.locks.Reentr
import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
import org.apache.lucene.search.Query;
+import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
/**
@@ -66,7 +67,7 @@ import org.apache.lucene.util.BytesRef;
* will also not be added to its private deletes neither to the global deletes.
*
*/
-final class DocumentsWriterDeleteQueue {
+final class DocumentsWriterDeleteQueue implements Accountable {
private volatile Node<?> tail;
@@ -448,8 +449,9 @@ final class DocumentsWriterDeleteQueue {
globalBufferLock.unlock();
}
}
-
- public long bytesUsed() {
+
+ @Override
+ public long ramBytesUsed() {
return globalBufferedUpdates.bytesUsed.get();
}
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java Mon Jun 16 19:26:19 2014
@@ -19,14 +19,15 @@ package org.apache.lucene.index;
import java.util.ArrayList;
import java.util.IdentityHashMap;
-import java.util.List;
import java.util.Iterator;
import java.util.LinkedList;
+import java.util.List;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
+import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.ThreadInterruptedException;
@@ -42,7 +43,7 @@ import org.apache.lucene.util.ThreadInte
* {@link IndexWriterConfig#getRAMPerThreadHardLimitMB()} to prevent address
* space exhaustion.
*/
-final class DocumentsWriterFlushControl {
+final class DocumentsWriterFlushControl implements Accountable {
private final long hardMaxBytesPerDWPT;
private long activeBytes = 0;
@@ -428,7 +429,12 @@ final class DocumentsWriterFlushControl
}
public long getDeleteBytesUsed() {
- return documentsWriter.deleteQueue.bytesUsed() + bufferedUpdatesStream.bytesUsed();
+ return documentsWriter.deleteQueue.ramBytesUsed() + bufferedUpdatesStream.ramBytesUsed();
+ }
+
+ @Override
+ public long ramBytesUsed() {
+ return getDeleteBytesUsed() + netBytes();
}
synchronized int numFlushingDWPT() {
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java Mon Jun 16 19:26:19 2014
@@ -106,6 +106,11 @@ public final class FieldInfo {
*/
SORTED,
/**
+ * A pre-sorted Number[]. Fields with this type store numeric values in sorted
+ * order according to {@link Long#compare(long, long)}.
+ */
+ SORTED_NUMERIC,
+ /**
* A pre-sorted Set<byte[]>. Fields with this type only store distinct byte values
* and store additional offset pointers per document to dereference the shared
* byte[]s. The stored byte[] is presorted and allows access via document id,
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java Mon Jun 16 19:26:19 2014
@@ -413,6 +413,12 @@ public class FilterAtomicReader extends
ensureOpen();
return in.getSortedDocValues(field);
}
+
+ @Override
+ public SortedNumericDocValues getSortedNumericDocValues(String field) throws IOException {
+ ensureOpen();
+ return in.getSortedNumericDocValues(field);
+ }
@Override
public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java Mon Jun 16 19:26:19 2014
@@ -112,9 +112,9 @@ class FrozenBufferedUpdates {
}
binaryDVUpdates = allBinaryUpdates.toArray(new BinaryDocValuesUpdate[allBinaryUpdates.size()]);
- bytesUsed = (int) terms.ramBytesUsed() + queries.length * BYTES_PER_DEL_QUERY
- + numericUpdatesSize + numericDVUpdates.length * RamUsageEstimator.NUM_BYTES_OBJECT_REF
- + binaryUpdatesSize + binaryDVUpdates.length * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
+ bytesUsed = (int) (terms.ramBytesUsed() + queries.length * BYTES_PER_DEL_QUERY
+ + numericUpdatesSize + RamUsageEstimator.shallowSizeOf(numericDVUpdates)
+ + binaryUpdatesSize + RamUsageEstimator.shallowSizeOf(binaryDVUpdates));
numTermDeletes = deletes.numTermDeletes.get();
}
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Mon Jun 16 19:26:19 2014
@@ -56,6 +56,7 @@ import org.apache.lucene.store.Lock;
import org.apache.lucene.store.LockObtainFailedException;
import org.apache.lucene.store.MergeInfo;
import org.apache.lucene.store.TrackingDirectoryWrapper;
+import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Constants;
@@ -192,7 +193,7 @@ import org.apache.lucene.util.Version;
* referenced by the "front" of the index). For this, IndexFileDeleter
* keeps track of the last non commit checkpoint.
*/
-public class IndexWriter implements Closeable, TwoPhaseCommit{
+public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
private static final int UNBOUNDED_MAX_MERGE_SEGMENTS = -1;
@@ -425,6 +426,12 @@ public class IndexWriter implements Clos
return r;
}
+ @Override
+ public final long ramBytesUsed() {
+ ensureOpen();
+ return docWriter.ramBytesUsed();
+ }
+
/** Holds shared SegmentReader instances. IndexWriter uses
* SegmentReaders for 1) applying deletes, 2) doing
* merges, 3) handing out a real-time reader. This pool
@@ -3090,7 +3097,7 @@ public class IndexWriter implements Clos
}
applyAllDeletesAndUpdates();
} else if (infoStream.isEnabled("IW")) {
- infoStream.message("IW", "don't apply deletes now delTermCount=" + bufferedUpdatesStream.numTerms() + " bytesUsed=" + bufferedUpdatesStream.bytesUsed());
+ infoStream.message("IW", "don't apply deletes now delTermCount=" + bufferedUpdatesStream.numTerms() + " bytesUsed=" + bufferedUpdatesStream.ramBytesUsed());
}
}
@@ -3120,14 +3127,6 @@ public class IndexWriter implements Clos
bufferedUpdatesStream.prune(segmentInfos);
}
- /** Expert: Return the total size of all index files currently cached in memory.
- * Useful for size management with flushRamDocs()
- */
- public final long ramSizeInBytes() {
- ensureOpen();
- return docWriter.flushControl.netBytes() + bufferedUpdatesStream.bytesUsed();
- }
-
// for testing only
DocumentsWriter getDocsWriter() {
boolean test = false;
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java Mon Jun 16 19:26:19 2014
@@ -25,6 +25,7 @@ import org.apache.lucene.index.MultiTerm
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
import org.apache.lucene.util.packed.PackedInts;
@@ -228,6 +229,63 @@ public class MultiDocValues {
}
}
+ /** Returns a SortedNumericDocValues for a reader's docvalues (potentially merging on-the-fly)
+ * <p>
+ * This is a slow way to access sorted numeric values. Instead, access them per-segment
+ * with {@link AtomicReader#getSortedNumericDocValues(String)}
+ * </p>
+ * */
+ public static SortedNumericDocValues getSortedNumericValues(final IndexReader r, final String field) throws IOException {
+ final List<AtomicReaderContext> leaves = r.leaves();
+ final int size = leaves.size();
+ if (size == 0) {
+ return null;
+ } else if (size == 1) {
+ return leaves.get(0).reader().getSortedNumericDocValues(field);
+ }
+
+ boolean anyReal = false;
+ final SortedNumericDocValues[] values = new SortedNumericDocValues[size];
+ final int[] starts = new int[size+1];
+ for (int i = 0; i < size; i++) {
+ AtomicReaderContext context = leaves.get(i);
+ SortedNumericDocValues v = context.reader().getSortedNumericDocValues(field);
+ if (v == null) {
+ v = DocValues.emptySortedNumeric();
+ } else {
+ anyReal = true;
+ }
+ values[i] = v;
+ starts[i] = context.docBase;
+ }
+ starts[size] = r.maxDoc();
+
+ if (!anyReal) {
+ return null;
+ } else {
+ return new SortedNumericDocValues() {
+ SortedNumericDocValues current;
+
+ @Override
+ public void setDocument(int doc) {
+ int subIndex = ReaderUtil.subIndex(doc, starts);
+ current = values[subIndex];
+ current.setDocument(doc - starts[subIndex]);
+ }
+
+ @Override
+ public long valueAt(int index) {
+ return current.valueAt(index);
+ }
+
+ @Override
+ public int count() {
+ return current.count();
+ }
+ };
+ }
+ }
+
/** Returns a SortedDocValues for a reader's docvalues (potentially doing extremely slow things).
* <p>
* This is an extremely slow way to access sorted values. Instead, access them per-segment
@@ -320,6 +378,9 @@ public class MultiDocValues {
// TODO: use more efficient packed ints structures?
// TODO: pull this out? its pretty generic (maps between N ord()-enabled TermsEnums)
public static class OrdinalMap implements Accountable {
+
+ private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(OrdinalMap.class);
+
// cache key of whoever asked for this awful thing
final Object owner;
// globalOrd -> (globalOrd - segmentOrd) where segmentOrd is the the ordinal in the first segment that contains this term
@@ -416,7 +477,7 @@ public class MultiDocValues {
@Override
public long ramBytesUsed() {
- long size = globalOrdDeltas.ramBytesUsed() + firstSegments.ramBytesUsed();
+ long size = BASE_RAM_BYTES_USED + globalOrdDeltas.ramBytesUsed() + firstSegments.ramBytesUsed() + RamUsageEstimator.shallowSizeOf(ordDeltas);
for (int i = 0; i < ordDeltas.length; i++) {
size += ordDeltas[i].ramBytesUsed();
}
Modified: lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java?rev=1602968&r1=1602967&r2=1602968&view=diff
==============================================================================
--- lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java (original)
+++ lucene/dev/branches/lucene5752/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java Mon Jun 16 19:26:19 2014
@@ -287,6 +287,13 @@ public class ParallelAtomicReader extend
AtomicReader reader = fieldToReader.get(field);
return reader == null ? null : reader.getSortedDocValues(field);
}
+
+ @Override
+ public SortedNumericDocValues getSortedNumericDocValues(String field) throws IOException {
+ ensureOpen();
+ AtomicReader reader = fieldToReader.get(field);
+ return reader == null ? null : reader.getSortedNumericDocValues(field);
+ }
@Override
public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {