commit current state

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4765@1444647 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2013-02-11 02:54:53 +00:00
parent 0d1dd2415f
commit 2dcf80718c
38 changed files with 1882 additions and 268 deletions

View File

@ -123,6 +123,11 @@ public class DiskDocValuesConsumer extends DocValuesConsumer {
addNumericField(field, docToOrd);
}
@Override
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
throw new UnsupportedOperationException(); // nocommit
}
@Override
public void close() throws IOException {
boolean success = false;

View File

@ -31,6 +31,7 @@ import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -251,6 +252,11 @@ class DiskDocValuesProducer extends DocValuesProducer {
};
}
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
throw new UnsupportedOperationException(); // nocommit
}
@Override
public void close() throws IOException {
data.close();

View File

@ -36,6 +36,7 @@ import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.StringHelper;
@ -284,6 +285,11 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
};
}
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
throw new UnsupportedOperationException(); // nocommit
}
@Override
public void close() throws IOException {
data.close();

View File

@ -250,6 +250,11 @@ class SimpleTextDocValuesWriter extends DocValuesConsumer {
}
}
@Override
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
throw new UnsupportedOperationException(); // nocommit
}
/** write the header for this field */
private void writeFieldEntry(FieldInfo field, FieldInfo.DocValuesType type) throws IOException {
SimpleTextUtil.write(data, FIELD);

View File

@ -19,7 +19,6 @@ package org.apache.lucene.codecs;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.NoSuchElementException;
@ -27,15 +26,20 @@ import java.util.NoSuchElementException;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FilteredTermsEnum;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.MultiDocValues.OrdinalMap;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedDocValuesTermsEnum;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.SortedSetDocValues.OrdIterator;
import org.apache.lucene.index.SortedSetDocValuesTermsEnum;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.PriorityQueue;
import org.apache.lucene.util.packed.AppendingLongBuffer;
/**
* Abstract API that consumes numeric, binary and
@ -89,6 +93,16 @@ public abstract class DocValuesConsumer implements Closeable {
*/
public abstract void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException;
/**
* Writes pre-sorted set docvalues for a field
* @param field field information
* @param values Iterable of binary values in sorted order (deduplicated).
* @param docToOrdCount Iterable of the number of values for each document.
* @param ords Iterable of ordinal occurrences (docToOrdCount*maxDoc total).
* @throws IOException if an I/O error occurred.
*/
public abstract void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException;
/**
* Merges the numeric docvalues from <code>toMerge</code>.
* <p>
@ -237,134 +251,6 @@ public abstract class DocValuesConsumer implements Closeable {
});
}
static class SortedBytesMerger {
public int numMergedTerms;
final AppendingLongBuffer ordToReaderId = new AppendingLongBuffer();
final List<SegmentState> segStates = new ArrayList<SegmentState>();
private static class SegmentState {
int segmentID;
AtomicReader reader;
FixedBitSet liveTerms;
int ord = -1;
SortedDocValues values;
BytesRef scratch = new BytesRef();
AppendingLongBuffer ordDeltas = new AppendingLongBuffer();
// TODO: use another scheme?
// currently we +/- delta merged-ord from segment-ord (is this good? makes sense to me?)
// but we have a good idea "roughly" what
// the ord should be (linear projection) so we only
// need to encode the delta from that ...:
AppendingLongBuffer segOrdToMergedOrd = new AppendingLongBuffer();
public BytesRef nextTerm() {
while (ord < values.getValueCount()-1) {
ord++;
if (liveTerms == null || liveTerms.get(ord)) {
values.lookupOrd(ord, scratch);
return scratch;
}
}
return null;
}
}
private static class TermMergeQueue extends PriorityQueue<SegmentState> {
public TermMergeQueue(int maxSize) {
super(maxSize);
}
@Override
protected boolean lessThan(SegmentState a, SegmentState b) {
return a.scratch.compareTo(b.scratch) <= 0;
}
}
public void merge(MergeState mergeState, List<SortedDocValues> toMerge) throws IOException {
// First pass: mark "live" terms
for (int readerIDX=0;readerIDX<toMerge.size();readerIDX++) {
AtomicReader reader = mergeState.readers.get(readerIDX);
int maxDoc = reader.maxDoc();
SegmentState state = new SegmentState();
state.segmentID = readerIDX;
state.reader = reader;
state.values = toMerge.get(readerIDX);
segStates.add(state);
assert state.values.getValueCount() < Integer.MAX_VALUE;
if (reader.hasDeletions()) {
state.liveTerms = new FixedBitSet(state.values.getValueCount());
Bits liveDocs = reader.getLiveDocs();
assert liveDocs != null;
for(int docID=0;docID<maxDoc;docID++) {
if (liveDocs.get(docID)) {
state.liveTerms.set(state.values.getOrd(docID));
}
}
}
// TODO: we can unload the bits/packed ints to disk to reduce
// transient ram spike... most of these just require iterators
}
// Second pass: merge only the live terms
TermMergeQueue q = new TermMergeQueue(segStates.size());
for(SegmentState segState : segStates) {
if (segState.nextTerm() != null) {
q.add(segState);
}
}
int lastOrds[] = new int[segStates.size()];
BytesRef lastTerm = null;
int ord = 0;
while (q.size() != 0) {
SegmentState top = q.top();
if (lastTerm == null || !lastTerm.equals(top.scratch)) {
// a new unique term: record its segment ID / sourceOrd pair
int readerId = top.segmentID;
ordToReaderId.add(readerId);
int sourceOrd = top.ord;
int delta = sourceOrd - lastOrds[readerId];
lastOrds[readerId] = sourceOrd;
top.ordDeltas.add(delta);
if (lastTerm == null) {
lastTerm = BytesRef.deepCopyOf(top.scratch);
} else {
lastTerm.copyBytes(top.scratch);
}
ord++;
}
long signedDelta = (ord-1) - top.ord; // global ord space - segment ord space
// fill in any holes for unused ords, then finally the value we want (segOrdToMergedOrd[top.ord])
// TODO: is there a better way...
while (top.segOrdToMergedOrd.size() <= top.ord) {
top.segOrdToMergedOrd.add(signedDelta);
}
if (top.nextTerm() == null) {
q.pop();
} else {
q.updateTop();
}
}
numMergedTerms = ord;
// clear our bitsets for GC: we dont need them anymore (e.g. while flushing merged stuff to codec)
for (SegmentState state : segStates) {
state.liveTerms = null;
}
}
}
/**
* Merges the sorted docvalues from <code>toMerge</code>.
@ -373,40 +259,44 @@ public abstract class DocValuesConsumer implements Closeable {
* an Iterable that merges ordinals and values and filters deleted documents .
*/
public void mergeSortedField(FieldInfo fieldInfo, final MergeState mergeState, List<SortedDocValues> toMerge) throws IOException {
final SortedBytesMerger merger = new SortedBytesMerger();
final AtomicReader readers[] = mergeState.readers.toArray(new AtomicReader[toMerge.size()]);
final SortedDocValues dvs[] = toMerge.toArray(new SortedDocValues[toMerge.size()]);
// Does the heavy lifting to merge sort all "live" ords:
merger.merge(mergeState, toMerge);
// step 1: iterate thru each sub and mark terms still in use
TermsEnum liveTerms[] = new TermsEnum[dvs.length];
for (int sub = 0; sub < liveTerms.length; sub++) {
AtomicReader reader = readers[sub];
SortedDocValues dv = dvs[sub];
Bits liveDocs = reader.getLiveDocs();
if (liveDocs == null) {
liveTerms[sub] = new SortedDocValuesTermsEnum(dv);
} else {
FixedBitSet bitset = new FixedBitSet(dv.getValueCount());
for (int i = 0; i < reader.maxDoc(); i++) {
if (liveDocs.get(i)) {
bitset.set(dv.getOrd(i));
}
}
liveTerms[sub] = new BitsFilteredTermsEnum(new SortedDocValuesTermsEnum(dv), bitset);
}
}
// step 2: create ordinal map (this conceptually does the "merging")
final OrdinalMap map = new OrdinalMap(this, liveTerms);
// step 3: add field
addSortedField(fieldInfo,
// ord -> value
new Iterable<BytesRef>() {
@Override
public Iterator<BytesRef> iterator() {
// for each next(), tells us what reader to go to
final AppendingLongBuffer.Iterator readerIDs = merger.ordToReaderId.iterator();
// for each next(), gives us the original ord
final AppendingLongBuffer.Iterator ordDeltas[] = new AppendingLongBuffer.Iterator[merger.segStates.size()];
final int lastOrds[] = new int[ordDeltas.length];
for (int i = 0; i < ordDeltas.length; i++) {
ordDeltas[i] = merger.segStates.get(i).ordDeltas.iterator();
}
final BytesRef scratch = new BytesRef();
return new Iterator<BytesRef>() {
int ordUpto;
final BytesRef scratch = new BytesRef();
int currentOrd;
@Override
public boolean hasNext() {
return ordUpto < merger.numMergedTerms;
}
@Override
public void remove() {
throw new UnsupportedOperationException();
return currentOrd < map.getValueCount();
}
@Override
@ -414,17 +304,20 @@ public abstract class DocValuesConsumer implements Closeable {
if (!hasNext()) {
throw new NoSuchElementException();
}
int readerID = (int) readerIDs.next();
int ord = lastOrds[readerID] + (int) ordDeltas[readerID].next();
merger.segStates.get(readerID).values.lookupOrd(ord, scratch);
lastOrds[readerID] = ord;
ordUpto++;
int segmentNumber = map.getSegmentNumber(currentOrd);
int segmentOrd = (int)map.getSegmentOrd(segmentNumber, currentOrd);
dvs[segmentNumber].lookupOrd(segmentOrd, scratch);
currentOrd++;
return scratch;
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
};
}
},
// doc -> ord
new Iterable<Number>() {
@Override
@ -433,7 +326,7 @@ public abstract class DocValuesConsumer implements Closeable {
int readerUpto = -1;
int docIDUpto;
int nextValue;
SortedBytesMerger.SegmentState currentReader;
AtomicReader currentReader;
Bits currentLiveDocs;
boolean nextIsSet;
@ -460,15 +353,15 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == merger.segStates.size()) {
if (readerUpto == readers.length) {
return false;
}
if (currentReader == null || docIDUpto == currentReader.reader.maxDoc()) {
if (currentReader == null || docIDUpto == currentReader.maxDoc()) {
readerUpto++;
if (readerUpto < merger.segStates.size()) {
currentReader = merger.segStates.get(readerUpto);
currentLiveDocs = currentReader.reader.getLiveDocs();
if (readerUpto < readers.length) {
currentReader = readers[readerUpto];
currentLiveDocs = currentReader.getLiveDocs();
}
docIDUpto = 0;
continue;
@ -476,8 +369,8 @@ public abstract class DocValuesConsumer implements Closeable {
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
nextIsSet = true;
int segOrd = currentReader.values.getOrd(docIDUpto);
nextValue = (int) (segOrd + currentReader.segOrdToMergedOrd.get(segOrd));
int segOrd = dvs[readerUpto].getOrd(docIDUpto);
nextValue = (int) map.getGlobalOrd(readerUpto, segOrd);
docIDUpto++;
return true;
}
@ -487,7 +380,246 @@ public abstract class DocValuesConsumer implements Closeable {
}
};
}
});
}
);
}
/**
* Merges the sortedset docvalues from <code>toMerge</code>.
* <p>
* The default implementation calls {@link #addSortedSetField}, passing
* an Iterable that merges ordinals and values and filters deleted documents .
*/
public void mergeSortedSetField(FieldInfo fieldInfo, final MergeState mergeState, List<SortedSetDocValues> toMerge) throws IOException {
final AtomicReader readers[] = mergeState.readers.toArray(new AtomicReader[toMerge.size()]);
final SortedSetDocValues dvs[] = toMerge.toArray(new SortedSetDocValues[toMerge.size()]);
// step 1: iterate thru each sub and mark terms still in use
TermsEnum liveTerms[] = new TermsEnum[dvs.length];
for (int sub = 0; sub < liveTerms.length; sub++) {
AtomicReader reader = readers[sub];
SortedSetDocValues dv = dvs[sub];
Bits liveDocs = reader.getLiveDocs();
if (liveDocs == null) {
liveTerms[sub] = new SortedSetDocValuesTermsEnum(dv);
} else {
// nocommit: need a "pagedbits"
if (dv.getValueCount() > Integer.MAX_VALUE) {
throw new UnsupportedOperationException();
}
FixedBitSet bitset = new FixedBitSet((int)dv.getValueCount());
OrdIterator iterator = null;
for (int i = 0; i < reader.maxDoc(); i++) {
if (liveDocs.get(i)) {
iterator = dv.getOrds(i, iterator);
long ord;
while ((ord = iterator.nextOrd()) != OrdIterator.NO_MORE_ORDS) {
bitset.set((int)ord); // nocommit
}
}
}
liveTerms[sub] = new BitsFilteredTermsEnum(new SortedSetDocValuesTermsEnum(dv), bitset);
}
}
// step 2: create ordinal map (this conceptually does the "merging")
final OrdinalMap map = new OrdinalMap(this, liveTerms);
// step 3: add field
addSortedSetField(fieldInfo,
// ord -> value
new Iterable<BytesRef>() {
@Override
public Iterator<BytesRef> iterator() {
return new Iterator<BytesRef>() {
final BytesRef scratch = new BytesRef();
long currentOrd;
@Override
public boolean hasNext() {
return currentOrd < map.getValueCount();
}
@Override
public BytesRef next() {
if (!hasNext()) {
throw new NoSuchElementException();
}
int segmentNumber = map.getSegmentNumber(currentOrd);
long segmentOrd = map.getSegmentOrd(segmentNumber, currentOrd);
dvs[segmentNumber].lookupOrd(segmentOrd, scratch);
currentOrd++;
return scratch;
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
};
}
},
// doc -> ord count
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
return new Iterator<Number>() {
int readerUpto = -1;
int docIDUpto;
int nextValue;
AtomicReader currentReader;
OrdIterator iterator;
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;
// TODO make a mutable number
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;
iterator = dvs[readerUpto].getOrds(docIDUpto, iterator);
nextValue = 0;
while (iterator.nextOrd() != OrdIterator.NO_MORE_ORDS) {
nextValue++;
}
docIDUpto++;
return true;
}
docIDUpto++;
}
}
};
}
},
// ords
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
return new Iterator<Number>() {
int readerUpto = -1;
int docIDUpto;
long nextValue;
AtomicReader currentReader;
OrdIterator iterator;
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;
// TODO make a mutable number
return nextValue;
}
private boolean setNext() {
while (true) {
if (readerUpto == readers.length) {
return false;
}
if (iterator != null) {
final long segmentOrd = iterator.nextOrd();
if (segmentOrd != OrdIterator.NO_MORE_ORDS) {
nextValue = map.getGlobalOrd(readerUpto, segmentOrd);
nextIsSet = true;
return true;
} else {
docIDUpto++;
}
}
if (currentReader == null || docIDUpto == currentReader.maxDoc()) {
readerUpto++;
if (readerUpto < readers.length) {
currentReader = readers[readerUpto];
currentLiveDocs = currentReader.getLiveDocs();
}
docIDUpto = 0;
iterator = null;
continue;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
assert docIDUpto < currentReader.maxDoc();
iterator = dvs[readerUpto].getOrds(docIDUpto, iterator);
continue;
}
docIDUpto++;
}
}
};
}
}
);
}
// nocommit: need a "pagedbits"
static class BitsFilteredTermsEnum extends FilteredTermsEnum {
final Bits liveTerms;
BitsFilteredTermsEnum(TermsEnum in, Bits liveTerms) {
super(in, false); // <-- not passing false here wasted about 3 hours of my time!!!!!!!!!!!!!
assert liveTerms != null;
this.liveTerms = liveTerms;
}
@Override
protected AcceptStatus accept(BytesRef term) throws IOException {
if (liveTerms.get((int) ord())) {
return AcceptStatus.YES;
} else {
return AcceptStatus.NO;
}
}
}
}

View File

@ -24,6 +24,7 @@ import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedSetDocValues;
/** Abstract API that produces numeric, binary and
* sorted docvalues.
@ -50,4 +51,9 @@ public abstract class DocValuesProducer implements Closeable {
* The returned instance need not be thread-safe: it will only be
* used by a single thread. */
public abstract SortedDocValues getSorted(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. */
public abstract SortedSetDocValues getSortedSet(FieldInfo field) throws IOException;
}

View File

@ -31,6 +31,7 @@ import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
@ -614,6 +615,11 @@ final class Lucene40DocValuesReader extends DocValuesProducer {
};
}
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
throw new IllegalStateException("Lucene 4.0 does not support SortedSet: how did you pull this off?");
}
@Override
public void close() throws IOException {
dir.close();

View File

@ -20,13 +20,17 @@ package org.apache.lucene.codecs.lucene42;
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.NoSuchElementException;
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.store.ByteArrayDataOutput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRef;
@ -196,12 +200,7 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
}
}
@Override
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
// write the ordinals as numerics
addNumericField(field, docToOrd);
// write the values as FST
private void writeFST(FieldInfo field, Iterable<BytesRef> values) throws IOException {
meta.writeVInt(field.number);
meta.writeByte(FST);
meta.writeLong(data.getFilePointer());
@ -215,6 +214,91 @@ class Lucene42DocValuesConsumer extends DocValuesConsumer {
}
FST<Long> fst = builder.finish();
fst.save(data);
meta.writeVInt((int)ord);
meta.writeVLong(ord);
}
@Override
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
// write the ordinals as numerics
addNumericField(field, docToOrd);
// write the values as FST
writeFST(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 {
// 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);
}
// per-document vint-encoded byte[]
static class SortedSetIterator implements Iterator<BytesRef> {
byte[] buffer = new byte[10];
ByteArrayDataOutput out = new ByteArrayDataOutput();
BytesRef ref = new BytesRef();
final Iterator<Number> counts;
final Iterator<Number> ords;
SortedSetIterator(Iterator<Number> counts, Iterator<Number> ords) {
this.counts = counts;
this.ords = ords;
}
@Override
public boolean hasNext() {
return counts.hasNext();
}
@Override
public BytesRef next() {
if (!hasNext()) {
throw new NoSuchElementException();
}
int count = counts.next().intValue();
int maxSize = count*9; // worst case
if (maxSize > buffer.length) {
buffer = ArrayUtil.grow(buffer, maxSize);
}
try {
encodeValues(count);
} catch (IOException bogus) {
throw new RuntimeException(bogus);
}
ref.bytes = buffer;
ref.offset = 0;
ref.length = out.getPosition();
return ref;
}
// encodes count values to buffer
private void encodeValues(int count) throws IOException {
out.reset(buffer);
long lastOrd = 0;
for (int i = 0; i < count; i++) {
long ord = ords.next().longValue();
out.writeVLong(ord - lastOrd);
lastOrd = ord;
}
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
}
}

View File

@ -31,6 +31,9 @@ import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.SortedSetDocValues.OrdIterator;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -123,7 +126,7 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
} else if (fieldType == Lucene42DocValuesConsumer.FST) {
FSTEntry entry = new FSTEntry();
entry.offset = meta.readLong();
entry.numOrds = meta.readVInt();
entry.numOrds = meta.readVLong();
fsts.put(fieldNumber, entry);
} else {
throw new CorruptIndexException("invalid entry type: " + fieldType + ", input=" + meta);
@ -281,11 +284,111 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
@Override
public int getValueCount() {
return (int)entry.numOrds;
}
};
}
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
final FSTEntry entry = fsts.get(field.number);
FST<Long> instance;
synchronized(this) {
instance = fstInstances.get(field.number);
if (instance == null) {
data.seek(entry.offset);
instance = new FST<Long>(data, PositiveIntOutputs.getSingleton(true));
fstInstances.put(field.number, instance);
}
}
final BinaryDocValues docToOrds = getBinary(field);
final FST<Long> fst = instance;
// per-thread resources
final BytesReader in = fst.getBytesReader();
final Arc<Long> firstArc = new Arc<Long>();
final Arc<Long> scratchArc = new Arc<Long>();
final IntsRef scratchInts = new IntsRef();
final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<Long>(fst);
return new SortedSetDocValues() {
@Override
public OrdIterator getOrds(int docID, OrdIterator reuse) {
final Lucene42OrdsIterator iterator;
if (reuse instanceof Lucene42OrdsIterator) {
iterator = (Lucene42OrdsIterator) reuse;
} else {
iterator = new Lucene42OrdsIterator(docToOrds);
}
iterator.reset(docID);
return iterator;
}
@Override
public void lookupOrd(long ord, BytesRef result) {
try {
in.setPosition(0);
fst.getFirstArc(firstArc);
IntsRef output = Util.getByOutput(fst, ord, in, firstArc, scratchArc, scratchInts);
result.bytes = new byte[output.length];
result.offset = 0;
result.length = 0;
Util.toBytesRef(output, result);
} catch (IOException bogus) {
throw new RuntimeException(bogus);
}
}
@Override
public long lookupTerm(BytesRef key) {
try {
InputOutput<Long> o = fstEnum.seekCeil(key);
if (o == null) {
return -getValueCount()-1;
} else if (o.input.equals(key)) {
return o.output.intValue();
} else {
return -o.output-1;
}
} catch (IOException bogus) {
throw new RuntimeException(bogus);
}
}
@Override
public long getValueCount() {
return entry.numOrds;
}
};
}
static class Lucene42OrdsIterator extends OrdIterator {
final BinaryDocValues data;
final BytesRef ref = new BytesRef();
final ByteArrayDataInput input = new ByteArrayDataInput();
long currentOrd;
Lucene42OrdsIterator(BinaryDocValues data) {
this.data = data;
}
@Override
public long nextOrd() {
if (input.eof()) {
return NO_MORE_ORDS;
} else {
currentOrd += input.readVLong();
return currentOrd;
}
}
void reset(int docid) {
data.get(docid, ref);
input.reset(ref.bytes, ref.offset, ref.length);
currentOrd = 0;
}
}
@Override
public void close() throws IOException {
data.close();
@ -308,6 +411,6 @@ class Lucene42DocValuesProducer extends DocValuesProducer {
static class FSTEntry {
long offset;
int numOrds;
long numOrds;
}
}

View File

@ -114,6 +114,8 @@ final class Lucene42FieldInfosReader extends FieldInfosReader {
return DocValuesType.BINARY;
} else if (b == 3) {
return DocValuesType.SORTED;
} else if (b == 4) {
return DocValuesType.SORTED_SET;
} else {
throw new CorruptIndexException("invalid docvalues byte: " + b + " (resource=" + input + ")");
}

View File

@ -99,6 +99,8 @@ final class Lucene42FieldInfosWriter extends FieldInfosWriter {
return 2;
} else if (type == DocValuesType.SORTED) {
return 3;
} else if (type == DocValuesType.SORTED_SET) {
return 4;
} else {
throw new AssertionError();
}

View File

@ -35,6 +35,7 @@ import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -114,6 +115,11 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
getInstance(field).addSortedField(field, values, docToOrd);
}
@Override
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
getInstance(field).addSortedSetField(field, values, docToOrdCount, ords);
}
private DocValuesConsumer getInstance(FieldInfo field) throws IOException {
final DocValuesFormat format = getDocValuesFormatForField(field.name);
if (format == null) {
@ -254,6 +260,12 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
return producer == null ? null : producer.getSorted(field);
}
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
DocValuesProducer producer = fields.get(field.name);
return producer == null ? null : producer.getSortedSet(field);
}
@Override
public void close() throws IOException {
IOUtils.close(formats.values());

View File

@ -0,0 +1,61 @@
package org.apache.lucene.document;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.util.BytesRef;
/**
* <p>
* Field that stores
* a set of per-document {@link BytesRef} values, indexed for
* faceting,grouping,joining. Here's an example usage:
*
* <pre class="prettyprint">
* document.add(new SortedSetDocValuesField(name, new BytesRef("hello")));
* document.add(new SortedSetDocValuesField(name, new BytesRef("world")));
* </pre>
*
* <p>
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
*
* */
public class SortedSetDocValuesField extends StoredField {
/**
* Type for sorted bytes DocValues
*/
public static final FieldType TYPE = new FieldType();
static {
TYPE.setDocValueType(FieldInfo.DocValuesType.SORTED_SET);
TYPE.freeze();
}
/**
* Create a new sorted DocValues field.
* @param name field name
* @param bytes binary content
* @throws IllegalArgumentException if the field name is null
*/
public SortedSetDocValuesField(String name, BytesRef bytes) {
super(name, TYPE);
fieldsData = bytes;
}
}

View File

@ -175,6 +175,12 @@ public abstract class AtomicReader extends IndexReader {
* used by a single thread. */
public abstract SortedDocValues getSortedDocValues(String field) throws IOException;
/** Returns {@link SortedSetDocValues} for this field, or
* null if no {@link SortedSetDocValues} were indexed for
* this field. The returned instance should only be
* used by a single thread. */
public abstract SortedSetDocValues getSortedSetDocValues(String field) throws IOException;
/** Returns {@link NumericDocValues} representing norms
* for this field, or null if no {@link NumericDocValues}
* were indexed. The returned instance should only be

View File

@ -91,7 +91,7 @@ class BinaryDocValuesWriter extends DocValuesWriter {
private class BytesIterator implements Iterator<BytesRef> {
final BytesRef value = new BytesRef();
final AppendingLongBuffer.Iterator lengthsIterator = lengths.iterator();
final int size = lengths.size();
final int size = (int) lengths.size();
final int maxDoc;
int upto;
long byteOffset;

View File

@ -33,6 +33,7 @@ import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat; // javadocs
import org.apache.lucene.document.FieldType; // for javadocs
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.SortedSetDocValues.OrdIterator;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
@ -1333,6 +1334,50 @@ public class CheckIndex {
}
}
private static void checkSortedSetDocValues(String fieldName, AtomicReader reader, SortedSetDocValues dv) {
final long maxOrd = dv.getValueCount()-1;
// nocommit
FixedBitSet seenOrds = new FixedBitSet((int)dv.getValueCount());
long maxOrd2 = -1;
OrdIterator iterator = null;
for (int i = 0; i < reader.maxDoc(); i++) {
iterator = dv.getOrds(i, iterator);
long lastOrd = -1;
long ord;
while ((ord = iterator.nextOrd()) != OrdIterator.NO_MORE_ORDS) {
if (ord <= lastOrd) {
throw new RuntimeException("ords out of order: " + ord + " <= " + lastOrd + " for doc: " + i);
}
if (ord < 0 || ord > maxOrd) {
throw new RuntimeException("ord out of bounds: " + ord);
}
lastOrd = ord;
maxOrd2 = Math.max(maxOrd2, ord);
// nocommit
seenOrds.set((int)ord);
}
}
if (maxOrd != maxOrd2) {
throw new RuntimeException("dv for field: " + fieldName + " reports wrong maxOrd=" + maxOrd + " but this is not the case: " + maxOrd2);
}
if (seenOrds.cardinality() != dv.getValueCount()) {
throw new RuntimeException("dv for field: " + fieldName + " has holes in its ords, valueCount=" + dv.getValueCount() + " but only used: " + seenOrds.cardinality());
}
BytesRef lastValue = null;
BytesRef scratch = new BytesRef();
for (long i = 0; i <= maxOrd; i++) {
dv.lookupOrd(i, scratch);
assert scratch.isValid();
if (lastValue != null) {
if (scratch.compareTo(lastValue) <= 0) {
throw new RuntimeException("dv for field: " + fieldName + " has ords out of order: " + lastValue + " >=" + scratch);
}
}
lastValue = BytesRef.deepCopyOf(scratch);
}
}
private static void checkNumericDocValues(String fieldName, AtomicReader reader, NumericDocValues ndv) {
for (int i = 0; i < reader.maxDoc(); i++) {
ndv.get(i);
@ -1344,6 +1389,9 @@ public class CheckIndex {
case SORTED:
checkSortedDocValues(fi.name, reader, reader.getSortedDocValues(fi.name));
break;
case SORTED_SET:
checkSortedSetDocValues(fi.name, reader, reader.getSortedSetDocValues(fi.name));
break;
case BINARY:
checkBinaryDocValues(fi.name, reader, reader.getBinaryDocValues(fi.name));
break;

View File

@ -57,6 +57,8 @@ final class DocValuesProcessor extends StoredFieldsConsumer {
addBinaryField(fieldInfo, docID, field.binaryValue());
} else if (dvType == DocValuesType.SORTED) {
addSortedField(fieldInfo, docID, field.binaryValue());
} else if (dvType == DocValuesType.SORTED_SET) {
addSortedSetField(fieldInfo, docID, field.binaryValue());
} else if (dvType == DocValuesType.NUMERIC) {
if (!(field.numericValue() instanceof Long)) {
throw new IllegalArgumentException("illegal type " + field.numericValue().getClass() + ": DocValues types must be Long");
@ -123,6 +125,20 @@ final class DocValuesProcessor extends StoredFieldsConsumer {
sortedWriter.addValue(docID, value);
}
void addSortedSetField(FieldInfo fieldInfo, int docID, BytesRef value) {
DocValuesWriter writer = writers.get(fieldInfo.name);
SortedSetDocValuesWriter sortedSetWriter;
if (writer == null) {
sortedSetWriter = new SortedSetDocValuesWriter(fieldInfo, bytesUsed);
writers.put(fieldInfo.name, sortedSetWriter);
} else if (!(writer instanceof SortedSetDocValuesWriter)) {
throw new IllegalArgumentException("Incompatible DocValues type: field \"" + fieldInfo.name + "\" changed from " + getTypeDesc(writer) + " to sorted");
} else {
sortedSetWriter = (SortedSetDocValuesWriter) writer;
}
sortedSetWriter.addValue(docID, value);
}
void addNumericField(FieldInfo fieldInfo, int docID, long value) {
DocValuesWriter writer = writers.get(fieldInfo.name);
NumericDocValuesWriter numericWriter;

View File

@ -101,7 +101,14 @@ public final class FieldInfo {
* byte[]. The stored byte[] is presorted and allows access via document id,
* ordinal and by-value.
*/
SORTED
SORTED,
/**
* 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,
* ordinal and by-value.
*/
SORTED_SET
};
/**

View File

@ -423,6 +423,12 @@ public class FilterAtomicReader extends AtomicReader {
return in.getSortedDocValues(field);
}
@Override
public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
ensureOpen();
return in.getSortedSetDocValues(field);
}
@Override
public NumericDocValues getNormValues(String field) throws IOException {
ensureOpen();

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import org.apache.lucene.index.MultiTermsEnum.TermsEnumIndex;
@ -214,14 +215,62 @@ public class MultiDocValues {
if (!anyReal) {
return null;
} else {
OrdinalMap mapping = new OrdinalMap(r.getCoreCacheKey(), values);
TermsEnum enums[] = new TermsEnum[values.length];
for (int i = 0; i < values.length; i++) {
enums[i] = new SortedDocValuesTermsEnum(values[i]);
}
OrdinalMap mapping = new OrdinalMap(r.getCoreCacheKey(), enums);
return new MultiSortedDocValues(values, starts, mapping);
}
}
/** Returns a SortedSetDocValues 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
* with {@link AtomicReader#getSortedSetDocValues(String)}
* </p>
*/
public static SortedSetDocValues getSortedSetValues(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().getSortedSetDocValues(field);
}
boolean anyReal = false;
final SortedSetDocValues[] values = new SortedSetDocValues[size];
final int[] starts = new int[size+1];
for (int i = 0; i < size; i++) {
AtomicReaderContext context = leaves.get(i);
SortedSetDocValues v = context.reader().getSortedSetDocValues(field);
if (v == null) {
v = SortedSetDocValues.EMPTY;
} else {
anyReal = true;
}
values[i] = v;
starts[i] = context.docBase;
}
starts[size] = r.maxDoc();
if (!anyReal) {
return null;
} else {
TermsEnum enums[] = new TermsEnum[values.length];
for (int i = 0; i < values.length; i++) {
enums[i] = new SortedSetDocValuesTermsEnum(values[i]);
}
OrdinalMap mapping = new OrdinalMap(r.getCoreCacheKey(), enums);
return new MultiSortedSetDocValues(values, starts, mapping);
}
}
/** maps per-segment ordinals to/from global ordinal space */
// TODO: use more efficient packed ints structures (these are all positive values!)
static class OrdinalMap {
// TODO: use more efficient packed ints structures?
public static class OrdinalMap {
// cache key of whoever asked for this aweful thing
final Object owner;
// globalOrd -> (globalOrd - segmentOrd)
@ -231,7 +280,7 @@ public class MultiDocValues {
// segmentOrd -> (globalOrd - segmentOrd)
final AppendingLongBuffer ordDeltas[];
OrdinalMap(Object owner, SortedDocValues subs[]) throws IOException {
public OrdinalMap(Object owner, TermsEnum subs[]) throws IOException {
// create the ordinal mappings by pulling a termsenum over each sub's
// unique terms, and walking a multitermsenum over those
this.owner = owner;
@ -241,34 +290,52 @@ public class MultiDocValues {
for (int i = 0; i < ordDeltas.length; i++) {
ordDeltas[i] = new AppendingLongBuffer();
}
int segmentOrds[] = new int[subs.length];
long segmentOrds[] = new long[subs.length];
ReaderSlice slices[] = new ReaderSlice[subs.length];
TermsEnumIndex indexes[] = new TermsEnumIndex[slices.length];
for (int i = 0; i < slices.length; i++) {
slices[i] = new ReaderSlice(0, 0, i);
indexes[i] = new TermsEnumIndex(new SortedDocValuesTermsEnum(subs[i]), i);
indexes[i] = new TermsEnumIndex(subs[i], i);
}
MultiTermsEnum mte = new MultiTermsEnum(slices);
mte.reset(indexes);
int globalOrd = 0;
long globalOrd = 0;
while (mte.next() != null) {
TermsEnumWithSlice matches[] = mte.getMatchArray();
for (int i = 0; i < mte.getMatchCount(); i++) {
int subIndex = matches[i].index;
int delta = globalOrd - segmentOrds[subIndex];
assert delta >= 0;
long segmentOrd = matches[i].terms.ord();
long delta = globalOrd - segmentOrd;
// for each unique term, just mark the first subindex/delta where it occurs
if (i == 0) {
subIndexes.add(subIndex);
globalOrdDeltas.add(delta);
}
// for each per-segment ord, map it back to the global term.
while (segmentOrds[subIndex] <= segmentOrd) {
ordDeltas[subIndex].add(delta);
segmentOrds[subIndex]++;
}
}
globalOrd++;
}
}
public long getGlobalOrd(int subIndex, long segmentOrd) {
return segmentOrd + ordDeltas[subIndex].get(segmentOrd);
}
public long getSegmentOrd(int subIndex, long globalOrd) {
return globalOrd - globalOrdDeltas.get(globalOrd);
}
public int getSegmentNumber(long globalOrd) {
return (int) subIndexes.get(globalOrd);
}
public long getValueCount() {
return globalOrdDeltas.size();
}
}
/** implements SortedDocValues over n subs, using an OrdinalMap */
@ -289,19 +356,78 @@ public class MultiDocValues {
public int getOrd(int docID) {
int subIndex = ReaderUtil.subIndex(docID, docStarts);
int segmentOrd = values[subIndex].getOrd(docID - docStarts[subIndex]);
return (int) (segmentOrd + mapping.ordDeltas[subIndex].get(segmentOrd));
return (int) mapping.getGlobalOrd(subIndex, segmentOrd);
}
@Override
public void lookupOrd(int ord, BytesRef result) {
int subIndex = (int) mapping.subIndexes.get(ord);
int segmentOrd = (int) (ord - mapping.globalOrdDeltas.get(ord));
int subIndex = mapping.getSegmentNumber(ord);
int segmentOrd = (int) mapping.getSegmentOrd(subIndex, ord);
values[subIndex].lookupOrd(segmentOrd, result);
}
@Override
public int getValueCount() {
return mapping.globalOrdDeltas.size();
return (int) mapping.getValueCount();
}
}
/** implements MultiSortedDocValues over n subs, using an OrdinalMap */
static class MultiSortedSetDocValues extends SortedSetDocValues {
final int docStarts[];
final SortedSetDocValues values[];
final OrdinalMap mapping;
MultiSortedSetDocValues(SortedSetDocValues values[], int docStarts[], OrdinalMap mapping) throws IOException {
assert values.length == mapping.ordDeltas.length;
assert docStarts.length == values.length + 1;
this.values = values;
this.docStarts = docStarts;
this.mapping = mapping;
}
@Override
public OrdIterator getOrds(int docID, OrdIterator reuse) {
MultiOrdIterator iterator;
if (reuse instanceof MultiOrdIterator) {
iterator = (MultiOrdIterator) reuse;
} else {
iterator = new MultiOrdIterator();
}
iterator.reset(docID);
return iterator;
}
@Override
public void lookupOrd(long ord, BytesRef result) {
int subIndex = mapping.getSegmentNumber(ord);
long segmentOrd = mapping.getSegmentOrd(subIndex, ord);
values[subIndex].lookupOrd(segmentOrd, result);
}
@Override
public long getValueCount() {
return mapping.getValueCount();
}
class MultiOrdIterator extends OrdIterator {
private OrdIterator inner;
private int subIndex;
@Override
public long nextOrd() {
long segmentOrd = inner.nextOrd();
if (segmentOrd == NO_MORE_ORDS) {
return NO_MORE_ORDS;
} else {
return mapping.getGlobalOrd(subIndex, segmentOrd);
}
}
void reset(int docID) {
subIndex = ReaderUtil.subIndex(docID, docStarts);
inner = values[subIndex].getOrds(docID - docStarts[subIndex], inner);
}
}
}
}

View File

@ -497,7 +497,7 @@ public final class MultiTermsEnum extends TermsEnum {
final static class TermsEnumWithSlice {
private final ReaderSlice subSlice;
private TermsEnum terms;
TermsEnum terms;
public BytesRef current;
final int index;

View File

@ -50,7 +50,7 @@ class NumericDocValuesWriter extends DocValuesWriter {
}
// Fill in any holes:
for (int i = pending.size(); i < docID; ++i) {
for (int i = (int)pending.size(); i < docID; ++i) {
pending.add(MISSING);
}
@ -90,7 +90,7 @@ class NumericDocValuesWriter extends DocValuesWriter {
// iterates over the values we have in ram
private class NumericIterator implements Iterator<Number> {
final AppendingLongBuffer.Iterator iter = pending.iterator();
final int size = pending.size();
final int size = (int)pending.size();
final int maxDoc;
int upto;

View File

@ -284,6 +284,13 @@ public final class ParallelAtomicReader extends AtomicReader {
return reader == null ? null : reader.getSortedDocValues(field);
}
@Override
public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
ensureOpen();
AtomicReader reader = fieldToReader.get(field);
return reader == null ? null : reader.getSortedSetDocValues(field);
}
@Override
public NumericDocValues getNormValues(String field) throws IOException {
ensureOpen();

View File

@ -254,6 +254,34 @@ final class SegmentCoreReaders {
return dvs;
}
SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
FieldInfo fi = fieldInfos.fieldInfo(field);
if (fi == null) {
// Field does not exist
return null;
}
if (fi.getDocValuesType() == null) {
// Field was not indexed with doc values
return null;
}
if (fi.getDocValuesType() != DocValuesType.SORTED_SET) {
// DocValues were not sorted
return null;
}
assert dvProducer != null;
Map<String,Object> dvFields = docValuesLocal.get();
SortedSetDocValues dvs = (SortedSetDocValues) dvFields.get(field);
if (dvs == null) {
dvs = dvProducer.getSortedSet(fi);
dvFields.put(field, dvs);
}
return dvs;
}
NumericDocValues getNormValues(String field) throws IOException {
FieldInfo fi = fieldInfos.fieldInfo(field);
if (fi == null) {

View File

@ -199,6 +199,16 @@ final class SegmentMerger {
toMerge.add(values);
}
consumer.mergeSortedField(field, mergeState, toMerge);
} else if (type == DocValuesType.SORTED_SET) {
List<SortedSetDocValues> toMerge = new ArrayList<SortedSetDocValues>();
for (AtomicReader reader : mergeState.readers) {
SortedSetDocValues values = reader.getSortedSetDocValues(field.name);
if (values == null) {
values = SortedSetDocValues.EMPTY;
}
toMerge.add(values);
}
consumer.mergeSortedSetField(field, mergeState, toMerge);
} else {
throw new AssertionError("type=" + type);
}

View File

@ -247,6 +247,12 @@ public final class SegmentReader extends AtomicReader {
return core.getSortedDocValues(field);
}
@Override
public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
ensureOpen();
return core.getSortedSetDocValues(field);
}
@Override
public NumericDocValues getNormValues(String field) throws IOException {
ensureOpen();

View File

@ -25,6 +25,7 @@ import org.apache.lucene.util.Bits;
import org.apache.lucene.index.DirectoryReader; // javadoc
import org.apache.lucene.index.MultiDocValues.MultiSortedDocValues;
import org.apache.lucene.index.MultiDocValues.MultiSortedSetDocValues;
import org.apache.lucene.index.MultiDocValues.OrdinalMap;
import org.apache.lucene.index.MultiReader; // javadoc
@ -131,6 +132,42 @@ public final class SlowCompositeReaderWrapper extends AtomicReader {
return new MultiSortedDocValues(values, starts, map);
}
@Override
public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
ensureOpen();
OrdinalMap map = null;
synchronized (cachedOrdMaps) {
map = cachedOrdMaps.get(field);
if (map == null) {
// uncached, or not a multi dv
SortedSetDocValues dv = MultiDocValues.getSortedSetValues(in, field);
if (dv instanceof MultiSortedSetDocValues) {
map = ((MultiSortedSetDocValues)dv).mapping;
if (map.owner == getCoreCacheKey()) {
cachedOrdMaps.put(field, map);
}
}
return dv;
}
}
// cached multi dv
assert map != null;
int size = in.leaves().size();
final SortedSetDocValues[] values = new SortedSetDocValues[size];
final int[] starts = new int[size+1];
for (int i = 0; i < size; i++) {
AtomicReaderContext context = in.leaves().get(i);
SortedSetDocValues v = context.reader().getSortedSetDocValues(field);
if (v == null) {
v = SortedSetDocValues.EMPTY;
}
values[i] = v;
starts[i] = context.docBase;
}
starts[size] = maxDoc();
return new MultiSortedSetDocValues(values, starts, map);
}
// TODO: this could really be a weak map somewhere else on the coreCacheKey,
// but do we really need to optimize slow-wrapper any more?
private final Map<String,OrdinalMap> cachedOrdMaps = new HashMap<String,OrdinalMap>();

View File

@ -0,0 +1,120 @@
package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.util.BytesRef;
/**
* A per-document set of presorted byte[] values.
* <p>
* Per-Document values in a SortedDocValues are deduplicated, dereferenced,
* and sorted into a dictionary of unique values. A pointer to the
* dictionary value (ordinal) can be retrieved for each document. Ordinals
* are dense and in increasing sorted order.
*/
public abstract class SortedSetDocValues {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected SortedSetDocValues() {}
/**
* Returns an iterator over the ordinals for the specified docID.
* @param docID document ID to lookup
* @return iterator over ordinals for the document: these are dense,
* start at 0, then increment by 1 for the next value in sorted order.
*/
public abstract OrdIterator getOrds(int docID, OrdIterator reuse);
/** Retrieves the value for the specified ordinal.
* @param ord ordinal to lookup
* @param result will be populated with the ordinal's value
* @see #getOrds
*/
public abstract void lookupOrd(long ord, BytesRef result);
/**
* Returns the number of unique values.
* @return number of unique values in this SortedDocValues. This is
* also equivalent to one plus the maximum ordinal.
*/
public abstract long getValueCount();
/** An empty SortedDocValues which returns {@link OrdIterator#EMPTY} for every document */
public static final SortedSetDocValues EMPTY = new SortedSetDocValues() {
@Override
public OrdIterator getOrds(int docID, OrdIterator reuse) {
return OrdIterator.EMPTY;
}
@Override
public void lookupOrd(long ord, BytesRef result) {
throw new IndexOutOfBoundsException();
}
@Override
public long getValueCount() {
return 0;
}
};
/** If {@code key} exists, returns its ordinal, else
* returns {@code -insertionPoint-1}, like {@code
* Arrays.binarySearch}.
*
* @param key Key to look up
**/
public long lookupTerm(BytesRef key) {
BytesRef spare = new BytesRef();
long low = 0;
long high = getValueCount()-1;
while (low <= high) {
long mid = (low + high) >>> 1;
lookupOrd(mid, spare);
int cmp = spare.compareTo(key);
if (cmp < 0) {
low = mid + 1;
} else if (cmp > 0) {
high = mid - 1;
} else {
return mid; // key found
}
}
return -(low + 1); // key not found.
}
/** An iterator over the ordinals in a document (in increasing order) */
public static abstract class OrdIterator {
/** Indicates enumeration has ended: no more ordinals for this document */
public static final long NO_MORE_ORDS = Long.MAX_VALUE;
/** An iterator that always returns {@link #NO_MORE_ORDS} */
public static final OrdIterator EMPTY = new OrdIterator() {
@Override
public long nextOrd() {
return NO_MORE_ORDS;
}
};
/** Returns next ordinal, or {@link #NO_MORE_ORDS} */
public abstract long nextOrd();
}
}

View File

@ -0,0 +1,141 @@
package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.Comparator;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
/** Implements a {@link TermsEnum} wrapping a provided
* {@link SortedSetDocValues}. */
// nocommit: if we are ok with ords being 'long' for SortedDocValues,
// then we don't need this...
public class SortedSetDocValuesTermsEnum extends TermsEnum {
private final SortedSetDocValues values;
private long currentOrd = -1;
private final BytesRef term = new BytesRef();
/** Creates a new TermsEnum over the provided values */
public SortedSetDocValuesTermsEnum(SortedSetDocValues values) {
this.values = values;
}
@Override
public SeekStatus seekCeil(BytesRef text, boolean useCache /* ignored */) throws IOException {
long ord = values.lookupTerm(text);
if (ord >= 0) {
currentOrd = ord;
term.offset = 0;
// TODO: is there a cleaner way?
// term.bytes may be pointing to codec-private byte[]
// storage, so we must force new byte[] allocation:
term.bytes = new byte[text.length];
term.copyBytes(text);
return SeekStatus.FOUND;
} else {
currentOrd = -ord-1;
if (currentOrd == values.getValueCount()) {
return SeekStatus.END;
} else {
// TODO: hmm can we avoid this "extra" lookup?:
values.lookupOrd(currentOrd, term);
return SeekStatus.NOT_FOUND;
}
}
}
@Override
public boolean seekExact(BytesRef text, boolean useCache) throws IOException {
long ord = values.lookupTerm(text);
if (ord >= 0) {
currentOrd = ord;
return true;
} else {
return false;
}
}
@Override
public void seekExact(long ord) throws IOException {
assert ord >= 0 && ord < values.getValueCount();
currentOrd = (int) ord;
values.lookupOrd(currentOrd, term);
}
@Override
public BytesRef next() throws IOException {
currentOrd++;
if (currentOrd >= values.getValueCount()) {
return null;
}
values.lookupOrd(currentOrd, term);
return term;
}
@Override
public BytesRef term() throws IOException {
return term;
}
@Override
public long ord() throws IOException {
return currentOrd;
}
@Override
public int docFreq() {
throw new UnsupportedOperationException();
}
@Override
public long totalTermFreq() {
return -1;
}
@Override
public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public Comparator<BytesRef> getComparator() {
return BytesRef.getUTF8SortedAsUnicodeComparator();
}
@Override
public void seekExact(BytesRef term, TermState state) throws IOException {
assert state != null && state instanceof OrdTermState;
this.seekExact(((OrdTermState)state).ord);
}
@Override
public TermState termState() throws IOException {
OrdTermState state = new OrdTermState();
state.ord = currentOrd;
return state;
}
}

View File

@ -0,0 +1,285 @@
package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
import java.io.IOException;
import java.util.Arrays;
import java.util.Iterator;
import java.util.NoSuchElementException;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
import org.apache.lucene.util.BytesRefHash;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.AppendingLongBuffer;
/** Buffers up pending byte[]s per doc, deref and sorting via
* int ord, then flushes when segment flushes. */
class SortedSetDocValuesWriter extends DocValuesWriter {
final BytesRefHash hash;
private AppendingLongBuffer pending; // stream of all ords
private AppendingLongBuffer pendingCounts; // ords per doc
private final Counter iwBytesUsed;
private long bytesUsed; // this only tracks differences in 'pending' and 'pendingCounts'
private final FieldInfo fieldInfo;
private int currentDoc;
private int currentValues[] = new int[8];
private int currentUpto = 0;
public SortedSetDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
this.fieldInfo = fieldInfo;
this.iwBytesUsed = iwBytesUsed;
hash = new BytesRefHash(
new ByteBlockPool(
new ByteBlockPool.DirectTrackingAllocator(iwBytesUsed)),
BytesRefHash.DEFAULT_CAPACITY,
new DirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, iwBytesUsed));
pending = new AppendingLongBuffer();
pendingCounts = new AppendingLongBuffer();
bytesUsed = pending.ramBytesUsed() + pendingCounts.ramBytesUsed();
iwBytesUsed.addAndGet(bytesUsed);
}
public void addValue(int docID, BytesRef value) {
if (value == null) {
throw new IllegalArgumentException("field \"" + fieldInfo.name + "\": null value not allowed");
}
if (value.length > (BYTE_BLOCK_SIZE - 2)) {
throw new IllegalArgumentException("DocValuesField \"" + fieldInfo.name + "\" is too large, must be <= " + (BYTE_BLOCK_SIZE - 2));
}
if (docID != currentDoc) {
finishCurrentDoc();
}
// Fill in any holes:
while(currentDoc < docID) {
pendingCounts.add(0); // no values
currentDoc++;
}
addOneValue(value);
updateBytesUsed();
}
// finalize currentDoc
private void finishCurrentDoc() {
Arrays.sort(currentValues, 0, currentUpto);
int lastValue = -1;
int count = 0;
for (int i = 0; i < currentUpto; i++) {
int v = currentValues[i];
// if its not a duplicate
if (v != lastValue) {
pending.add(v); // record the ord
count++;
}
lastValue = v;
}
// record the number of unique ords for this doc
pendingCounts.add(count);
currentUpto = 0;
currentDoc++;
}
@Override
public void finish(int maxDoc) {
finishCurrentDoc();
// fill in any holes
for (int i = currentDoc; i < maxDoc; i++) {
pendingCounts.add(0); // no values
}
}
private void addOneValue(BytesRef value) {
int ord = hash.add(value);
if (ord < 0) {
ord = -ord-1;
} else {
// reserve additional space for each unique value:
// 1. when indexing, when hash is 50% full, rehash() suddenly needs 2*size ints.
// TODO: can this same OOM happen in THPF?
// 2. when flushing, we need 1 int per value (slot in the ordMap).
iwBytesUsed.addAndGet(2 * RamUsageEstimator.NUM_BYTES_INT);
}
if (currentUpto == currentValues.length) {
currentValues = ArrayUtil.grow(currentValues, currentValues.length+1);
iwBytesUsed.addAndGet((currentValues.length - currentUpto) * RamUsageEstimator.NUM_BYTES_INT);
}
currentValues[currentUpto] = ord;
currentUpto++;
}
private void updateBytesUsed() {
final long newBytesUsed = pending.ramBytesUsed() + pendingCounts.ramBytesUsed();
iwBytesUsed.addAndGet(newBytesUsed - bytesUsed);
bytesUsed = newBytesUsed;
}
@Override
public void flush(SegmentWriteState state, DocValuesConsumer dvConsumer) throws IOException {
final int maxDoc = state.segmentInfo.getDocCount();
assert pendingCounts.size() == maxDoc;
final int valueCount = hash.size();
final int[] sortedValues = hash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
final int[] ordMap = new int[valueCount];
for(int ord=0;ord<valueCount;ord++) {
ordMap[sortedValues[ord]] = ord;
}
dvConsumer.addSortedSetField(fieldInfo,
// ord -> value
new Iterable<BytesRef>() {
@Override
public Iterator<BytesRef> iterator() {
return new ValuesIterator(sortedValues, valueCount);
}
},
// doc -> ordCount
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
return new OrdCountIterator(maxDoc);
}
},
// ords
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
return new OrdsIterator(ordMap);
}
});
}
@Override
public void abort() {
}
// iterates over the unique values we have in ram
private class ValuesIterator implements Iterator<BytesRef> {
final int sortedValues[];
final BytesRef scratch = new BytesRef();
final int valueCount;
int ordUpto;
ValuesIterator(int sortedValues[], int valueCount) {
this.sortedValues = sortedValues;
this.valueCount = valueCount;
}
@Override
public boolean hasNext() {
return ordUpto < valueCount;
}
@Override
public BytesRef next() {
if (!hasNext()) {
throw new NoSuchElementException();
}
hash.get(sortedValues[ordUpto], scratch);
ordUpto++;
return scratch;
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
}
// iterates over the ords for each doc we have in ram
private class OrdsIterator implements Iterator<Number> {
final AppendingLongBuffer.Iterator iter = pending.iterator();
final int ordMap[];
final long numOrds;
long ordUpto;
OrdsIterator(int ordMap[]) {
this.ordMap = ordMap;
this.numOrds = pending.size();
}
@Override
public boolean hasNext() {
return ordUpto < numOrds;
}
@Override
public Number next() {
if (!hasNext()) {
throw new NoSuchElementException();
}
int ord = (int) iter.next();
ordUpto++;
// TODO: make reusable Number
return ordMap[ord];
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
}
private class OrdCountIterator implements Iterator<Number> {
final AppendingLongBuffer.Iterator iter = pendingCounts.iterator();
final int maxDoc;
int docUpto;
OrdCountIterator(int maxDoc) {
this.maxDoc = maxDoc;
assert pendingCounts.size() == maxDoc;
}
@Override
public boolean hasNext() {
return docUpto < maxDoc;
}
@Override
public Number next() {
if (!hasNext()) {
throw new NoSuchElementException();
}
docUpto++;
// TODO: make reusable Number
return iter.next();
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
}
}

View File

@ -61,11 +61,11 @@ public class AppendingLongBuffer {
* <p>
* <b>NOTE</b>: This class is not really designed for random access!
* You will likely get better performance by using packed ints in another way! */
public long get(int index) {
assert index < size(); // TODO: do a better check, and throw IndexOutOfBoundsException?
public long get(long index) {
assert index < size() : "index=" + index + ",size=" + size(); // TODO: do a better check, and throw IndexOutOfBoundsException?
// This class is currently only used by the indexer.
int block = index >> BLOCK_BITS;
int element = index & BLOCK_MASK;
int block = (int) (index >> BLOCK_BITS);
int element = (int) (index & BLOCK_MASK);
if (block == valuesOff) {
return pending[element];
} else if (values[block] == null) {
@ -115,8 +115,8 @@ public class AppendingLongBuffer {
}
/** Get the number of values that have been added to the buffer. */
public int size() {
return valuesOff * MAX_PENDING_COUNT + pendingOff;
public long size() {
return valuesOff * (long)MAX_PENDING_COUNT + pendingOff;
}
/** Return an iterator over the values of this buffer. */

View File

@ -0,0 +1,216 @@
package org.apache.lucene;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.SortedSetDocValues.OrdIterator;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
/**
* A very simple demo used in the API documentation (src/java/overview.html).
*
* Please try to keep src/java/overview.html up-to-date when making changes
* to this class.
*/
public class TestDemoDocValue extends LuceneTestCase {
// nocommit: only Lucene42/Asserting implemented right now
private Codec saved;
@Override
public void setUp() throws Exception {
super.setUp();
saved = Codec.getDefault();
Codec.setDefault(_TestUtil.alwaysDocValuesFormat(DocValuesFormat.forName("Asserting")));
}
@Override
public void tearDown() throws Exception {
Codec.setDefault(saved);
super.tearDown();
}
public void testOneValue() throws IOException {
Analyzer analyzer = new MockAnalyzer(random());
// Store the index in memory:
Directory directory = newDirectory();
// To store an index on disk, use this instead:
// Directory directory = FSDirectory.open(new File("/tmp/testindex"));
RandomIndexWriter iwriter = new RandomIndexWriter(random(), directory, analyzer);
Document doc = new Document();
doc.add(new SortedSetDocValuesField("field", new BytesRef("hello")));
iwriter.addDocument(doc);
iwriter.close();
// Now search the index:
DirectoryReader ireader = DirectoryReader.open(directory); // read-only=true
SortedSetDocValues dv = getOnlySegmentReader(ireader).getSortedSetDocValues("field");
OrdIterator oi = dv.getOrds(0, null);
assertEquals(0, oi.nextOrd());
assertEquals(OrdIterator.NO_MORE_ORDS, oi.nextOrd());
BytesRef bytes = new BytesRef();
dv.lookupOrd(0, bytes);
assertEquals(new BytesRef("hello"), bytes);
ireader.close();
directory.close();
}
public void testTwoDocumentsMerged() throws IOException {
Analyzer analyzer = new MockAnalyzer(random());
// Store the index in memory:
Directory directory = newDirectory();
// To store an index on disk, use this instead:
// Directory directory = FSDirectory.open(new File("/tmp/testindex"));
IndexWriterConfig iwconfig = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
iwconfig.setMergePolicy(newLogMergePolicy());
RandomIndexWriter iwriter = new RandomIndexWriter(random(), directory, iwconfig);
Document doc = new Document();
doc.add(new SortedSetDocValuesField("field", new BytesRef("hello")));
iwriter.addDocument(doc);
iwriter.commit();
doc = new Document();
doc.add(new SortedSetDocValuesField("field", new BytesRef("world")));
iwriter.addDocument(doc);
iwriter.forceMerge(1);
iwriter.close();
// Now search the index:
DirectoryReader ireader = DirectoryReader.open(directory); // read-only=true
SortedSetDocValues dv = getOnlySegmentReader(ireader).getSortedSetDocValues("field");
OrdIterator oi = dv.getOrds(0, null);
assertEquals(0, oi.nextOrd());
assertEquals(OrdIterator.NO_MORE_ORDS, oi.nextOrd());
BytesRef bytes = new BytesRef();
dv.lookupOrd(0, bytes);
assertEquals(new BytesRef("hello"), bytes);
oi = dv.getOrds(1, oi);
assertEquals(1, oi.nextOrd());
assertEquals(OrdIterator.NO_MORE_ORDS, oi.nextOrd());
dv.lookupOrd(1, bytes);
assertEquals(new BytesRef("world"), bytes);
assertEquals(2, dv.getValueCount());
ireader.close();
directory.close();
}
public void testTwoValues() throws IOException {
Analyzer analyzer = new MockAnalyzer(random());
// Store the index in memory:
Directory directory = newDirectory();
// To store an index on disk, use this instead:
// Directory directory = FSDirectory.open(new File("/tmp/testindex"));
RandomIndexWriter iwriter = new RandomIndexWriter(random(), directory, analyzer);
Document doc = new Document();
doc.add(new SortedSetDocValuesField("field", new BytesRef("hello")));
doc.add(new SortedSetDocValuesField("field", new BytesRef("world")));
iwriter.addDocument(doc);
iwriter.close();
// Now search the index:
DirectoryReader ireader = DirectoryReader.open(directory); // read-only=true
SortedSetDocValues dv = getOnlySegmentReader(ireader).getSortedSetDocValues("field");
OrdIterator oi = dv.getOrds(0, null);
assertEquals(0, oi.nextOrd());
assertEquals(1, oi.nextOrd());
assertEquals(OrdIterator.NO_MORE_ORDS, oi.nextOrd());
BytesRef bytes = new BytesRef();
dv.lookupOrd(0, bytes);
assertEquals(new BytesRef("hello"), bytes);
dv.lookupOrd(1, bytes);
assertEquals(new BytesRef("world"), bytes);
ireader.close();
directory.close();
}
public void testThreeValuesTwoDocs() throws IOException {
Analyzer analyzer = new MockAnalyzer(random());
// Store the index in memory:
Directory directory = newDirectory();
// To store an index on disk, use this instead:
// Directory directory = FSDirectory.open(new File("/tmp/testindex"));
IndexWriterConfig iwconfig = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
iwconfig.setMergePolicy(newLogMergePolicy());
RandomIndexWriter iwriter = new RandomIndexWriter(random(), directory, iwconfig);
Document doc = new Document();
doc.add(new SortedSetDocValuesField("field", new BytesRef("hello")));
doc.add(new SortedSetDocValuesField("field", new BytesRef("world")));
iwriter.addDocument(doc);
iwriter.commit();
doc = new Document();
doc.add(new SortedSetDocValuesField("field", new BytesRef("hello")));
doc.add(new SortedSetDocValuesField("field", new BytesRef("beer")));
iwriter.addDocument(doc);
iwriter.forceMerge(1);
iwriter.close();
// Now search the index:
DirectoryReader ireader = DirectoryReader.open(directory); // read-only=true
SortedSetDocValues dv = getOnlySegmentReader(ireader).getSortedSetDocValues("field");
assertEquals(3, dv.getValueCount());
OrdIterator oi = dv.getOrds(0, null);
assertEquals(1, oi.nextOrd());
assertEquals(2, oi.nextOrd());
assertEquals(OrdIterator.NO_MORE_ORDS, oi.nextOrd());
oi = dv.getOrds(1, null);
assertEquals(0, oi.nextOrd());
assertEquals(1, oi.nextOrd());
assertEquals(OrdIterator.NO_MORE_ORDS, oi.nextOrd());
BytesRef bytes = new BytesRef();
dv.lookupOrd(0, bytes);
assertEquals(new BytesRef("beer"), bytes);
dv.lookupOrd(1, bytes);
assertEquals(new BytesRef("hello"), bytes);
dv.lookupOrd(2, bytes);
assertEquals(new BytesRef("world"), bytes);
ireader.close();
directory.close();
}
}

View File

@ -46,6 +46,7 @@ import org.apache.lucene.index.Fields;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.OrdTermState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.Terms;
@ -753,6 +754,11 @@ public class MemoryIndex {
return null;
}
@Override
public SortedSetDocValues getSortedSetDocValues(String field) {
return null;
}
private class MemoryFields extends Fields {
@Override
public Iterator<String> iterator() {

View File

@ -32,6 +32,7 @@ import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
@ -127,6 +128,12 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
in.addSortedField(field, values, docToOrd);
}
@Override
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
// nocommit: add checks
in.addSortedSetField(field, values, docToOrdCount, ords);
}
private <T> void checkIterator(Iterator<T> iterator, int expectedSize) {
for (int i = 0; i < expectedSize; i++) {
boolean hasNext = iterator.hasNext();
@ -189,6 +196,14 @@ public class AssertingDocValuesFormat extends DocValuesFormat {
return new AssertingAtomicReader.AssertingSortedDocValues(values, maxDoc);
}
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
assert field.getDocValuesType() == FieldInfo.DocValuesType.SORTED_SET;
SortedSetDocValues values = in.getSortedSet(field);
assert values != null;
return new AssertingAtomicReader.AssertingSortedSetDocValues(values, maxDoc);
}
@Override
public void close() throws IOException {
in.close();

View File

@ -31,6 +31,7 @@ import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
@ -231,6 +232,11 @@ class CheapBastardDocValuesProducer extends DocValuesProducer {
};
}
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
throw new UnsupportedOperationException(); // nocommit
}
@Override
public void close() throws IOException {
data.close();

View File

@ -493,6 +493,11 @@ class Lucene40DocValuesWriter extends DocValuesConsumer {
ords.finish();
}
@Override
public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
throw new UnsupportedOperationException("Lucene 4.0 does not support SortedSet docvalues");
}
@Override
public void close() throws IOException {
dir.close();

View File

@ -3,6 +3,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.Iterator;
import org.apache.lucene.index.SortedSetDocValues.OrdIterator;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@ -454,6 +455,78 @@ public class AssertingAtomicReader extends FilterAtomicReader {
}
}
/** Wraps a SortedSetDocValues but with additional asserts */
public static class AssertingSortedSetDocValues extends SortedSetDocValues {
private final SortedSetDocValues in;
private final int maxDoc;
private final long valueCount;
public AssertingSortedSetDocValues(SortedSetDocValues in, int maxDoc) {
this.in = in;
this.maxDoc = maxDoc;
this.valueCount = in.getValueCount();
assert valueCount >= 0;
}
@Override
public OrdIterator getOrds(int docID, OrdIterator reuse) {
assert docID >= 0 && docID < maxDoc : "docid=" + docID + ",maxDoc=" + maxDoc;
if (reuse instanceof AssertingOrdIterator) {
reuse = ((AssertingOrdIterator) reuse).in;
}
OrdIterator iterator = in.getOrds(docID, reuse);
assert iterator != null;
return new AssertingOrdIterator(iterator, valueCount);
}
@Override
public void lookupOrd(long ord, BytesRef result) {
assert ord >= 0 && ord < valueCount;
assert result.isValid();
in.lookupOrd(ord, result);
assert result.isValid();
}
@Override
public long getValueCount() {
long valueCount = in.getValueCount();
assert valueCount == this.valueCount; // should not change
return valueCount;
}
@Override
public long lookupTerm(BytesRef key) {
assert key.isValid();
long result = in.lookupTerm(key);
assert result < valueCount;
assert key.isValid();
return result;
}
}
/** Wraps a OrdIterator but with additional asserts */
public static class AssertingOrdIterator extends OrdIterator {
final OrdIterator in;
final long valueCount;
long lastOrd = Long.MIN_VALUE;
AssertingOrdIterator(OrdIterator in, long valueCount) {
this.in = in;
this.valueCount = valueCount;
assert lastOrd != NO_MORE_ORDS;
}
@Override
public long nextOrd() {
assert lastOrd != NO_MORE_ORDS;
long ord = in.nextOrd();
assert ord == NO_MORE_ORDS || ord < valueCount;
assert ord > lastOrd;
lastOrd = ord;
return ord;
}
}
@Override
public NumericDocValues getNumericDocValues(String field) throws IOException {
NumericDocValues dv = super.getNumericDocValues(field);
@ -496,6 +569,20 @@ public class AssertingAtomicReader extends FilterAtomicReader {
}
}
@Override
public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
SortedSetDocValues dv = super.getSortedSetDocValues(field);
FieldInfo fi = getFieldInfos().fieldInfo(field);
if (dv != null) {
assert fi != null;
assert fi.getDocValuesType() == FieldInfo.DocValuesType.SORTED_SET;
return new AssertingSortedSetDocValues(dv, maxDoc());
} else {
assert fi == null || fi.getDocValuesType() != FieldInfo.DocValuesType.SORTED_SET;
return null;
}
}
@Override
public NumericDocValues getNormValues(String field) throws IOException {
NumericDocValues dv = super.getNormValues(field);

View File

@ -33,6 +33,7 @@ import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.index.MultiReader;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
@ -402,6 +403,11 @@ public class TestDocSet extends LuceneTestCase {
return null;
}
@Override
public SortedSetDocValues getSortedSetDocValues(String field) {
return null;
}
@Override
public NumericDocValues getNormValues(String field) {
return null;