LUCENE-6766: initial patch

This commit is contained in:
Mike McCandless 2016-05-04 05:37:37 -04:00
parent 7448abb3bc
commit fdc220eeb4
61 changed files with 2486 additions and 1832 deletions

View File

@ -36,6 +36,7 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
private IndexOutput out;
private final BytesRefBuilder scratch = new BytesRefBuilder();
private final SegmentWriteState writeState;
final String segment;
final static BytesRef END = new BytesRef("END");
final static BytesRef FIELD = new BytesRef("field ");
@ -49,6 +50,7 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
public SimpleTextFieldsWriter(SegmentWriteState writeState) throws IOException {
final String fileName = SimpleTextPostingsFormat.getPostingsFileName(writeState.segmentInfo.name, writeState.segmentSuffix);
segment = writeState.segmentInfo.name;
out = writeState.directory.createOutput(fileName, writeState.context);
this.writeState = writeState;
}

View File

@ -31,6 +31,8 @@ import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
@ -59,6 +61,11 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
final static BytesRef SI_NUM_FILES = new BytesRef(" files ");
final static BytesRef SI_FILE = new BytesRef(" file ");
final static BytesRef SI_ID = new BytesRef(" id ");
final static BytesRef SI_SORT = new BytesRef(" sort ");
final static BytesRef SI_SORT_FIELD = new BytesRef(" field ");
final static BytesRef SI_SORT_TYPE = new BytesRef(" type ");
final static BytesRef SI_SORT_REVERSE = new BytesRef(" reverse ");
final static BytesRef SI_SORT_MISSING = new BytesRef(" missing ");
public static final String SI_EXTENSION = "si";
@ -137,10 +144,93 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
+ ", got: " + StringHelper.idToString(id), input);
}
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch.get(), SI_SORT);
final int numSortFields = Integer.parseInt(readString(SI_SORT.length, scratch));
SortField[] sortField = new SortField[numSortFields];
for (int i = 0; i < numSortFields; ++i) {
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch.get(), SI_SORT_FIELD);
final String field = readString(SI_SORT_FIELD.length, scratch);
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch.get(), SI_SORT_TYPE);
final String typeAsString = readString(SI_SORT_TYPE.length, scratch);
final SortField.Type type;
switch (typeAsString) {
case "string":
type = SortField.Type.STRING;
break;
case "long":
type = SortField.Type.LONG;
break;
case "int":
type = SortField.Type.INT;
break;
default:
throw new CorruptIndexException("unable to parse sort type string: " + typeAsString, input);
}
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch.get(), SI_SORT_REVERSE);
final boolean reverse = Boolean.parseBoolean(readString(SI_SORT_REVERSE.length, scratch));
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch.get(), SI_SORT_MISSING);
final String missingLastAsString = readString(SI_SORT_MISSING.length, scratch);
final Object missingValue;
switch (type) {
case STRING:
switch (missingLastAsString) {
case "null":
missingValue = null;
break;
case "first":
missingValue = SortField.STRING_FIRST;
break;
case "last":
missingValue = SortField.STRING_LAST;
break;
default:
throw new CorruptIndexException("unable to parse missing string: " + typeAsString, input);
}
break;
case LONG:
switch (missingLastAsString) {
case "null":
missingValue = null;
break;
default:
missingValue = Long.parseLong(missingLastAsString);
break;
}
break;
case INT:
switch (missingLastAsString) {
case "null":
missingValue = null;
break;
default:
missingValue = Integer.parseInt(missingLastAsString);
break;
}
break;
default:
throw new AssertionError();
}
sortField[i] = new SortField(field, type, reverse);
if (missingValue != null) {
sortField[i].setMissingValue(missingValue);
}
}
Sort indexSort = sortField.length == 0 ? null : new Sort(sortField);
SimpleTextUtil.checkFooter(input);
SegmentInfo info = new SegmentInfo(directory, version, segmentName, docCount,
isCompoundFile, null, Collections.unmodifiableMap(diagnostics), id, Collections.unmodifiableMap(attributes));
isCompoundFile, null, Collections.unmodifiableMap(diagnostics),
id, Collections.unmodifiableMap(attributes), indexSort);
info.setFiles(files);
return info;
}
@ -223,6 +313,58 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
SimpleTextUtil.write(output, new BytesRef(si.getId()));
SimpleTextUtil.writeNewline(output);
Sort indexSort = si.getIndexSort();
SimpleTextUtil.write(output, SI_SORT);
final int numSortFields = indexSort == null ? 0 : indexSort.getSort().length;
SimpleTextUtil.write(output, Integer.toString(numSortFields), scratch);
SimpleTextUtil.writeNewline(output);
for (int i = 0; i < numSortFields; ++i) {
final SortField sortField = indexSort.getSort()[i];
SimpleTextUtil.write(output, SI_SORT_FIELD);
SimpleTextUtil.write(output, sortField.getField(), scratch);
SimpleTextUtil.writeNewline(output);
SimpleTextUtil.write(output, SI_SORT_TYPE);
final String sortType;
switch (sortField.getType()) {
case STRING:
sortType = "string";
break;
case LONG:
sortType = "long";
break;
case INT:
sortType = "int";
break;
default:
throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
}
SimpleTextUtil.write(output, sortType, scratch);
SimpleTextUtil.writeNewline(output);
SimpleTextUtil.write(output, SI_SORT_REVERSE);
SimpleTextUtil.write(output, Boolean.toString(sortField.getReverse()), scratch);
SimpleTextUtil.writeNewline(output);
SimpleTextUtil.write(output, SI_SORT_MISSING);
final Object missingValue = sortField.getMissingValue();
final String missing;
if (missingValue == null) {
missing = "null";
} else if (missingValue == SortField.STRING_FIRST) {
missing = "first";
} else if (missingValue == SortField.STRING_LAST) {
missing = "last";
} else if (missingValue instanceof Long) {
missing = Long.toString((Long) missingValue);
} else {
throw new IllegalStateException("Unexpected missing sort value: " + missingValue);
}
SimpleTextUtil.write(output, missing, scratch);
SimpleTextUtil.writeNewline(output);
}
SimpleTextUtil.writeChecksum(output, scratch);
}
}

View File

@ -143,7 +143,6 @@ public class SimpleTextStoredFieldsWriter extends StoredFieldsWriter {
} else {
write(TYPE_STRING);
newLine();
write(VALUE);
write(field.stringValue());
newLine();

View File

@ -16,7 +16,6 @@
*/
package org.apache.lucene.codecs;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
@ -25,12 +24,13 @@ import java.util.List;
import java.util.NoSuchElementException;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.DocIDMerger;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FilteredTermsEnum;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.MultiDocValues.OrdinalMap;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentWriteState; // javadocs
import org.apache.lucene.index.SortedDocValues;
@ -44,6 +44,8 @@ import org.apache.lucene.util.LongBitSet;
import org.apache.lucene.util.LongValues;
import org.apache.lucene.util.packed.PackedInts;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
/**
* Abstract API that consumes numeric, binary and
* sorted docvalues. Concrete implementations of this
@ -240,6 +242,32 @@ public abstract class DocValuesConsumer implements Closeable {
}
}
}
/** Tracks state of one numeric sub-reader that we are merging */
private static class NumericDocValuesSub extends DocIDMerger.Sub {
private final NumericDocValues values;
private final Bits docsWithField;
private int docID = -1;
private final int maxDoc;
public NumericDocValuesSub(MergeState.DocMap docMap, Bits liveDocs, NumericDocValues values, Bits docsWithField, int maxDoc) {
super(docMap, liveDocs);
this.values = values;
this.docsWithField = docsWithField;
this.maxDoc = maxDoc;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
return docID;
}
}
}
/**
* Merges the numeric docvalues from <code>toMerge</code>.
@ -248,20 +276,23 @@ public abstract class DocValuesConsumer implements Closeable {
* an Iterable that merges and filters deleted documents on the fly.
*/
public void mergeNumericField(final FieldInfo fieldInfo, final MergeState mergeState, final List<NumericDocValues> toMerge, final List<Bits> docsWithField) throws IOException {
addNumericField(fieldInfo,
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
// We must make a new DocIDMerger for each iterator:
List<NumericDocValuesSub> subs = new ArrayList<>();
assert mergeState.docMaps.length == toMerge.size();
for(int i=0;i<toMerge.size();i++) {
subs.add(new NumericDocValuesSub(mergeState.docMaps[i], mergeState.liveDocs[i], toMerge.get(i), docsWithField.get(i), mergeState.maxDocs[i]));
}
final DocIDMerger<NumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
return new Iterator<Number>() {
int readerUpto = -1;
int docIDUpto;
long nextValue;
boolean nextHasValue;
int currentMaxDoc;
NumericDocValues currentValues;
Bits currentLiveDocs;
Bits currentDocsWithField;
boolean nextIsSet;
@Override
@ -276,7 +307,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public Number next() {
if (!hasNext()) {
if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@ -286,35 +317,14 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == toMerge.size()) {
NumericDocValuesSub sub = docIDMerger.next();
if (sub == null) {
return false;
}
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < toMerge.size()) {
currentValues = toMerge.get(readerUpto);
currentDocsWithField = docsWithField.get(readerUpto);
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
nextIsSet = true;
nextValue = currentValues.get(docIDUpto);
if (nextValue == 0 && currentDocsWithField.get(docIDUpto) == false) {
nextHasValue = false;
} else {
nextHasValue = true;
}
docIDUpto++;
return true;
}
docIDUpto++;
nextIsSet = true;
nextValue = sub.values.get(sub.docID);
nextHasValue = nextValue != 0 || sub.docsWithField.get(sub.docID);
return true;
}
}
};
@ -322,6 +332,32 @@ public abstract class DocValuesConsumer implements Closeable {
});
}
/** Tracks state of one binary sub-reader that we are merging */
private static class BinaryDocValuesSub extends DocIDMerger.Sub {
private final BinaryDocValues values;
private final Bits docsWithField;
private int docID = -1;
private final int maxDoc;
public BinaryDocValuesSub(MergeState.DocMap docMap, Bits liveDocs, BinaryDocValues values, Bits docsWithField, int maxDoc) {
super(docMap, liveDocs);
this.values = values;
this.docsWithField = docsWithField;
this.maxDoc = maxDoc;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
return docID;
}
}
}
/**
* Merges the binary docvalues from <code>toMerge</code>.
* <p>
@ -329,20 +365,23 @@ public abstract class DocValuesConsumer implements Closeable {
* an Iterable that merges and filters deleted documents on the fly.
*/
public void mergeBinaryField(FieldInfo fieldInfo, final MergeState mergeState, final List<BinaryDocValues> toMerge, final List<Bits> docsWithField) throws IOException {
addBinaryField(fieldInfo,
new Iterable<BytesRef>() {
@Override
public Iterator<BytesRef> iterator() {
// We must make a new DocIDMerger for each iterator:
List<BinaryDocValuesSub> subs = new ArrayList<>();
assert mergeState.docMaps.length == toMerge.size();
for(int i=0;i<toMerge.size();i++) {
subs.add(new BinaryDocValuesSub(mergeState.docMaps[i], mergeState.liveDocs[i], toMerge.get(i), docsWithField.get(i), mergeState.maxDocs[i]));
}
final DocIDMerger<BinaryDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
return new Iterator<BytesRef>() {
int readerUpto = -1;
int docIDUpto;
BytesRef nextValue;
BytesRef nextPointer; // points to null if missing, or nextValue
int currentMaxDoc;
BinaryDocValues currentValues;
Bits currentLiveDocs;
Bits currentDocsWithField;
boolean nextIsSet;
@Override
@ -357,7 +396,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public BytesRef next() {
if (!hasNext()) {
if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@ -367,42 +406,49 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == toMerge.size()) {
return false;
BinaryDocValuesSub sub = docIDMerger.next();
if (sub == null) {
return false;
}
nextIsSet = true;
if (sub.docsWithField.get(sub.docID)) {
nextPointer = nextValue = sub.values.get(sub.docID);
} else {
nextPointer = null;
}
return true;
}
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < toMerge.size()) {
currentValues = toMerge.get(readerUpto);
currentDocsWithField = docsWithField.get(readerUpto);
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
nextIsSet = true;
if (currentDocsWithField.get(docIDUpto)) {
nextValue = currentValues.get(docIDUpto);
nextPointer = nextValue;
} else {
nextPointer = null;
}
docIDUpto++;
return true;
}
docIDUpto++;
}
}
};
}
});
}
/** Tracks state of one sorted numeric sub-reader that we are merging */
private static class SortedNumericDocValuesSub extends DocIDMerger.Sub {
private final SortedNumericDocValues values;
private int docID = -1;
private final int maxDoc;
public SortedNumericDocValuesSub(MergeState.DocMap docMap, Bits liveDocs, SortedNumericDocValues values, int maxDoc) {
super(docMap, liveDocs);
this.values = values;
this.maxDoc = maxDoc;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
values.setDocument(docID);
return docID;
}
}
}
/**
* Merges the sorted docvalues from <code>toMerge</code>.
* <p>
@ -410,21 +456,24 @@ public abstract class DocValuesConsumer implements Closeable {
* iterables that filter deleted documents.
*/
public void mergeSortedNumericField(FieldInfo fieldInfo, final MergeState mergeState, List<SortedNumericDocValues> toMerge) throws IOException {
final int numReaders = toMerge.size();
final SortedNumericDocValues dvs[] = toMerge.toArray(new SortedNumericDocValues[numReaders]);
// step 3: add field
addSortedNumericField(fieldInfo,
// doc -> value count
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
// We must make a new DocIDMerger for each iterator:
List<SortedNumericDocValuesSub> subs = new ArrayList<>();
assert mergeState.docMaps.length == toMerge.size();
for(int i=0;i<toMerge.size();i++) {
subs.add(new SortedNumericDocValuesSub(mergeState.docMaps[i], mergeState.liveDocs[i], toMerge.get(i), mergeState.maxDocs[i]));
}
final DocIDMerger<SortedNumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
return new Iterator<Number>() {
int readerUpto = -1;
int docIDUpto;
int nextValue;
int currentMaxDoc;
Bits currentLiveDocs;
boolean nextIsSet;
@Override
@ -439,7 +488,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public Number next() {
if (!hasNext()) {
if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@ -449,30 +498,13 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == numReaders) {
SortedNumericDocValuesSub sub = docIDMerger.next();
if (sub == null) {
return false;
}
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < numReaders) {
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
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++;
nextIsSet = true;
nextValue = sub.values.count();
return true;
}
}
};
@ -482,15 +514,21 @@ public abstract class DocValuesConsumer implements Closeable {
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
// We must make a new DocIDMerger for each iterator:
List<SortedNumericDocValuesSub> subs = new ArrayList<>();
assert mergeState.docMaps.length == toMerge.size();
for(int i=0;i<toMerge.size();i++) {
subs.add(new SortedNumericDocValuesSub(mergeState.docMaps[i], mergeState.liveDocs[i], toMerge.get(i), mergeState.maxDocs[i]));
}
final DocIDMerger<SortedNumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
return new Iterator<Number>() {
int readerUpto = -1;
int docIDUpto;
long nextValue;
int currentMaxDoc;
Bits currentLiveDocs;
boolean nextIsSet;
int valueUpto;
int valueLength;
SortedNumericDocValuesSub current;
@Override
public boolean hasNext() {
@ -504,7 +542,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public Number next() {
if (!hasNext()) {
if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@ -514,38 +552,21 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == numReaders) {
return false;
}
if (valueUpto < valueLength) {
nextValue = dvs[readerUpto].valueAt(valueUpto);
nextValue = current.values.valueAt(valueUpto);
valueUpto++;
nextIsSet = true;
return true;
}
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < numReaders) {
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
current = docIDMerger.next();
if (current == null) {
return false;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
assert docIDUpto < currentMaxDoc;
SortedNumericDocValues dv = dvs[readerUpto];
dv.setDocument(docIDUpto);
valueUpto = 0;
valueLength = dv.count();
docIDUpto++;
continue;
}
docIDUpto++;
valueUpto = 0;
valueLength = current.values.count();
continue;
}
}
};
@ -554,6 +575,32 @@ public abstract class DocValuesConsumer implements Closeable {
);
}
/** Tracks state of one sorted sub-reader that we are merging */
private static class SortedDocValuesSub extends DocIDMerger.Sub {
private final SortedDocValues values;
private int docID = -1;
private final int maxDoc;
private final LongValues map;
public SortedDocValuesSub(MergeState.DocMap docMap, Bits liveDocs, SortedDocValues values, int maxDoc, LongValues map) {
super(docMap, liveDocs);
this.values = values;
this.maxDoc = maxDoc;
this.map = map;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
return docID;
}
}
}
/**
* Merges the sorted docvalues from <code>toMerge</code>.
* <p>
@ -608,7 +655,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public BytesRef next() {
if (!hasNext()) {
if (hasNext() == false) {
throw new NoSuchElementException();
}
int segmentNumber = map.getFirstSegmentNumber(currentOrd);
@ -629,13 +676,17 @@ public abstract class DocValuesConsumer implements Closeable {
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
// We must make a new DocIDMerger for each iterator:
List<SortedDocValuesSub> subs = new ArrayList<>();
assert mergeState.docMaps.length == toMerge.size();
for(int i=0;i<toMerge.size();i++) {
subs.add(new SortedDocValuesSub(mergeState.docMaps[i], mergeState.liveDocs[i], toMerge.get(i), mergeState.maxDocs[i], map.getGlobalOrds(i)));
}
final DocIDMerger<SortedDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
return new Iterator<Number>() {
int readerUpto = -1;
int docIDUpto;
int nextValue;
int currentMaxDoc;
Bits currentLiveDocs;
LongValues currentMap;
boolean nextIsSet;
@Override
@ -650,7 +701,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public Number next() {
if (!hasNext()) {
if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@ -661,30 +712,15 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == numReaders) {
SortedDocValuesSub sub = docIDMerger.next();
if (sub == null) {
return false;
}
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < numReaders) {
currentMap = map.getGlobalOrds(readerUpto);
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
nextIsSet = true;
int segOrd = dvs[readerUpto].getOrd(docIDUpto);
nextValue = segOrd == -1 ? -1 : (int) currentMap.get(segOrd);
docIDUpto++;
return true;
}
docIDUpto++;
nextIsSet = true;
int segOrd = sub.values.getOrd(sub.docID);
nextValue = segOrd == -1 ? -1 : (int) sub.map.get(segOrd);
return true;
}
}
};
@ -693,6 +729,37 @@ public abstract class DocValuesConsumer implements Closeable {
);
}
/** Tracks state of one sorted set sub-reader that we are merging */
private static class SortedSetDocValuesSub extends DocIDMerger.Sub {
private final SortedSetDocValues values;
int docID = -1;
private final int maxDoc;
private final LongValues map;
public SortedSetDocValuesSub(MergeState.DocMap docMap, Bits liveDocs, SortedSetDocValues values, int maxDoc, LongValues map) {
super(docMap, liveDocs);
this.values = values;
this.maxDoc = maxDoc;
this.map = map;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
return docID;
}
}
@Override
public String toString() {
return "SortedSetDocValuesSub(docID=" + docID + " mappedDocID=" + mappedDocID + " values=" + values + ")";
}
}
/**
* Merges the sortedset docvalues from <code>toMerge</code>.
* <p>
@ -700,14 +767,12 @@ public abstract class DocValuesConsumer implements Closeable {
* 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 SortedSetDocValues dvs[] = toMerge.toArray(new SortedSetDocValues[toMerge.size()]);
final int numReaders = mergeState.maxDocs.length;
// step 1: iterate thru each sub and mark terms still in use
TermsEnum liveTerms[] = new TermsEnum[dvs.length];
TermsEnum liveTerms[] = new TermsEnum[toMerge.size()];
long[] weights = new long[liveTerms.length];
for (int sub = 0; sub < liveTerms.length; sub++) {
SortedSetDocValues dv = dvs[sub];
SortedSetDocValues dv = toMerge.get(sub);
Bits liveDocs = mergeState.liveDocs[sub];
int maxDoc = mergeState.maxDocs[sub];
if (liveDocs == null) {
@ -748,12 +813,12 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public BytesRef next() {
if (!hasNext()) {
if (hasNext() == false) {
throw new NoSuchElementException();
}
int segmentNumber = map.getFirstSegmentNumber(currentOrd);
long segmentOrd = map.getFirstSegmentOrd(currentOrd);
final BytesRef term = dvs[segmentNumber].lookupOrd(segmentOrd);
final BytesRef term = toMerge.get(segmentNumber).lookupOrd(segmentOrd);
currentOrd++;
return term;
}
@ -769,12 +834,18 @@ public abstract class DocValuesConsumer implements Closeable {
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
// We must make a new DocIDMerger for each iterator:
List<SortedSetDocValuesSub> subs = new ArrayList<>();
assert mergeState.docMaps.length == toMerge.size();
for(int i=0;i<toMerge.size();i++) {
subs.add(new SortedSetDocValuesSub(mergeState.docMaps[i], mergeState.liveDocs[i], toMerge.get(i), mergeState.maxDocs[i], map.getGlobalOrds(i)));
}
final DocIDMerger<SortedSetDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
return new Iterator<Number>() {
int readerUpto = -1;
int docIDUpto;
int nextValue;
int currentMaxDoc;
Bits currentLiveDocs;
boolean nextIsSet;
@Override
@ -789,7 +860,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public Number next() {
if (!hasNext()) {
if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@ -800,33 +871,18 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == numReaders) {
SortedSetDocValuesSub sub = docIDMerger.next();
if (sub == null) {
return false;
}
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < numReaders) {
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
sub.values.setDocument(sub.docID);
nextValue = 0;
while (sub.values.nextOrd() != SortedSetDocValues.NO_MORE_ORDS) {
nextValue++;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
nextIsSet = true;
SortedSetDocValues dv = dvs[readerUpto];
dv.setDocument(docIDUpto);
nextValue = 0;
while (dv.nextOrd() != SortedSetDocValues.NO_MORE_ORDS) {
nextValue++;
}
docIDUpto++;
return true;
}
docIDUpto++;
//System.out.println(" doc " + sub + " -> ord count = " + nextValue);
nextIsSet = true;
return true;
}
}
};
@ -836,13 +892,18 @@ public abstract class DocValuesConsumer implements Closeable {
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
// We must make a new DocIDMerger for each iterator:
List<SortedSetDocValuesSub> subs = new ArrayList<>();
assert mergeState.docMaps.length == toMerge.size();
for(int i=0;i<toMerge.size();i++) {
subs.add(new SortedSetDocValuesSub(mergeState.docMaps[i], mergeState.liveDocs[i], toMerge.get(i), mergeState.maxDocs[i], map.getGlobalOrds(i)));
}
final DocIDMerger<SortedSetDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
return new Iterator<Number>() {
int readerUpto = -1;
int docIDUpto;
long nextValue;
int currentMaxDoc;
Bits currentLiveDocs;
LongValues currentMap;
boolean nextIsSet;
long ords[] = new long[8];
int ordUpto;
@ -860,7 +921,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public Number next() {
if (!hasNext()) {
if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@ -871,10 +932,6 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == numReaders) {
return false;
}
if (ordUpto < ordLength) {
nextValue = ords[ordUpto];
ordUpto++;
@ -882,35 +939,22 @@ public abstract class DocValuesConsumer implements Closeable {
return true;
}
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < numReaders) {
currentMap = map.getGlobalOrds(readerUpto);
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
assert docIDUpto < currentMaxDoc;
SortedSetDocValues dv = dvs[readerUpto];
dv.setDocument(docIDUpto);
ordUpto = ordLength = 0;
long ord;
while ((ord = dv.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
if (ordLength == ords.length) {
ords = ArrayUtil.grow(ords, ordLength+1);
}
ords[ordLength] = currentMap.get(ord);
ordLength++;
}
docIDUpto++;
continue;
SortedSetDocValuesSub sub = docIDMerger.next();
if (sub == null) {
return false;
}
sub.values.setDocument(sub.docID);
docIDUpto++;
ordUpto = ordLength = 0;
long ord;
while ((ord = sub.values.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
if (ordLength == ords.length) {
ords = ArrayUtil.grow(ords, ordLength+1);
}
ords[ordLength] = sub.map.get(ord);
ordLength++;
}
continue;
}
}
};

View File

@ -76,7 +76,6 @@ public abstract class PointsWriter implements Closeable {
}
MergeState.DocMap docMap = mergeState.docMaps[i];
int docBase = mergeState.docBase[i];
pointsReader.intersect(fieldInfo.name,
new IntersectVisitor() {
@Override
@ -90,7 +89,7 @@ public abstract class PointsWriter implements Closeable {
int newDocID = docMap.get(docID);
if (newDocID != -1) {
// Not deleted:
mergedVisitor.visit(docBase + newDocID, packedValue);
mergedVisitor.visit(newDocID, packedValue);
}
}

View File

@ -20,10 +20,13 @@ import java.io.Closeable;
import java.io.IOException;
import java.io.Reader;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.DocIDMerger;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexableField;
@ -33,6 +36,8 @@ import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
/**
* Codec API for writing stored fields:
* <ol>
@ -73,6 +78,30 @@ public abstract class StoredFieldsWriter implements Closeable {
* check that this is the case to detect the JRE bug described
* in LUCENE-1282. */
public abstract void finish(FieldInfos fis, int numDocs) throws IOException;
private static class StoredFieldsMergeSub extends DocIDMerger.Sub {
private final StoredFieldsReader reader;
private final int maxDoc;
private final MergeVisitor visitor;
int docID = -1;
public StoredFieldsMergeSub(MergeVisitor visitor, MergeState.DocMap docMap, Bits liveDocs, StoredFieldsReader reader, int maxDoc) {
super(docMap, liveDocs);
this.maxDoc = maxDoc;
this.reader = reader;
this.visitor = visitor;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
return docID;
}
}
}
/** Merges in the stored fields from the readers in
* <code>mergeState</code>. The default implementation skips
@ -82,23 +111,26 @@ public abstract class StoredFieldsWriter implements Closeable {
* Implementations can override this method for more sophisticated
* merging (bulk-byte copying, etc). */
public int merge(MergeState mergeState) throws IOException {
int docCount = 0;
for (int i=0;i<mergeState.storedFieldsReaders.length;i++) {
List<StoredFieldsMergeSub> subs = new ArrayList<>();
for(int i=0;i<mergeState.storedFieldsReaders.length;i++) {
StoredFieldsReader storedFieldsReader = mergeState.storedFieldsReaders[i];
storedFieldsReader.checkIntegrity();
MergeVisitor visitor = new MergeVisitor(mergeState, i);
int maxDoc = mergeState.maxDocs[i];
Bits liveDocs = mergeState.liveDocs[i];
for (int docID=0;docID<maxDoc;docID++) {
if (liveDocs != null && !liveDocs.get(docID)) {
// skip deleted docs
continue;
}
startDocument();
storedFieldsReader.visitDocument(docID, visitor);
finishDocument();
docCount++;
subs.add(new StoredFieldsMergeSub(new MergeVisitor(mergeState, i), mergeState.docMaps[i], mergeState.liveDocs[i], storedFieldsReader, mergeState.maxDocs[i]));
}
final DocIDMerger<StoredFieldsMergeSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
int docCount = 0;
while (true) {
StoredFieldsMergeSub sub = docIDMerger.next();
if (sub == null) {
break;
}
assert sub.mappedDocID == docCount;
startDocument();
sub.reader.visitDocument(sub.docID, sub.visitor);
finishDocument();
docCount++;
}
finish(mergeState.mergeFieldInfos, docCount);
return docCount;

View File

@ -16,16 +16,18 @@
*/
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 org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.DocIDMerger;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSetIterator;
@ -34,6 +36,8 @@ import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
/**
* Codec API for writing term vectors:
* <ol>
@ -160,6 +164,28 @@ public abstract class TermVectorsWriter implements Closeable {
}
}
private static class TermVectorsMergeSub extends DocIDMerger.Sub {
private final TermVectorsReader reader;
private final int maxDoc;
int docID = -1;
public TermVectorsMergeSub(MergeState.DocMap docMap, Bits liveDocs, TermVectorsReader reader, int maxDoc) {
super(docMap, liveDocs);
this.maxDoc = maxDoc;
this.reader = reader;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
return docID;
}
}
}
/** Merges in the term vectors from the readers in
* <code>mergeState</code>. The default implementation skips
* over deleted documents, and uses {@link #startDocument(int)},
@ -170,32 +196,36 @@ public abstract class TermVectorsWriter implements Closeable {
* Implementations can override this method for more sophisticated
* merging (bulk-byte copying, etc). */
public int merge(MergeState mergeState) throws IOException {
List<TermVectorsMergeSub> subs = new ArrayList<>();
for(int i=0;i<mergeState.termVectorsReaders.length;i++) {
TermVectorsReader reader = mergeState.termVectorsReaders[i];
if (reader != null) {
reader.checkIntegrity();
}
// nocommit make sure the else case tested here
subs.add(new TermVectorsMergeSub(mergeState.docMaps[i], mergeState.liveDocs[i], reader, mergeState.maxDocs[i]));
}
final DocIDMerger<TermVectorsMergeSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
int docCount = 0;
int numReaders = mergeState.maxDocs.length;
for (int i = 0; i < numReaders; i++) {
int maxDoc = mergeState.maxDocs[i];
Bits liveDocs = mergeState.liveDocs[i];
TermVectorsReader termVectorsReader = mergeState.termVectorsReaders[i];
if (termVectorsReader != null) {
termVectorsReader.checkIntegrity();
while (true) {
TermVectorsMergeSub sub = docIDMerger.next();
if (sub == null) {
break;
}
for (int docID=0;docID<maxDoc;docID++) {
if (liveDocs != null && !liveDocs.get(docID)) {
// skip deleted docs
continue;
}
// NOTE: it's very important to first assign to vectors then pass it to
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
Fields vectors;
if (termVectorsReader == null) {
vectors = null;
} else {
vectors = termVectorsReader.get(docID);
}
addAllDocVectors(vectors, mergeState);
docCount++;
// NOTE: it's very important to first assign to vectors then pass it to
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
Fields vectors;
if (sub.reader == null) {
vectors = null;
} else {
vectors = sub.reader.get(sub.docID);
}
addAllDocVectors(vectors, mergeState);
docCount++;
}
finish(mergeState.mergeFieldInfos, docCount);
return docCount;

View File

@ -109,7 +109,7 @@ public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
attributes = Collections.unmodifiableMap(input.readStringStringMap());
}
si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes);
si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, null);
si.setFiles(files);
} catch (Throwable exception) {
priorE = exception;
@ -123,6 +123,10 @@ public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
@Override
public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene50SegmentInfoFormat.SI_EXTENSION);
// nocommit indexSort
if (si.getIndexSort() != null) {
throw new IllegalArgumentException("teach me to write indexSort");
}
try (IndexOutput output = dir.createOutput(fileName, ioContext)) {
// Only add the file once we've successfully created it, else IFD assert can trip:
@ -153,6 +157,7 @@ public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
}
output.writeSetOfStrings(files);
output.writeMapOfStrings(si.getAttributes());
CodecUtil.writeFooter(output);
}
}

View File

@ -16,6 +16,8 @@
*/
package org.apache.lucene.codecs.lucene60;
// nocommit if index time sorting is in use, don't try to bulk merge ... later we can make crazy bulk merger that looks for long runs from
// one sub?
import java.util.Objects;

View File

@ -171,7 +171,6 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
singleValuePerDoc)) {
List<BKDReader> bkdReaders = new ArrayList<>();
List<MergeState.DocMap> docMaps = new ArrayList<>();
List<Integer> docIDBases = new ArrayList<>();
for(int i=0;i<mergeState.pointsReaders.length;i++) {
PointsReader reader = mergeState.pointsReaders[i];
@ -191,7 +190,6 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
if (readerFieldInfo != null) {
BKDReader bkdReader = reader60.readers.get(readerFieldInfo.number);
if (bkdReader != null) {
docIDBases.add(mergeState.docBase[i]);
bkdReaders.add(bkdReader);
docMaps.add(mergeState.docMaps[i]);
}
@ -199,7 +197,7 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
}
}
long fp = writer.merge(dataOut, docMaps, bkdReaders, docIDBases);
long fp = writer.merge(dataOut, docMaps, bkdReaders);
if (fp != -1) {
indexFPs.put(fieldInfo.name, fp);
}

View File

@ -2498,6 +2498,9 @@ public final class CheckIndex implements Closeable {
}
}
// nocommit must check index is sorted, if it claims to be
// nocommit must check that all segments have the same sort, if any segment is sorted
/**
* Parse command line args into fields
* @param args The command line arguments

View File

@ -0,0 +1,173 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.io.IOException;
import java.util.List;
import org.apache.lucene.search.DocIdSetIterator; // javadocs
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.PriorityQueue;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
/** Utility class to help merging documents from sub-readers according to either simple
* concatenated (unsorted) order, or by a specified index-time sort, skipping
* deleted documents and remapping non-deleted documents. */
public class DocIDMerger<T extends DocIDMerger.Sub> {
private final List<T> subs;
// Used when indexSort != null:
private final PriorityQueue<T> queue;
private boolean first;
// Used when indexIsSorted
private T current;
private int nextIndex;
public static abstract class Sub {
public int mappedDocID;
final MergeState.DocMap docMap;
final Bits liveDocs;
// nocommit isn't liveDocs redundant? docMap returns -1 for us?
public Sub(MergeState.DocMap docMap, Bits liveDocs) {
this.docMap = docMap;
this.liveDocs = liveDocs;
}
/** Returns the next document ID from this sub reader, and {@link DocIdSetIterator#NO_MORE_DOCS} when done */
public abstract int nextDoc();
}
public DocIDMerger(List<T> subs, int maxCount, boolean indexIsSorted) {
this.subs = subs;
if (indexIsSorted) {
queue = new PriorityQueue<T>(maxCount) {
@Override
protected boolean lessThan(Sub a, Sub b) {
assert a.mappedDocID != b.mappedDocID;
return a.mappedDocID < b.mappedDocID;
}
};
} else {
// We simply concatentate
queue = null;
}
reset();
}
// nocommit it's awkward that we must pass in this boolean, when the subs should "know" this based on what docMap they have?
public DocIDMerger(List<T> subs, boolean indexIsSorted) {
this(subs, subs.size(), indexIsSorted);
}
/** Reuse API, currently only used by postings during merge */
public void reset() {
if (queue != null) {
assert queue.size() == 0;
for(T sub : subs) {
while (true) {
int docID = sub.nextDoc();
if (docID == NO_MORE_DOCS) {
// all docs in this sub were deleted; do not add it to the queue!
break;
} else if (sub.liveDocs != null && sub.liveDocs.get(docID) == false) {
// nocommit is it sub's job to skip deleted docs?
continue;
} else {
sub.mappedDocID = sub.docMap.get(docID);
assert sub.mappedDocID != -1;
queue.add(sub);
break;
}
}
}
first = true;
} else {
if (subs.size() > 0) {
current = subs.get(0);
nextIndex = 1;
} else {
current = null;
nextIndex = 0;
}
}
}
/** Returns null when done */
public T next() {
// Loop until we find a non-deleted document
if (queue != null) {
T top = queue.top();
if (top == null) {
// NOTE: it's annoying that caller is allowed to call us again even after we returned null before
return null;
}
if (first == false) {
while (true) {
int docID = top.nextDoc();
if (docID == NO_MORE_DOCS) {
queue.pop();
top = queue.top();
break;
} else if (top.liveDocs != null && top.liveDocs.get(docID) == false) {
continue;
} else {
top.mappedDocID = top.docMap.get(docID);
top = queue.updateTop();
break;
}
}
}
first = false;
return top;
} else {
while (true) {
if (current == null) {
// NOTE: it's annoying that caller is allowed to call us again even after we returned null before
return null;
}
int docID = current.nextDoc();
if (docID == NO_MORE_DOCS) {
if (nextIndex == subs.size()) {
current = null;
return null;
}
current = subs.get(nextIndex);
nextIndex++;
continue;
} else if (current.liveDocs != null && current.liveDocs.get(docID) == false) {
// Document is deleted
continue;
}
current.mappedDocID = current.docMap.get(docID);
return current;
}
}
}
}

View File

@ -178,7 +178,7 @@ class DocumentsWriterPerThread {
pendingUpdates.clear();
deleteSlice = deleteQueue.newSlice();
segmentInfo = new SegmentInfo(directoryOrig, Version.LATEST, segmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
segmentInfo = new SegmentInfo(directoryOrig, Version.LATEST, segmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
assert numDocsInRAM == 0;
if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segmentName + " delQueue=" + deleteQueue);

View File

@ -25,6 +25,7 @@ import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.search.Sort;
import org.apache.lucene.util.Bits;
/**
@ -101,6 +102,11 @@ public abstract class FilterCodecReader extends CodecReader {
return in.maxDoc();
}
@Override
public Sort getIndexSort() {
return in.getIndexSort();
}
@Override
public void addCoreClosedListener(CoreClosedListener listener) {
in.addCoreClosedListener(listener);

View File

@ -22,6 +22,7 @@ import java.util.Iterator;
import java.util.Objects;
import org.apache.lucene.search.QueryCache;
import org.apache.lucene.search.Sort;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@ -471,6 +472,12 @@ public abstract class FilterLeafReader extends LeafReader {
return in.getDocsWithField(field);
}
@Override
public Sort getIndexSort() {
ensureOpen();
return in.getIndexSort();
}
@Override
public void checkIntegrity() throws IOException {
ensureOpen();

View File

@ -16,6 +16,7 @@
*/
package org.apache.lucene.index;
// nocommit must add sorted indices to back compat tests
import java.io.Closeable;
import java.io.FileNotFoundException;
@ -32,8 +33,8 @@ import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
@ -49,6 +50,7 @@ import org.apache.lucene.index.FieldInfos.FieldNumbers;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Sort;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
@ -937,6 +939,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// NOTE: this is correct even for an NRT reader because we'll pull FieldInfos even for the un-committed segments:
globalFieldNumberMap = getFieldNumberMap();
validateIndexSort();
config.getFlushPolicy().init(config);
docWriter = new DocumentsWriter(this, config, directoryOrig, directory);
eventQueue = docWriter.eventQueue();
@ -1000,6 +1004,22 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
}
// nocommit can we improve this from just best effort?
/** Confirms that the incoming index sort (if any) matches the existing index sort (if any). This is unfortunately just best effort,
* because it could be the old index only has flushed segments. */
private void validateIndexSort() {
Sort indexSort = config.getIndexSort();
if (indexSort != null) {
for(SegmentCommitInfo info : segmentInfos) {
Sort segmentIndexSort = info.info.getIndexSort();
if (segmentIndexSort != null && indexSort.equals(segmentIndexSort) == false) {
throw new IllegalArgumentException("cannot change previous indexSort=" + segmentIndexSort + " (from segment=" + info + ") to new indexSort=" + indexSort);
}
}
}
}
// reads latest field infos for the commit
// this is used on IW init and addIndexes(Dir) to create/update the global field map.
// TODO: fix tests abusing this method!
@ -2474,6 +2494,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* @throws IllegalArgumentException if addIndexes would cause
* the index to exceed {@link #MAX_DOCS}
*/
// nocommit doesn't support index sorting? or sorts must be the same?
public void addIndexes(Directory... dirs) throws IOException {
ensureOpen();
@ -2603,6 +2624,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* @throws IllegalArgumentException
* if addIndexes would cause the index to exceed {@link #MAX_DOCS}
*/
// nocommit make sure if you add "sorted by X" to "sorted by Y" index, we catch it
public void addIndexes(CodecReader... readers) throws IOException {
ensureOpen();
@ -2630,7 +2652,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(directory);
SegmentInfo info = new SegmentInfo(directoryOrig, Version.LATEST, mergedName, -1,
false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), config.getIndexSort());
SegmentMerger merger = new SegmentMerger(Arrays.asList(readers), info, infoStream, trackingDir,
globalFieldNumberMap,
@ -2715,7 +2737,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// Same SI as before but we change directory and name
SegmentInfo newInfo = new SegmentInfo(directoryOrig, info.info.getVersion(), segName, info.info.maxDoc(),
info.info.getUseCompoundFile(), info.info.getCodec(),
info.info.getDiagnostics(), info.info.getId(), info.info.getAttributes());
info.info.getDiagnostics(), info.info.getId(), info.info.getAttributes(), info.info.getIndexSort());
SegmentCommitInfo newInfoPerCommit = new SegmentCommitInfo(newInfo, info.getDelCount(), info.getDelGen(),
info.getFieldInfosGen(), info.getDocValuesGen());
@ -3243,16 +3265,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
private static class MergedDeletesAndUpdates {
ReadersAndUpdates mergedDeletesAndUpdates = null;
MergePolicy.DocMap docMap = null;
boolean initializedWritableLiveDocs = false;
MergedDeletesAndUpdates() {}
final void init(ReaderPool readerPool, MergePolicy.OneMerge merge, MergeState mergeState, boolean initWritableLiveDocs) throws IOException {
final void init(ReaderPool readerPool, MergePolicy.OneMerge merge, boolean initWritableLiveDocs) throws IOException {
if (mergedDeletesAndUpdates == null) {
mergedDeletesAndUpdates = readerPool.get(merge.info, true);
docMap = merge.getDocMap(mergeState);
assert docMap.isConsistent(merge.info.info.maxDoc());
}
if (initWritableLiveDocs && !initializedWritableLiveDocs) {
mergedDeletesAndUpdates.initWritableLiveDocs();
@ -3262,18 +3281,18 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
private void maybeApplyMergedDVUpdates(MergePolicy.OneMerge merge, MergeState mergeState, int docUpto,
private void maybeApplyMergedDVUpdates(MergePolicy.OneMerge merge, MergeState mergeState,
MergedDeletesAndUpdates holder, String[] mergingFields, DocValuesFieldUpdates[] dvFieldUpdates,
DocValuesFieldUpdates.Iterator[] updatesIters, int curDoc) throws IOException {
DocValuesFieldUpdates.Iterator[] updatesIters, int segment, int curDoc) throws IOException {
int newDoc = -1;
for (int idx = 0; idx < mergingFields.length; idx++) {
DocValuesFieldUpdates.Iterator updatesIter = updatesIters[idx];
if (updatesIter.doc() == curDoc) { // document has an update
if (holder.mergedDeletesAndUpdates == null) {
holder.init(readerPool, merge, mergeState, false);
holder.init(readerPool, merge, false);
}
if (newDoc == -1) { // map once per all field updates, but only if there are any updates
newDoc = holder.docMap.map(docUpto);
newDoc = mergeState.docMaps[segment].get(curDoc);
}
DocValuesFieldUpdates dvUpdates = dvFieldUpdates[idx];
dvUpdates.add(newDoc, updatesIter.value());
@ -3306,13 +3325,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// Carefully merge deletes that occurred after we
// started merging:
int docUpto = 0;
long minGen = Long.MAX_VALUE;
// Lazy init (only when we find a delete to carry over):
final MergedDeletesAndUpdates holder = new MergedDeletesAndUpdates();
final DocValuesFieldUpdates.Container mergedDVUpdates = new DocValuesFieldUpdates.Container();
assert sourceSegments.size() == mergeState.docMaps.length;
for (int i = 0; i < sourceSegments.size(); i++) {
SegmentCommitInfo info = sourceSegments.get(i);
minGen = Math.min(info.getBufferedDeletesGen(), minGen);
@ -3375,21 +3394,20 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// since we started the merge, so we
// must merge them:
for (int j = 0; j < maxDoc; j++) {
if (!prevLiveDocs.get(j)) {
assert !currentLiveDocs.get(j);
} else {
if (!currentLiveDocs.get(j)) {
if (holder.mergedDeletesAndUpdates == null || !holder.initializedWritableLiveDocs) {
holder.init(readerPool, merge, mergeState, true);
}
holder.mergedDeletesAndUpdates.delete(holder.docMap.map(docUpto));
if (mergingFields != null) { // advance all iters beyond the deleted document
skipDeletedDoc(updatesIters, j);
}
} else if (mergingFields != null) {
maybeApplyMergedDVUpdates(merge, mergeState, docUpto, holder, mergingFields, dvFieldUpdates, updatesIters, j);
if (prevLiveDocs.get(j) == false) {
// if the document was deleted before, it better still be deleted!
assert currentLiveDocs.get(j) == false;
} else if (currentLiveDocs.get(j) == false) {
// the document was deleted while we were merging:
if (holder.mergedDeletesAndUpdates == null || holder.initializedWritableLiveDocs == false) {
holder.init(readerPool, merge, true);
}
docUpto++;
holder.mergedDeletesAndUpdates.delete(mergeState.docMaps[i].get(mergeState.leafDocMaps[i].get(j)));
if (mergingFields != null) { // advance all iters beyond the deleted document
skipDeletedDoc(updatesIters, j);
}
} else if (mergingFields != null) {
maybeApplyMergedDVUpdates(merge, mergeState, holder, mergingFields, dvFieldUpdates, updatesIters, i, j);
}
}
} else if (mergingFields != null) {
@ -3397,50 +3415,38 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
for (int j = 0; j < maxDoc; j++) {
if (prevLiveDocs.get(j)) {
// document isn't deleted, check if any of the fields have an update to it
maybeApplyMergedDVUpdates(merge, mergeState, docUpto, holder, mergingFields, dvFieldUpdates, updatesIters, j);
// advance docUpto for every non-deleted document
docUpto++;
maybeApplyMergedDVUpdates(merge, mergeState, holder, mergingFields, dvFieldUpdates, updatesIters, i, j);
} else {
// advance all iters beyond the deleted document
skipDeletedDoc(updatesIters, j);
}
}
} else {
docUpto += info.info.maxDoc() - info.getDelCount() - rld.getPendingDeleteCount();
}
} else if (currentLiveDocs != null) {
assert currentLiveDocs.length() == maxDoc;
// This segment had no deletes before but now it
// does:
for (int j = 0; j < maxDoc; j++) {
if (!currentLiveDocs.get(j)) {
if (currentLiveDocs.get(j) == false) {
if (holder.mergedDeletesAndUpdates == null || !holder.initializedWritableLiveDocs) {
holder.init(readerPool, merge, mergeState, true);
holder.init(readerPool, merge, true);
}
holder.mergedDeletesAndUpdates.delete(holder.docMap.map(docUpto));
holder.mergedDeletesAndUpdates.delete(mergeState.docMaps[i].get(mergeState.leafDocMaps[i].get(j)));
if (mergingFields != null) { // advance all iters beyond the deleted document
skipDeletedDoc(updatesIters, j);
}
} else if (mergingFields != null) {
maybeApplyMergedDVUpdates(merge, mergeState, docUpto, holder, mergingFields, dvFieldUpdates, updatesIters, j);
maybeApplyMergedDVUpdates(merge, mergeState, holder, mergingFields, dvFieldUpdates, updatesIters, i, j);
}
docUpto++;
}
} else if (mergingFields != null) {
// no deletions before or after, but there were updates
for (int j = 0; j < maxDoc; j++) {
maybeApplyMergedDVUpdates(merge, mergeState, docUpto, holder, mergingFields, dvFieldUpdates, updatesIters, j);
// advance docUpto for every non-deleted document
docUpto++;
maybeApplyMergedDVUpdates(merge, mergeState, holder, mergingFields, dvFieldUpdates, updatesIters, i, j);
}
} else {
// No deletes or updates before or after
docUpto += info.info.maxDoc();
}
}
assert docUpto == merge.info.info.maxDoc();
if (mergedDVUpdates.any()) {
// System.out.println("[" + Thread.currentThread().getName() + "] IW.commitMergedDeletes: mergedDeletes.info=" + mergedDeletes.info + ", mergedFieldUpdates=" + mergedFieldUpdates);
boolean success = false;
@ -3881,7 +3887,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// ConcurrentMergePolicy we keep deterministic segment
// names.
final String mergeSegmentName = newSegmentName();
SegmentInfo si = new SegmentInfo(directoryOrig, Version.LATEST, mergeSegmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
SegmentInfo si = new SegmentInfo(directoryOrig, Version.LATEST, mergeSegmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), config.getIndexSort());
Map<String,String> details = new HashMap<>();
details.put("mergeMaxNumSegments", "" + merge.maxNumSegments);
details.put("mergeFactor", Integer.toString(merge.segments.size()));
@ -4082,10 +4088,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
// System.out.println("[" + Thread.currentThread().getName() + "] IW.mergeMiddle: merging " + merge.getMergeReaders());
// we pass merge.getMergeReaders() instead of merge.readers to allow the
// OneMerge to return a view over the actual segments to merge
final SegmentMerger merger = new SegmentMerger(merge.getMergeReaders(),
// Let the merge wrap readers
List<CodecReader> mergeReaders = new ArrayList<>();
for (SegmentReader reader : merge.readers) {
mergeReaders.add(merge.wrapForMerge(reader));
}
final SegmentMerger merger = new SegmentMerger(mergeReaders,
merge.info.info, infoStream, dirWrapper,
globalFieldNumberMap,
context);

View File

@ -18,16 +18,19 @@ package org.apache.lucene.index;
import java.io.PrintStream;
import java.util.EnumSet;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.PrintStreamInfoStream;
import org.apache.lucene.util.SetOnce;
import org.apache.lucene.util.SetOnce.AlreadySetException;
import org.apache.lucene.util.SetOnce;
/**
* Holds all the configuration that is used to create an {@link IndexWriter}.
@ -439,6 +442,27 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
return this;
}
/** We only allow sorting on these types */
private static final EnumSet<SortField.Type> ALLOWED_INDEX_SORT_TYPES = EnumSet.of(SortField.Type.STRING,
SortField.Type.INT,
SortField.Type.FLOAT,
SortField.Type.LONG,
SortField.Type.DOUBLE,
SortField.Type.BYTES);
/**
* Set the {@link Sort} order to use when merging segments. Note that newly flushed segments will remain unsorted.
*/
public IndexWriterConfig setIndexSort(Sort sort) {
for(SortField sortField : sort.getSort()) {
if (ALLOWED_INDEX_SORT_TYPES.contains(sortField.getType()) == false) {
throw new IllegalArgumentException("invalid SortField type: must be one of " + ALLOWED_INDEX_SORT_TYPES + " but got: " + sortField);
}
}
this.indexSort = sort;
return this;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder(super.toString());

View File

@ -20,6 +20,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import org.apache.lucene.index.IndexReader.ReaderClosedListener;
import org.apache.lucene.search.Sort;
import org.apache.lucene.util.Bits;
/** {@code LeafReader} is an abstract class, providing an interface for accessing an
@ -312,4 +313,7 @@ public abstract class LeafReader extends IndexReader {
* @lucene.internal
*/
public abstract void checkIntegrity() throws IOException;
/** Returns null if this leaf is unsorted, or the {@link Sort} that it was sorted by */
public abstract Sort getIndexSort();
}

View File

@ -23,6 +23,7 @@ import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.util.InfoStream;
@ -94,6 +95,9 @@ public class LiveIndexWriterConfig {
/** True if calls to {@link IndexWriter#close()} should first do a commit. */
protected boolean commitOnClose = IndexWriterConfig.DEFAULT_COMMIT_ON_CLOSE;
/** The sort order to use to write merged segments. */
protected Sort indexSort = null;
// used by IndexWriterConfig
LiveIndexWriterConfig(Analyzer analyzer) {
this.analyzer = analyzer;
@ -445,6 +449,14 @@ public class LiveIndexWriterConfig {
return commitOnClose;
}
/**
* Set the index-time {@link Sort} order. Merged segments will be written
* in this order.
*/
public Sort getIndexSort() {
return indexSort;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
@ -467,6 +479,7 @@ public class LiveIndexWriterConfig {
sb.append("perThreadHardLimitMB=").append(getRAMPerThreadHardLimitMB()).append("\n");
sb.append("useCompoundFile=").append(getUseCompoundFile()).append("\n");
sb.append("commitOnClose=").append(getCommitOnClose()).append("\n");
sb.append("indexSort=").append(getIndexSort()).append("\n");
return sb.toString();
}
}

View File

@ -18,8 +18,11 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.index.MultiPostingsEnum.EnumWithSlice;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
/**
@ -30,52 +33,63 @@ import org.apache.lucene.util.BytesRef;
*/
final class MappingMultiPostingsEnum extends PostingsEnum {
private MultiPostingsEnum.EnumWithSlice[] subs;
int numSubs;
int upto;
MergeState.DocMap currentMap;
PostingsEnum current;
int currentBase;
int doc = -1;
private MergeState mergeState;
MultiPostingsEnum multiDocsAndPositionsEnum;
final String field;
final DocIDMerger<MappingPostingsSub> docIDMerger;
private MappingPostingsSub current;
private final MappingPostingsSub[] allSubs;
private final List<MappingPostingsSub> subs = new ArrayList<>();
private static class MappingPostingsSub extends DocIDMerger.Sub {
public PostingsEnum postings;
public MappingPostingsSub(MergeState.DocMap docMap, Bits liveDocs) {
super(docMap, liveDocs);
}
@Override
public int nextDoc() {
try {
return postings.nextDoc();
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
}
/** Sole constructor. */
public MappingMultiPostingsEnum(String field, MergeState mergeState) {
public MappingMultiPostingsEnum(String field, MergeState mergeState) throws IOException {
this.field = field;
this.mergeState = mergeState;
allSubs = new MappingPostingsSub[mergeState.fieldsProducers.length];
for(int i=0;i<allSubs.length;i++) {
// nocommit delDocMaps?
allSubs[i] = new MappingPostingsSub(mergeState.docMaps[i], mergeState.liveDocs[i]);
}
this.docIDMerger = new DocIDMerger<MappingPostingsSub>(subs, allSubs.length, mergeState.segmentInfo.getIndexSort() != null);
}
MappingMultiPostingsEnum reset(MultiPostingsEnum postingsEnum) {
this.numSubs = postingsEnum.getNumSubs();
this.subs = postingsEnum.getSubs();
upto = -1;
doc = -1;
current = null;
MappingMultiPostingsEnum reset(MultiPostingsEnum postingsEnum) throws IOException {
this.multiDocsAndPositionsEnum = postingsEnum;
MultiPostingsEnum.EnumWithSlice[] subsArray = postingsEnum.getSubs();
int count = postingsEnum.getNumSubs();
subs.clear();
for(int i=0;i<count;i++) {
MappingPostingsSub sub = allSubs[subsArray[i].slice.readerIndex];
sub.postings = subsArray[i].postingsEnum;
subs.add(sub);
}
docIDMerger.reset();
return this;
}
/** How many sub-readers we are merging.
* @see #getSubs */
public int getNumSubs() {
return numSubs;
}
/** Returns sub-readers we are merging. */
public EnumWithSlice[] getSubs() {
return subs;
}
@Override
public int freq() throws IOException {
return current.freq();
return current.postings.freq();
}
@Override
public int docID() {
return doc;
return current.mappedDocID;
}
@Override
@ -85,66 +99,47 @@ final class MappingMultiPostingsEnum extends PostingsEnum {
@Override
public int nextDoc() throws IOException {
while(true) {
if (current == null) {
if (upto == numSubs-1) {
return this.doc = NO_MORE_DOCS;
} else {
upto++;
final int reader = subs[upto].slice.readerIndex;
current = subs[upto].postingsEnum;
currentBase = mergeState.docBase[reader];
currentMap = mergeState.docMaps[reader];
}
}
int doc = current.nextDoc();
if (doc != NO_MORE_DOCS) {
// compact deletions
doc = currentMap.get(doc);
if (doc == -1) {
continue;
}
return this.doc = currentBase + doc;
} else {
current = null;
}
current = docIDMerger.next();
if (current == null) {
return NO_MORE_DOCS;
} else {
return current.mappedDocID;
}
}
@Override
public int nextPosition() throws IOException {
int pos = current.nextPosition();
int pos = current.postings.nextPosition();
if (pos < 0) {
throw new CorruptIndexException("position=" + pos + " is negative, field=\"" + field + " doc=" + doc,
mergeState.fieldsProducers[upto].toString());
throw new CorruptIndexException("position=" + pos + " is negative, field=\"" + field + " doc=" + current.mappedDocID,
current.postings.toString());
} else if (pos > IndexWriter.MAX_POSITION) {
throw new CorruptIndexException("position=" + pos + " is too large (> IndexWriter.MAX_POSITION=" + IndexWriter.MAX_POSITION + "), field=\"" + field + "\" doc=" + doc,
mergeState.fieldsProducers[upto].toString());
throw new CorruptIndexException("position=" + pos + " is too large (> IndexWriter.MAX_POSITION=" + IndexWriter.MAX_POSITION + "), field=\"" + field + "\" doc=" + current.mappedDocID,
current.postings.toString());
}
return pos;
}
@Override
public int startOffset() throws IOException {
return current.startOffset();
return current.postings.startOffset();
}
@Override
public int endOffset() throws IOException {
return current.endOffset();
return current.postings.endOffset();
}
@Override
public BytesRef getPayload() throws IOException {
return current.getPayload();
return current.postings.getPayload();
}
@Override
public long cost() {
long cost = 0;
for (EnumWithSlice enumWithSlice : subs) {
cost += enumWithSlice.postingsEnum.cost();
for (MappingPostingsSub sub : subs) {
cost += sub.postings.cost();
}
return cost;
}

View File

@ -58,31 +58,6 @@ import org.apache.lucene.util.FixedBitSet;
*/
public abstract class MergePolicy {
/** A map of doc IDs. */
public static abstract class DocMap {
/** Sole constructor, typically invoked from sub-classes constructors. */
protected DocMap() {}
/** Return the new doc ID according to its old value. */
public abstract int map(int old);
/** Useful from an assert. */
boolean isConsistent(int maxDoc) {
final FixedBitSet targets = new FixedBitSet(maxDoc);
for (int i = 0; i < maxDoc; ++i) {
final int target = map(i);
if (target < 0 || target >= maxDoc) {
assert false : "out of range: " + target + " not in [0-" + maxDoc + "[";
return false;
} else if (targets.get(target)) {
assert false : target + " is already taken (" + i + ")";
return false;
}
}
return true;
}
}
/** OneMerge provides the information necessary to perform
* an individual primitive merge operation, resulting in
* a single new segment. The merge spec includes the
@ -140,25 +115,11 @@ public abstract class MergePolicy {
public void mergeFinished() throws IOException {
}
/** Expert: Get the list of readers to merge. Note that this list does not
* necessarily match the list of segments to merge and should only be used
* to feed SegmentMerger to initialize a merge. When a {@link OneMerge}
* reorders doc IDs, it must override {@link #getDocMap} too so that
* deletes that happened during the merge can be applied to the newly
* merged segment. */
public List<CodecReader> getMergeReaders() throws IOException {
if (readers == null) {
throw new IllegalStateException("IndexWriter has not initialized readers from the segment infos yet");
}
final List<CodecReader> readers = new ArrayList<>(this.readers.size());
for (SegmentReader reader : this.readers) {
if (reader.numDocs() > 0) {
readers.add(reader);
}
}
return Collections.unmodifiableList(readers);
/** Wrap the reader in order to add/remove information to the merged segment. */
public CodecReader wrapForMerge(CodecReader reader) throws IOException {
return reader;
}
/**
* Expert: Sets the {@link SegmentCommitInfo} of the merged segment.
* Allows sub-classes to e.g. set diagnostics properties.
@ -175,20 +136,6 @@ public abstract class MergePolicy {
return info;
}
/** Expert: If {@link #getMergeReaders()} reorders document IDs, this method
* must be overridden to return a mapping from the <i>natural</i> doc ID
* (the doc ID that would result from a natural merge) to the actual doc
* ID. This mapping is used to apply deletions that happened during the
* merge to the new segment. */
public DocMap getDocMap(MergeState mergeState) {
return new DocMap() {
@Override
public int map(int docID) {
return docID;
}
};
}
/** Record that an exception occurred while executing
* this merge */
synchronized void setException(Throwable error) {

View File

@ -1,3 +1,5 @@
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
@ -14,7 +16,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.io.IOException;
@ -23,9 +24,10 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.search.Sort;
import org.apache.lucene.util.Bits;
/** this is a hack to make SortingMP fast! */
/** This is a hack to make index sorting fast, with a {@link LeafReader} that always returns merge instances when you ask for the codec readers. */
class MergeReaderWrapper extends LeafReader {
final SegmentReader in;
final FieldsProducer fields;
@ -256,4 +258,9 @@ class MergeReaderWrapper extends LeafReader {
public String toString() {
return "MergeReaderWrapper(" + in + ")";
}
@Override
public Sort getIndexSort() {
return in.getIndexSort();
}
}

View File

@ -18,6 +18,8 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.codecs.DocValuesProducer;
@ -26,6 +28,7 @@ import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.search.Sort;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.packed.PackedInts;
@ -36,6 +39,13 @@ import org.apache.lucene.util.packed.PackedLongValues;
* @lucene.experimental */
public class MergeState {
/** Maps document IDs from old segments to document IDs in the new segment */
// nocommit in the unsorted case, this should map correctly, e.g. apply per segment docBase
public final DocMap[] docMaps;
// nocommit can we somehow not need to expose this? should IW's reader pool always sort on load...?
public final DocMap[] leafDocMaps;
/** {@link SegmentInfo} of the newly merged segment. */
public final SegmentInfo segmentInfo;
@ -60,18 +70,12 @@ public class MergeState {
/** Live docs for each reader */
public final Bits[] liveDocs;
/** Maps docIDs around deletions. */
public final DocMap[] docMaps;
/** Postings to merge */
public final FieldsProducer[] fieldsProducers;
/** Point readers to merge */
public final PointsReader[] pointsReaders;
/** New docID base per reader. */
public final int[] docBase;
/** Max docs per reader */
public final int[] maxDocs;
@ -79,11 +83,13 @@ public class MergeState {
public final InfoStream infoStream;
/** Sole constructor. */
MergeState(List<CodecReader> readers, SegmentInfo segmentInfo, InfoStream infoStream) throws IOException {
MergeState(List<CodecReader> originalReaders, SegmentInfo segmentInfo, InfoStream infoStream) throws IOException {
final Sort indexSort = segmentInfo.getIndexSort();
int numReaders = originalReaders.size();
leafDocMaps = new DocMap[numReaders];
List<CodecReader> readers = maybeSortReaders(originalReaders, segmentInfo);
int numReaders = readers.size();
docMaps = new DocMap[numReaders];
docBase = new int[numReaders];
maxDocs = new int[numReaders];
fieldsProducers = new FieldsProducer[numReaders];
normsProducers = new NormsProducer[numReaders];
@ -94,6 +100,7 @@ public class MergeState {
fieldInfos = new FieldInfos[numReaders];
liveDocs = new Bits[numReaders];
int numDocs = 0;
for(int i=0;i<numReaders;i++) {
final CodecReader reader = readers.get(i);
@ -126,126 +133,137 @@ public class MergeState {
if (pointsReaders[i] != null) {
pointsReaders[i] = pointsReaders[i].getMergeInstance();
}
numDocs += reader.numDocs();
}
segmentInfo.setMaxDoc(numDocs);
this.segmentInfo = segmentInfo;
this.infoStream = infoStream;
setDocMaps(readers);
this.docMaps = buildDocMaps(readers, indexSort);
}
// NOTE: removes any "all deleted" readers from mergeState.readers
private void setDocMaps(List<CodecReader> readers) throws IOException {
final int numReaders = maxDocs.length;
private DocMap[] buildDocMaps(List<CodecReader> readers, Sort indexSort) throws IOException {
// Remap docIDs
int docBase = 0;
for(int i=0;i<numReaders;i++) {
final CodecReader reader = readers.get(i);
this.docBase[i] = docBase;
final DocMap docMap = DocMap.build(reader);
docMaps[i] = docMap;
docBase += docMap.numDocs();
}
int numReaders = readers.size();
segmentInfo.setMaxDoc(docBase);
}
if (indexSort == null) {
// no index sort ... we only must map around deletions, and rebase to the merged segment's docID space
/**
* Remaps docids around deletes during merge
*/
public static abstract class DocMap {
int totalDocs = 0;
DocMap[] docMaps = new DocMap[numReaders];
DocMap() {}
// Remap docIDs around deletions:
for (int i = 0; i < numReaders; i++) {
LeafReader reader = readers.get(i);
Bits liveDocs = reader.getLiveDocs();
/** Returns the mapped docID corresponding to the provided one. */
public abstract int get(int docID);
/** Returns the total number of documents, ignoring
* deletions. */
public abstract int maxDoc();
/** Returns the number of not-deleted documents. */
public final int numDocs() {
return maxDoc() - numDeletedDocs();
}
/** Returns the number of deleted documents. */
public abstract int numDeletedDocs();
/** Returns true if there are any deletions. */
public boolean hasDeletions() {
return numDeletedDocs() > 0;
}
/** Creates a {@link DocMap} instance appropriate for
* this reader. */
public static DocMap build(CodecReader reader) {
final int maxDoc = reader.maxDoc();
if (!reader.hasDeletions()) {
return new NoDelDocMap(maxDoc);
}
final Bits liveDocs = reader.getLiveDocs();
return build(maxDoc, liveDocs);
}
static DocMap build(final int maxDoc, final Bits liveDocs) {
assert liveDocs != null;
final PackedLongValues.Builder docMapBuilder = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
int del = 0;
for (int i = 0; i < maxDoc; ++i) {
docMapBuilder.add(i - del);
if (!liveDocs.get(i)) {
++del;
final PackedLongValues delDocMap;
if (liveDocs != null) {
delDocMap = removeDeletes(reader.maxDoc(), liveDocs);
} else {
delDocMap = null;
}
}
final PackedLongValues docMap = docMapBuilder.build();
final int numDeletedDocs = del;
assert docMap.size() == maxDoc;
return new DocMap() {
@Override
public int get(int docID) {
if (!liveDocs.get(docID)) {
return -1;
final int docBase = totalDocs;
docMaps[i] = new DocMap() {
@Override
public int get(int docID) {
if (liveDocs == null) {
return docBase + docID;
} else if (liveDocs.get(docID)) {
return docBase + (int) delDocMap.get(docID);
} else {
return -1;
}
}
return (int) docMap.get(docID);
}
};
totalDocs += reader.numDocs();
}
@Override
public int maxDoc() {
return maxDoc;
}
return docMaps;
@Override
public int numDeletedDocs() {
return numDeletedDocs;
}
};
} else {
// do a merge sort of the incoming leaves:
return MultiSorter.sort(indexSort, readers);
}
}
private static final class NoDelDocMap extends DocMap {
private List<CodecReader> maybeSortReaders(List<CodecReader> originalReaders, SegmentInfo segmentInfo) throws IOException {
private final int maxDoc;
NoDelDocMap(int maxDoc) {
this.maxDoc = maxDoc;
// Default to identity:
for(int i=0;i<originalReaders.size();i++) {
leafDocMaps[i] = new DocMap() {
@Override
public int get(int docID) {
return docID;
}
};
}
@Override
public int get(int docID) {
return docID;
Sort indexSort = segmentInfo.getIndexSort();
if (indexSort == null) {
return originalReaders;
}
@Override
public int maxDoc() {
return maxDoc;
// If an incoming reader is not sorted, because it was flushed by IW, we sort it here:
final Sorter sorter = new Sorter(indexSort);
List<CodecReader> readers = new ArrayList<>(originalReaders.size());
//System.out.println("MergeState.maybeSortReaders indexSort=" + indexSort);
for (CodecReader leaf : originalReaders) {
if (leaf instanceof SegmentReader) {
SegmentReader segmentReader = (SegmentReader) leaf;
Sort segmentSort = segmentReader.getSegmentInfo().info.getIndexSort();
//System.out.println(" leaf=" + leaf + " sort=" + segmentSort);
if (segmentSort == null) {
// TODO: fix IW to also sort when flushing? It's somewhat tricky because of stored fields and term vectors, which write "live"
// to the files on each indexed document:
// This segment was written by flush, so documents are not yet sorted, so we sort them now:
Sorter.DocMap sortDocMap = sorter.sort(leaf);
if (sortDocMap != null) {
//System.out.println(" sort!");
// nocommit what about MergedReaderWrapper in here?
leaf = SlowCodecReaderWrapper.wrap(SortingLeafReader.wrap(leaf, sortDocMap));
leafDocMaps[readers.size()] = new DocMap() {
@Override
public int get(int docID) {
return sortDocMap.oldToNew(docID);
}
};
}
} else if (segmentSort.equals(indexSort) == false) {
throw new IllegalArgumentException("index sort mismatch: merged segment has sort=" + indexSort + " but to-be-merged segment has sort=" + segmentSort);
}
} else {
throw new IllegalArgumentException("cannot sort index with foreign readers; leaf=" + leaf);
}
readers.add(leaf);
}
@Override
public int numDeletedDocs() {
return 0;
return readers;
}
/** A map of doc IDs. */
public static abstract class DocMap {
/** Return the mapped docID or -1 if the given doc is not mapped. */
public abstract int get(int docID);
}
static PackedLongValues removeDeletes(final int maxDoc, final Bits liveDocs) {
final PackedLongValues.Builder docMapBuilder = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
int del = 0;
for (int i = 0; i < maxDoc; ++i) {
docMapBuilder.add(i - del);
if (liveDocs.get(i) == false) {
++del;
}
}
return docMapBuilder.build();
}
}

View File

@ -51,6 +51,8 @@ public final class MultiFields extends Fields {
private final ReaderSlice[] subSlices;
private final Map<String,Terms> terms = new ConcurrentHashMap<>();
// nocommit should we somehow throw exc if you try to pass in "sorted" Fields?
/** Returns a single {@link Fields} instance for this
* reader, merging fields/terms/docs/positions on the
* fly. This method will return null if the reader

View File

@ -57,7 +57,9 @@ public final class MultiPostingsEnum extends PostingsEnum {
return this.parent == parent;
}
/** Rre-use and reset this instance on the provided slices. */
// nocommit is this class supposed to be aware of index sorting too???
/** Re-use and reset this instance on the provided slices. */
public MultiPostingsEnum reset(final EnumWithSlice[] subs, final int numSubs) {
this.numSubs = numSubs;
for(int i=0;i<numSubs;i++) {
@ -165,9 +167,6 @@ public final class MultiPostingsEnum extends PostingsEnum {
/** Holds a {@link PostingsEnum} along with the
* corresponding {@link ReaderSlice}. */
public final static class EnumWithSlice {
EnumWithSlice() {
}
/** {@link PostingsEnum} for this sub-reader. */
public PostingsEnum postingsEnum;

View File

@ -0,0 +1,221 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.search.LeafFieldComparator;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.PriorityQueue;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
final class MultiSorter {
/** Does a merge sort of the leaves of the incoming reader, returning {@link MergeState#DocMap} to map each leaf's
* documents into the merged segment. The documents for each incoming leaf reader must already be sorted by the same sort! */
static MergeState.DocMap[] sort(Sort sort, List<CodecReader> readers) throws IOException {
SortField fields[] = sort.getSort();
final CrossReaderComparator[] comparators = new CrossReaderComparator[fields.length];
for(int i=0;i<fields.length;i++) {
comparators[i] = getComparator(readers, fields[i]);
}
int leafCount = readers.size();
PriorityQueue<LeafAndDocID> queue = new PriorityQueue<LeafAndDocID>(leafCount) {
@Override
public boolean lessThan(LeafAndDocID a, LeafAndDocID b) {
for(int i=0;i<comparators.length;i++) {
int cmp = comparators[i].compare(a.readerIndex, a.docID, b.readerIndex, b.docID);
if (cmp != 0) {
return cmp < 0;
}
}
// tie-break by docID natural order:
if (a.readerIndex != b.readerIndex) {
return a.readerIndex < b.readerIndex;
}
return a.docID < b.docID;
}
};
PackedLongValues.Builder[] builders = new PackedLongValues.Builder[leafCount];
for(int i=0;i<leafCount;i++) {
CodecReader reader = readers.get(i);
queue.add(new LeafAndDocID(i, reader.getLiveDocs(), reader.maxDoc()));
builders[i] = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
}
int mappedDocID = 0;
while (queue.size() != 0) {
LeafAndDocID top = queue.top();
builders[top.readerIndex].add(mappedDocID);
if (top.liveDocs == null || top.liveDocs.get(top.docID)) {
mappedDocID++;
}
top.docID++;
if (top.docID < top.maxDoc) {
queue.updateTop();
} else {
queue.pop();
}
}
MergeState.DocMap[] docMaps = new MergeState.DocMap[leafCount];
for(int i=0;i<leafCount;i++) {
final PackedLongValues remapped = builders[i].build();
final Bits liveDocs = readers.get(i).getLiveDocs();
docMaps[i] = new MergeState.DocMap() {
@Override
public int get(int docID) {
if (liveDocs == null || liveDocs.get(docID)) {
return (int) remapped.get(docID);
} else {
return -1;
}
}
};
}
return docMaps;
}
private static class LeafAndDocID {
final int readerIndex;
final Bits liveDocs;
final int maxDoc;
int docID;
public LeafAndDocID(int readerIndex, Bits liveDocs, int maxDoc) {
this.readerIndex = readerIndex;
this.liveDocs = liveDocs;
this.maxDoc = maxDoc;
}
}
private interface CrossReaderComparator {
public int compare(int readerIndexA, int docIDA, int readerIndexB, int docIDB);
}
private static CrossReaderComparator getComparator(List<CodecReader> readers, SortField sortField) throws IOException {
switch(sortField.getType()) {
// TODO: use global ords for string sort
case INT:
{
List<NumericDocValues> values = new ArrayList<>();
List<Bits> docsWithFields = new ArrayList<>();
for(CodecReader reader : readers) {
values.add(DocValues.getNumeric(reader, sortField.getField()));
docsWithFields.add(DocValues.getDocsWithField(reader, sortField.getField()));
}
final int reverseMul;
if (sortField.getReverse()) {
reverseMul = -1;
} else {
reverseMul = 1;
}
final int missingValue;
if (sortField.getMissingValue() != null) {
missingValue = (Integer) sortField.getMissingValue();
} else {
missingValue = 0;
}
return new CrossReaderComparator() {
@Override
public int compare(int readerIndexA, int docIDA, int readerIndexB, int docIDB) {
int valueA;
if (docsWithFields.get(readerIndexA).get(docIDA)) {
valueA = (int) values.get(readerIndexA).get(docIDA);
} else {
valueA = missingValue;
}
int valueB;
if (docsWithFields.get(readerIndexB).get(docIDB)) {
valueB = (int) values.get(readerIndexB).get(docIDB);
} else {
valueB = missingValue;
}
return reverseMul * Integer.compare(valueA, valueB);
}
};
}
case LONG:
// nocommit refactor/share at least numerics here:
{
List<NumericDocValues> values = new ArrayList<>();
List<Bits> docsWithFields = new ArrayList<>();
for(CodecReader reader : readers) {
values.add(DocValues.getNumeric(reader, sortField.getField()));
docsWithFields.add(DocValues.getDocsWithField(reader, sortField.getField()));
}
final int reverseMul;
if (sortField.getReverse()) {
reverseMul = -1;
} else {
reverseMul = 1;
}
final int missingValue;
if (sortField.getMissingValue() != null) {
missingValue = (Integer) sortField.getMissingValue();
} else {
missingValue = 0;
}
return new CrossReaderComparator() {
@Override
public int compare(int readerIndexA, int docIDA, int readerIndexB, int docIDB) {
long valueA;
if (docsWithFields.get(readerIndexA).get(docIDA)) {
valueA = (int) values.get(readerIndexA).get(docIDA);
} else {
valueA = missingValue;
}
long valueB;
if (docsWithFields.get(readerIndexB).get(docIDB)) {
valueB = (int) values.get(readerIndexB).get(docIDB);
} else {
valueB = missingValue;
}
return reverseMul * Long.compare(valueA, valueB);
}
};
}
// nocommit do the rest:
default:
throw new IllegalArgumentException("unhandled SortField.getType()=" + sortField.getType());
}
}
}

View File

@ -26,6 +26,7 @@ import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import org.apache.lucene.search.Sort;
import org.apache.lucene.util.Bits;
/** An {@link LeafReader} which reads multiple, parallel indexes. Each index
@ -55,6 +56,7 @@ public class ParallelLeafReader extends LeafReader {
private final boolean closeSubReaders;
private final int maxDoc, numDocs;
private final boolean hasDeletions;
private final Sort indexSort;
private final SortedMap<String,LeafReader> fieldToReader = new TreeMap<>();
private final SortedMap<String,LeafReader> tvFieldToReader = new TreeMap<>();
@ -100,8 +102,17 @@ public class ParallelLeafReader extends LeafReader {
// TODO: make this read-only in a cleaner way?
FieldInfos.Builder builder = new FieldInfos.Builder();
Sort indexSort = null;
// build FieldInfos and fieldToReader map:
for (final LeafReader reader : this.parallelReaders) {
if (indexSort == null) {
indexSort = reader.getIndexSort();
} else if (indexSort.equals(reader.getIndexSort()) == false) {
throw new IllegalArgumentException("cannot combine LeafReaders that have different index sorts: saw both sort=" + indexSort + " and " + reader.getIndexSort());
}
final FieldInfos readerFieldInfos = reader.getFieldInfos();
for (FieldInfo fieldInfo : readerFieldInfos) {
// NOTE: first reader having a given field "wins":
@ -115,6 +126,7 @@ public class ParallelLeafReader extends LeafReader {
}
}
fieldInfos = builder.finish();
this.indexSort = indexSort;
// build Fields instance
for (final LeafReader reader : this.parallelReaders) {
@ -423,4 +435,10 @@ public class ParallelLeafReader extends LeafReader {
ensureOpen();
return parallelReaders;
}
@Override
public Sort getIndexSort() {
return indexSort;
}
}

View File

@ -28,6 +28,7 @@ import java.util.Set;
import java.util.regex.Matcher;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.search.Sort;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.TrackingDirectoryWrapper;
import org.apache.lucene.util.StringHelper;
@ -69,6 +70,8 @@ public final class SegmentInfo {
private final Map<String,String> attributes;
private final Sort indexSort;
// Tracks the Lucene version this segment was created with, since 3.1. Null
// indicates an older than 3.0 index, and it's used to detect a too old index.
// The format expected is "x.y" - "2.x" for pre-3.0 indexes (or null), and
@ -93,7 +96,7 @@ public final class SegmentInfo {
*/
public SegmentInfo(Directory dir, Version version, String name, int maxDoc,
boolean isCompoundFile, Codec codec, Map<String,String> diagnostics,
byte[] id, Map<String,String> attributes) {
byte[] id, Map<String,String> attributes, Sort indexSort) {
assert !(dir instanceof TrackingDirectoryWrapper);
this.dir = Objects.requireNonNull(dir);
this.version = Objects.requireNonNull(version);
@ -107,6 +110,7 @@ public final class SegmentInfo {
throw new IllegalArgumentException("invalid id: " + Arrays.toString(id));
}
this.attributes = Objects.requireNonNull(attributes);
this.indexSort = indexSort;
}
/**
@ -194,13 +198,13 @@ public final class SegmentInfo {
s.append('/').append(delCount);
}
final String sorter_key = "sorter"; // SortingMergePolicy.SORTER_ID_PROP; // TODO: use this once we can import SortingMergePolicy (currently located in 'misc' instead of 'core')
final String sorter_val = diagnostics.get(sorter_key);
if (sorter_val != null) {
s.append(":[");
s.append(sorter_key);
s.append('=');
s.append(sorter_val);
// nocommit does search time "do the right thing" automatically when segment is sorted?
// nocommit remove sorter_key from diagnostics
if (indexSort != null) {
s.append(":[indexSort=");
s.append(indexSort);
s.append(']');
}
@ -311,5 +315,10 @@ public final class SegmentInfo {
public Map<String,String> getAttributes() {
return attributes;
}
/** Return the sort order of this segment, or null if the index has no sort. */
public Sort getIndexSort() {
return indexSort;
}
}

View File

@ -28,6 +28,7 @@ import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.search.Sort;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Bits;
@ -303,4 +304,9 @@ public final class SegmentReader extends CodecReader {
ensureOpen();
core.removeCoreClosedListener(listener);
}
@Override
public Sort getIndexSort() {
return si.info.getIndexSort();
}
}

View File

@ -26,6 +26,7 @@ import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.search.Sort;
import org.apache.lucene.util.Bits;
/**
@ -125,6 +126,16 @@ public final class SlowCodecReaderWrapper {
public void removeCoreClosedListener(CoreClosedListener listener) {
reader.removeCoreClosedListener(listener);
}
@Override
public String toString() {
return "SlowCodecReaderWrapper(" + reader + ")";
}
@Override
public Sort getIndexSort() {
return reader.getIndexSort();
}
};
}
}

View File

@ -33,6 +33,7 @@ import org.apache.lucene.util.packed.PackedLongValues;
* IDs.
* @lucene.experimental
*/
// nocommit rename to IndexSorter?
final class Sorter {
final Sort sort;
@ -168,6 +169,7 @@ final class Sorter {
}
final PackedLongValues newToOld = newToOldBuilder.build();
// invert the docs mapping:
for (int i = 0; i < maxDoc; ++i) {
docs[(int) newToOld.get(i)] = i;
} // docs is now the oldToNew mapping
@ -196,7 +198,7 @@ final class Sorter {
}
};
}
/**
* Returns a mapping from the old document ID to its new location in the
* sorted index. Implementations can use the auxiliary

View File

@ -1,3 +1,5 @@
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
@ -14,7 +16,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.io.IOException;
import java.util.Arrays;
@ -35,21 +36,13 @@ import org.apache.lucene.util.automaton.CompiledAutomaton;
/**
* An {@link org.apache.lucene.index.LeafReader} which supports sorting documents by a given
* {@link Sort}. You can use this class to sort an index as follows:
*
* <pre class="prettyprint">
* IndexWriter writer; // writer to which the sorted index will be added
* DirectoryReader reader; // reader on the input index
* Sort sort; // determines how the documents are sorted
* LeafReader sortingReader = SortingLeafReader.wrap(SlowCompositeReaderWrapper.wrap(reader), sort);
* writer.addIndexes(reader);
* writer.close();
* reader.close();
* </pre>
* {@link Sort}. This is package private and is only used by Lucene when it needs to merge
* a newly flushed (unsorted) segment.
*
* @lucene.experimental
*/
public class SortingLeafReader extends FilterLeafReader {
class SortingLeafReader extends FilterLeafReader {
private static class SortingFields extends FilterFields {
@ -111,25 +104,6 @@ public class SortingLeafReader extends FilterLeafReader {
this.hasPositions = hasPositions;
}
Bits newToOld(final Bits liveDocs) {
if (liveDocs == null) {
return null;
}
return new Bits() {
@Override
public boolean get(int index) {
return liveDocs.get(docMap.oldToNew(index));
}
@Override
public int length() {
return liveDocs.length();
}
};
}
@Override
public PostingsEnum postings( PostingsEnum reuse, final int flags) throws IOException {
@ -363,11 +337,15 @@ public class SortingLeafReader extends FilterLeafReader {
@Override
public long nextOrd() {
return in.nextOrd();
// nocommit
long v = in.nextOrd();
//System.out.println(" slr.sssdv.nextOrd return " + v + " this=" + this);
return v;
}
@Override
public void setDocument(int docID) {
//System.out.println(" slr.sssdv.setDocument docID=" + docID + " this=" + this);
in.setDocument(docMap.newToOld(docID));
}
@ -865,7 +843,7 @@ public class SortingLeafReader extends FilterLeafReader {
if (inPointValues == null) {
return null;
} else {
// TODO: this is untested!
// nocommit make sure this is tested
return new SortingPointValues(inPointValues, docMap);
}
}

View File

@ -147,6 +147,9 @@ public class Sort {
* etc. Finally, if there is still a tie after all SortFields
* are checked, the internal Lucene docid is used to break it. */
public void setSort(SortField... fields) {
if (fields.length == 0) {
throw new IllegalArgumentException("There must be at least 1 sort field");
}
this.fields = fields;
}

View File

@ -299,9 +299,6 @@ public class BKDWriter implements Closeable {
final BKDReader.IntersectState state;
final MergeState.DocMap docMap;
/** Base offset for all our docIDs */
final int docIDBase;
/** Current doc ID */
public int docID;
@ -314,7 +311,7 @@ public class BKDWriter implements Closeable {
/** Which leaf block we are up to */
private int blockID;
public MergeReader(BKDReader bkd, MergeState.DocMap docMap, int docIDBase) throws IOException {
public MergeReader(BKDReader bkd, MergeState.DocMap docMap) throws IOException {
this.bkd = bkd;
state = new BKDReader.IntersectState(bkd.in.clone(),
bkd.numDims,
@ -322,7 +319,6 @@ public class BKDWriter implements Closeable {
bkd.maxPointsInLeafNode,
null);
this.docMap = docMap;
this.docIDBase = docIDBase;
long minFP = Long.MAX_VALUE;
//System.out.println("MR.init " + this + " bkdreader=" + bkd + " leafBlockFPs.length=" + bkd.leafBlockFPs.length);
for(long fp : bkd.leafBlockFPs) {
@ -396,14 +392,14 @@ public class BKDWriter implements Closeable {
}
// Tie break by sorting smaller docIDs earlier:
return a.docIDBase < b.docIDBase;
return a.docID < b.docID;
}
}
/** More efficient bulk-add for incoming {@link BKDReader}s. This does a merge sort of the already
* sorted values and currently only works when numDims==1. This returns -1 if all documents containing
* dimensional values were deleted. */
public long merge(IndexOutput out, List<MergeState.DocMap> docMaps, List<BKDReader> readers, List<Integer> docIDBases) throws IOException {
public long merge(IndexOutput out, List<MergeState.DocMap> docMaps, List<BKDReader> readers) throws IOException {
if (numDims != 1) {
throw new UnsupportedOperationException("numDims must be 1 but got " + numDims);
}
@ -411,8 +407,6 @@ public class BKDWriter implements Closeable {
throw new IllegalStateException("cannot mix add and merge");
}
//System.out.println("BKDW.merge segs=" + readers.size());
// Catch user silliness:
if (heapPointWriter == null && tempInput == null) {
throw new IllegalStateException("already finished");
@ -433,7 +427,7 @@ public class BKDWriter implements Closeable {
} else {
docMap = docMaps.get(i);
}
MergeReader reader = new MergeReader(bkd, docMap, docIDBases.get(i));
MergeReader reader = new MergeReader(bkd, docMap);
if (reader.next()) {
queue.add(reader);
}
@ -468,7 +462,7 @@ public class BKDWriter implements Closeable {
// System.out.println("iter reader=" + reader);
// NOTE: doesn't work with subclasses (e.g. SimpleText!)
int docID = reader.docIDBase + reader.docID;
int docID = reader.docID;
leafBlockDocIDs[leafCount] = docID;
System.arraycopy(reader.state.scratchPackedValue, 0, leafBlockPackedValues[leafCount], 0, packedBytesLength);
docsSeen.set(docID);

View File

@ -222,7 +222,7 @@ public class TestCodecs extends LuceneTestCase {
final FieldInfos fieldInfos = builder.finish();
final Directory dir = newDirectory();
Codec codec = Codec.getDefault();
final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
this.write(si, fieldInfos, dir, fields);
final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random())));
@ -279,7 +279,7 @@ public class TestCodecs extends LuceneTestCase {
}
Codec codec = Codec.getDefault();
final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
this.write(si, fieldInfos, dir, fields);
if (VERBOSE) {

View File

@ -503,7 +503,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
class ReindexingOneMerge extends OneMerge {
List<LeafReader> parallelReaders;
final List<ParallelLeafReader> parallelReaders = new ArrayList<>();
final long schemaGen;
ReindexingOneMerge(List<SegmentCommitInfo> segments) {
@ -519,33 +519,23 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
}
@Override
public List<CodecReader> getMergeReaders() throws IOException {
if (parallelReaders == null) {
parallelReaders = new ArrayList<>();
for (CodecReader reader : super.getMergeReaders()) {
parallelReaders.add(getCurrentReader((SegmentReader)reader, schemaGen));
}
public CodecReader wrapForMerge(CodecReader reader) throws IOException {
LeafReader wrapped = getCurrentReader((SegmentReader)reader, schemaGen);
if (wrapped instanceof ParallelLeafReader) {
parallelReaders.add((ParallelLeafReader) wrapped);
}
// TODO: fix ParallelLeafReader, if this is a good use case
List<CodecReader> mergeReaders = new ArrayList<>();
for (LeafReader reader : parallelReaders) {
mergeReaders.add(SlowCodecReaderWrapper.wrap(reader));
}
return mergeReaders;
return SlowCodecReaderWrapper.wrap(wrapped);
}
@Override
public void mergeFinished() throws IOException {
Throwable th = null;
for(LeafReader r : parallelReaders) {
if (r instanceof ParallelLeafReader) {
try {
r.decRef();
} catch (Throwable t) {
if (th == null) {
th = t;
}
for (ParallelLeafReader r : parallelReaders) {
try {
r.decRef();
} catch (Throwable t) {
if (th == null) {
th = t;
}
}
}
@ -561,10 +551,6 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
super.setMergeInfo(info);
}
@Override
public MergePolicy.DocMap getDocMap(final MergeState mergeState) {
return super.getDocMap(mergeState);
}
}
class ReindexingMergeSpecification extends MergeSpecification {

View File

@ -218,7 +218,7 @@ public class TestDoc extends LuceneTestCase {
final Codec codec = Codec.getDefault();
TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(si1.info.dir);
final SegmentInfo si = new SegmentInfo(si1.info.dir, Version.LATEST, merged, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
final SegmentInfo si = new SegmentInfo(si1.info.dir, Version.LATEST, merged, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
SegmentMerger merger = new SegmentMerger(Arrays.<CodecReader>asList(r1, r2),
si, InfoStream.getDefault(), trackingDir,

View File

@ -0,0 +1,179 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
public class TestDocIDMerger extends LuceneTestCase {
private static class TestSubUnsorted extends DocIDMerger.Sub {
private int docID = -1;
final int valueStart;
final int maxDoc;
public TestSubUnsorted(MergeState.DocMap docMap, Bits liveDocs, int maxDoc, int valueStart) {
super(docMap, liveDocs);
this.maxDoc = maxDoc;
this.valueStart = valueStart;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
return docID;
}
}
public int getValue() {
return valueStart + docID;
}
}
public void testNoSort() throws Exception {
int subCount = TestUtil.nextInt(random(), 1, 20);
List<TestSubUnsorted> subs = new ArrayList<>();
int valueStart = 0;
for(int i=0;i<subCount;i++) {
int maxDoc = TestUtil.nextInt(random(), 1, 1000);
final int docBase = valueStart;
subs.add(new TestSubUnsorted(new MergeState.DocMap() {
@Override
public int get(int docID) {
return docBase + docID;
}
}, null, maxDoc, valueStart));
valueStart += maxDoc;
}
DocIDMerger<TestSubUnsorted> merger = new DocIDMerger<>(subs, false);
int count = 0;
while (true) {
TestSubUnsorted sub = merger.next();
if (sub == null) {
break;
}
assertEquals(count, sub.mappedDocID);
assertEquals(count, sub.getValue());
count++;
}
assertEquals(valueStart, count);
}
private static class TestSubSorted extends DocIDMerger.Sub {
private int docID = -1;
final int maxDoc;
final int index;
public TestSubSorted(MergeState.DocMap docMap, Bits liveDocs, int maxDoc, int index) {
super(docMap, liveDocs);
this.maxDoc = maxDoc;
this.index = index;
}
@Override
public int nextDoc() {
docID++;
if (docID == maxDoc) {
return NO_MORE_DOCS;
} else {
return docID;
}
}
@Override
public String toString() {
return "TestSubSorted(index=" + index + ", mappedDocID=" + mappedDocID+ ")";
}
}
public void testWithSort() throws Exception {
int subCount = TestUtil.nextInt(random(), 1, 20);
List<int[]> oldToNew = new ArrayList<>();
// how many docs we've written to each sub:
List<Integer> uptos = new ArrayList<>();
int totDocCount = 0;
for(int i=0;i<subCount;i++) {
int maxDoc = TestUtil.nextInt(random(), 1, 1000);
uptos.add(0);
oldToNew.add(new int[maxDoc]);
totDocCount += maxDoc;
}
List<int[]> completedSubs = new ArrayList<>();
// randomly distribute target docIDs into the segments:
for(int docID=0;docID<totDocCount;docID++) {
int sub = random().nextInt(oldToNew.size());
int upto = uptos.get(sub);
int[] subDocs = oldToNew.get(sub);
subDocs[upto] = docID;
upto++;
if (upto == subDocs.length) {
completedSubs.add(subDocs);
oldToNew.remove(sub);
uptos.remove(sub);
} else {
uptos.set(sub, upto);
}
}
assertEquals(0, oldToNew.size());
List<TestSubSorted> subs = new ArrayList<>();
for(int i=0;i<subCount;i++) {
final int[] docMap = completedSubs.get(i);
subs.add(new TestSubSorted(new MergeState.DocMap() {
@Override
public int get(int docID) {
return docMap[docID];
}
}, null, docMap.length, i));
}
// nocommit test w/ deletions too
DocIDMerger<TestSubSorted> merger = new DocIDMerger<>(subs, true);
int count = 0;
while (true) {
TestSubSorted sub = merger.next();
if (sub == null) {
break;
}
assertEquals(count, sub.mappedDocID);
count++;
}
assertEquals(totDocCount, count);
}
// nocommit more tests, e.g. deleted docs
}

View File

@ -0,0 +1,792 @@
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.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.BinaryPoint;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.PointValues.IntersectVisitor;
import org.apache.lucene.index.PointValues.Relation;
import org.apache.lucene.index.TermsEnum.SeekStatus;
import org.apache.lucene.search.CollectionStatistics;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TermStatistics;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.NumericUtils;
import org.apache.lucene.util.TestUtil;
import org.junit.AfterClass;
import org.junit.BeforeClass;
// nocommit test tie break
// nocommit test multiple sorts
// nocommit test update dvs
// nocommit test EarlyTerminatingCollector
public class TestIndexSorting extends LuceneTestCase {
public void testSortOnMerge(boolean withDeletes) throws IOException {
Directory dir = newDirectory();
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
iwc.setCodec(new SimpleTextCodec()); // nocommit only simple-text supports sorting so far
Sort indexSort = new Sort(new SortField("foo", SortField.Type.LONG));
iwc.setIndexSort(indexSort);
IndexWriter w = new IndexWriter(dir, iwc);
final int numDocs = atLeast(200);
final FixedBitSet deleted = new FixedBitSet(numDocs);
for (int i = 0; i < numDocs; ++i) {
Document doc = new Document();
doc.add(new NumericDocValuesField("foo", random().nextInt(20)));
doc.add(new StringField("id", Integer.toString(i), Store.YES));
doc.add(new NumericDocValuesField("id", i));
w.addDocument(doc);
if (random().nextInt(5) == 0) {
w.getReader().close();
} else if (random().nextInt(30) == 0) {
w.forceMerge(2);
} else if (random().nextInt(4) == 0) {
final int id = TestUtil.nextInt(random(), 0, i);
deleted.set(id);
w.deleteDocuments(new Term("id", Integer.toString(id)));
}
}
// Check that segments are sorted
DirectoryReader reader = w.getReader();
for (LeafReaderContext ctx : reader.leaves()) {
final SegmentReader leaf = (SegmentReader) ctx.reader();
SegmentInfo info = leaf.getSegmentInfo().info;
switch (info.getDiagnostics().get(IndexWriter.SOURCE)) {
case IndexWriter.SOURCE_FLUSH:
assertNull(info.getIndexSort());
break;
case IndexWriter.SOURCE_MERGE:
assertEquals(indexSort, info.getIndexSort());
final NumericDocValues values = leaf.getNumericDocValues("foo");
long previous = Long.MIN_VALUE;
for (int i = 0; i < leaf.maxDoc(); ++i) {
final long value = values.get(i);
assertTrue(value >= previous);
previous = value;
}
break;
default:
fail();
}
}
// Now check that the index is consistent
IndexSearcher searcher = newSearcher(reader);
for (int i = 0; i < numDocs; ++i) {
TermQuery termQuery = new TermQuery(new Term("id", Integer.toString(i)));
final TopDocs topDocs = searcher.search(termQuery, 1);
if (deleted.get(i)) {
assertEquals(0, topDocs.totalHits);
} else {
assertEquals(1, topDocs.totalHits);
assertEquals(i, MultiDocValues.getNumericValues(reader, "id").get(topDocs.scoreDocs[0].doc));
Document document = reader.document(topDocs.scoreDocs[0].doc);
assertEquals(Integer.toString(i), document.get("id"));
}
}
reader.close();
w.close();
dir.close();
}
public void testSortOnMerge() throws IOException {
testSortOnMerge(false);
}
public void testSortOnMergeWithDeletes() throws IOException {
testSortOnMerge(true);
}
static class UpdateRunnable implements Runnable {
private final int numDocs;
private final Random random;
private final AtomicInteger updateCount;
private final IndexWriter w;
private final Map<Integer, Long> values;
private final CountDownLatch latch;
UpdateRunnable(int numDocs, Random random, CountDownLatch latch, AtomicInteger updateCount, IndexWriter w, Map<Integer, Long> values) {
this.numDocs = numDocs;
this.random = random;
this.latch = latch;
this.updateCount = updateCount;
this.w = w;
this.values = values;
}
@Override
public void run() {
try {
latch.await();
while (updateCount.decrementAndGet() >= 0) {
final int id = random.nextInt(numDocs);
final long value = random.nextInt(20);
Document doc = new Document();
doc.add(new StringField("id", Integer.toString(id), Store.NO));
doc.add(new NumericDocValuesField("foo", value));
synchronized (values) {
w.updateDocument(new Term("id", Integer.toString(id)), doc);
values.put(id, value);
}
switch (random.nextInt(10)) {
case 0:
case 1:
// reopen
DirectoryReader.open(w).close();
break;
case 2:
w.forceMerge(3);
break;
}
}
} catch (IOException | InterruptedException e) {
throw new RuntimeException(e);
}
}
}
// There is tricky logic to resolve deletes that happened while merging
public void testConcurrentUpdates() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
iwc.setCodec(new SimpleTextCodec()); // nocommit only simple-text supports sorting so far
Sort indexSort = new Sort(new SortField("foo", SortField.Type.LONG));
iwc.setIndexSort(indexSort);
IndexWriter w = new IndexWriter(dir, iwc);
Map<Integer, Long> values = new HashMap<>();
final int numDocs = atLeast(100);
Thread[] threads = new Thread[2];
final AtomicInteger updateCount = new AtomicInteger(atLeast(1000));
final CountDownLatch latch = new CountDownLatch(1);
for (int i = 0; i < threads.length; ++i) {
Random r = new Random(random().nextLong());
threads[i] = new Thread(new UpdateRunnable(numDocs, r, latch, updateCount, w, values));
}
for (Thread thread : threads) {
thread.start();
}
latch.countDown();
for (Thread thread : threads) {
thread.join();
}
w.forceMerge(1);
DirectoryReader reader = DirectoryReader.open(w);
IndexSearcher searcher = newSearcher(reader);
for (int i = 0; i < numDocs; ++i) {
final TopDocs topDocs = searcher.search(new TermQuery(new Term("id", Integer.toString(i))), 1);
if (values.containsKey(i) == false) {
assertEquals(0, topDocs.totalHits);
} else {
assertEquals(1, topDocs.totalHits);
assertEquals(values.get(i).longValue(), MultiDocValues.getNumericValues(reader, "foo").get(topDocs.scoreDocs[0].doc));
}
}
reader.close();
w.close();
dir.close();
}
static class DVUpdateRunnable implements Runnable {
private final int numDocs;
private final Random random;
private final AtomicInteger updateCount;
private final IndexWriter w;
private final Map<Integer, Long> values;
private final CountDownLatch latch;
DVUpdateRunnable(int numDocs, Random random, CountDownLatch latch, AtomicInteger updateCount, IndexWriter w, Map<Integer, Long> values) {
this.numDocs = numDocs;
this.random = random;
this.latch = latch;
this.updateCount = updateCount;
this.w = w;
this.values = values;
}
@Override
public void run() {
try {
latch.await();
while (updateCount.decrementAndGet() >= 0) {
final int id = random.nextInt(numDocs);
final long value = random.nextInt(20);
synchronized (values) {
w.updateDocValues(new Term("id", Integer.toString(id)), new NumericDocValuesField("foo", value));
values.put(id, value);
}
switch (random.nextInt(10)) {
case 0:
case 1:
// reopen
DirectoryReader.open(w).close();
break;
case 2:
w.forceMerge(3);
break;
}
}
} catch (IOException | InterruptedException e) {
throw new RuntimeException(e);
}
}
}
// There is tricky logic to resolve dv updates that happened while merging
public void testConcurrentDVUpdates() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
iwc.setCodec(new SimpleTextCodec()); // nocommit only simple-text supports sorting so far
Sort indexSort = new Sort(new SortField("foo", SortField.Type.LONG));
iwc.setIndexSort(indexSort);
IndexWriter w = new IndexWriter(dir, iwc);
Map<Integer, Long> values = new HashMap<>();
final int numDocs = atLeast(100);
for (int i = 0; i < numDocs; ++i) {
Document doc = new Document();
doc.add(new StringField("id", Integer.toString(i), Store.NO));
doc.add(new NumericDocValuesField("foo", -1));
values.put(i, -1L);
}
Thread[] threads = new Thread[2];
final AtomicInteger updateCount = new AtomicInteger(atLeast(1000));
final CountDownLatch latch = new CountDownLatch(1);
for (int i = 0; i < threads.length; ++i) {
Random r = new Random(random().nextLong());
threads[i] = new Thread(new UpdateRunnable(numDocs, r, latch, updateCount, w, values));
}
for (Thread thread : threads) {
thread.start();
}
latch.countDown();
for (Thread thread : threads) {
thread.join();
}
w.forceMerge(1);
DirectoryReader reader = DirectoryReader.open(w);
IndexSearcher searcher = newSearcher(reader);
for (int i = 0; i < numDocs; ++i) {
final TopDocs topDocs = searcher.search(new TermQuery(new Term("id", Integer.toString(i))), 1);
assertEquals(1, topDocs.totalHits);
assertEquals(values.get(i).longValue(), MultiDocValues.getNumericValues(reader, "foo").get(topDocs.scoreDocs[0].doc));
}
reader.close();
w.close();
dir.close();
}
public void testAddIndexes(boolean withDeletes) throws Exception {
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
final int numDocs = atLeast(100);
for (int i = 0; i < numDocs; ++i) {
Document doc = new Document();
doc.add(new StringField("id", Integer.toString(i), Store.NO));
doc.add(new NumericDocValuesField("foo", random().nextInt(20)));
w.addDocument(doc);
}
if (withDeletes) {
for (int i = random().nextInt(5); i < numDocs; i += TestUtil.nextInt(random(), 1, 5)) {
w.deleteDocuments(new Term("id", Integer.toString(i)));
}
}
final IndexReader reader = w.getReader();
Directory dir2 = newDirectory();
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
iwc.setCodec(new SimpleTextCodec()); // nocommit only simple-text supports sorting so far
Sort indexSort = new Sort(new SortField("foo", SortField.Type.LONG));
iwc.setIndexSort(indexSort);
IndexWriter w2 = new IndexWriter(dir2, iwc);
CodecReader[] codecReaders = new CodecReader[reader.leaves().size()];
for (int i = 0; i < codecReaders.length; ++i) {
codecReaders[i] = (CodecReader) reader.leaves().get(i).reader();
}
w2.addIndexes(codecReaders);
final IndexReader reader2 = w2.getReader();
final IndexSearcher searcher = newSearcher(reader);
final IndexSearcher searcher2 = newSearcher(reader2);
for (int i = 0; i < numDocs; ++i) {
Query query = new TermQuery(new Term("id", Integer.toString(i)));
final TopDocs topDocs = searcher.search(query, 1);
final TopDocs topDocs2 = searcher2.search(query, 1);
assertEquals(topDocs.totalHits, topDocs2.totalHits);
if (topDocs.totalHits == 1) {
assertEquals(
MultiDocValues.getNumericValues(reader, "foo").get(topDocs.scoreDocs[0].doc),
MultiDocValues.getNumericValues(reader2, "foo").get(topDocs2.scoreDocs[0].doc));
}
}
IOUtils.close(reader, reader2, w, w2, dir, dir2);
}
public void testAddIndexes() throws Exception {
testAddIndexes(false);
}
public void testAddIndexesWithDeletions() throws Exception {
testAddIndexes(true);
}
public void testBadSort() throws Exception {
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
iwc.setIndexSort(Sort.RELEVANCE);
});
assertEquals("invalid SortField type: must be one of [STRING, INT, FLOAT, LONG, DOUBLE, BYTES] but got: <score>", expected.getMessage());
}
// you can't change the index sort on an existing index:
public void testIllegalChangeSort() throws Exception {
final Directory dir = newDirectory();
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
iwc.setCodec(new SimpleTextCodec()); // nocommit only simple-text supports sorting so far
iwc.setIndexSort(new Sort(new SortField("foo", SortField.Type.LONG)));
IndexWriter w = new IndexWriter(dir, iwc);
w.addDocument(new Document());
DirectoryReader.open(w).close();
w.addDocument(new Document());
w.forceMerge(1);
w.close();
final IndexWriterConfig iwc2 = new IndexWriterConfig(new MockAnalyzer(random()));
iwc2.setCodec(new SimpleTextCodec()); // nocommit only simple-text supports sorting so far
iwc2.setIndexSort(new Sort(new SortField("bar", SortField.Type.LONG)));
IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
new IndexWriter(dir, iwc2);
});
assertEquals("cannot change previous indexSort=<long: \"foo\"> (from segment=_2(7.0.0):c2:[indexSort=<long: \"foo\">]) to new indexSort=<long: \"bar\">", expected.getMessage());
dir.close();
}
static final class NormsSimilarity extends Similarity {
private final Similarity in;
public NormsSimilarity(Similarity in) {
this.in = in;
}
@Override
public long computeNorm(FieldInvertState state) {
if (state.getName().equals(NORMS_FIELD)) {
return Float.floatToIntBits(state.getBoost());
} else {
return in.computeNorm(state);
}
}
@Override
public SimWeight computeWeight(CollectionStatistics collectionStats, TermStatistics... termStats) {
return in.computeWeight(collectionStats, termStats);
}
@Override
public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
return in.simScorer(weight, context);
}
}
static final class PositionsTokenStream extends TokenStream {
private final CharTermAttribute term;
private final PayloadAttribute payload;
private final OffsetAttribute offset;
private int pos, off;
public PositionsTokenStream() {
term = addAttribute(CharTermAttribute.class);
payload = addAttribute(PayloadAttribute.class);
offset = addAttribute(OffsetAttribute.class);
}
@Override
public boolean incrementToken() throws IOException {
if (pos == 0) {
return false;
}
clearAttributes();
term.append(DOC_POSITIONS_TERM);
payload.setPayload(new BytesRef(Integer.toString(pos)));
offset.setOffset(off, off);
--pos;
++off;
return true;
}
void setId(int id) {
pos = id / 10 + 1;
off = 0;
}
}
private static Directory dir;
private static IndexReader sortedReader;
private static final FieldType TERM_VECTORS_TYPE = new FieldType(TextField.TYPE_NOT_STORED);
static {
TERM_VECTORS_TYPE.setStoreTermVectors(true);
TERM_VECTORS_TYPE.freeze();
}
private static final FieldType POSITIONS_TYPE = new FieldType(TextField.TYPE_NOT_STORED);
static {
POSITIONS_TYPE.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
POSITIONS_TYPE.freeze();
}
private static final String ID_FIELD = "id";
private static final String DOCS_ENUM_FIELD = "docs";
private static final String DOCS_ENUM_TERM = "$all$";
private static final String DOC_POSITIONS_FIELD = "positions";
private static final String DOC_POSITIONS_TERM = "$all$";
private static final String NUMERIC_DV_FIELD = "numeric";
private static final String SORTED_NUMERIC_DV_FIELD = "sorted_numeric";
private static final String NORMS_FIELD = "norm";
private static final String BINARY_DV_FIELD = "binary";
private static final String SORTED_DV_FIELD = "sorted";
private static final String SORTED_SET_DV_FIELD = "sorted_set";
private static final String TERM_VECTORS_FIELD = "term_vectors";
private static final String DIMENSIONAL_FIELD = "numeric1d";
private static Document doc(final int id, PositionsTokenStream positions) {
final Document doc = new Document();
doc.add(new StringField(ID_FIELD, Integer.toString(id), Store.YES));
doc.add(new StringField(DOCS_ENUM_FIELD, DOCS_ENUM_TERM, Store.NO));
positions.setId(id);
doc.add(new Field(DOC_POSITIONS_FIELD, positions, POSITIONS_TYPE));
doc.add(new NumericDocValuesField(NUMERIC_DV_FIELD, id));
TextField norms = new TextField(NORMS_FIELD, Integer.toString(id), Store.NO);
norms.setBoost(Float.intBitsToFloat(id));
doc.add(norms);
doc.add(new BinaryDocValuesField(BINARY_DV_FIELD, new BytesRef(Integer.toString(id))));
doc.add(new SortedDocValuesField(SORTED_DV_FIELD, new BytesRef(Integer.toString(id))));
doc.add(new SortedSetDocValuesField(SORTED_SET_DV_FIELD, new BytesRef(Integer.toString(id))));
doc.add(new SortedSetDocValuesField(SORTED_SET_DV_FIELD, new BytesRef(Integer.toString(id + 1))));
doc.add(new SortedNumericDocValuesField(SORTED_NUMERIC_DV_FIELD, id));
doc.add(new SortedNumericDocValuesField(SORTED_NUMERIC_DV_FIELD, id + 1));
doc.add(new Field(TERM_VECTORS_FIELD, Integer.toString(id), TERM_VECTORS_TYPE));
byte[] bytes = new byte[4];
NumericUtils.intToSortableBytes(id, bytes, 0);
doc.add(new BinaryPoint(DIMENSIONAL_FIELD, bytes));
return doc;
}
@AfterClass
public static void afterClass() throws Exception {
if (sortedReader != null) {
sortedReader.close();
sortedReader = null;
}
if (dir != null) {
dir.close();
dir = null;
}
}
@BeforeClass
public static void createIndex() throws Exception {
dir = newFSDirectory(createTempDir());
int numDocs = atLeast(100);
List<Integer> ids = new ArrayList<>();
for (int i = 0; i < numDocs; i++) {
ids.add(Integer.valueOf(i * 10));
}
// shuffle them for indexing
Collections.shuffle(ids, random());
if (VERBOSE) {
System.out.println("Shuffled IDs for indexing: " + Arrays.toString(ids.toArray()));
}
PositionsTokenStream positions = new PositionsTokenStream();
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
// nocommit:
conf.setCodec(new SimpleTextCodec());
conf.setMaxBufferedDocs(4); // create some segments
conf.setSimilarity(new NormsSimilarity(conf.getSimilarity())); // for testing norms field
// nocommit
conf.setMergeScheduler(new SerialMergeScheduler());
// sort the index by id (as integer, in NUMERIC_DV_FIELD)
conf.setIndexSort(new Sort(new SortField(NUMERIC_DV_FIELD, SortField.Type.INT)));
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
writer.setDoRandomForceMerge(false);
for (int id : ids) {
writer.addDocument(doc(id, positions));
}
// delete some documents
writer.commit();
// nocommit need thread safety test too
for (Integer id : ids) {
if (random().nextDouble() < 0.2) {
if (VERBOSE) {
System.out.println("delete doc_id " + id);
}
writer.deleteDocuments(new Term(ID_FIELD, id.toString()));
}
}
sortedReader = writer.getReader();
writer.close();
TestUtil.checkReader(sortedReader);
}
// nocommit just do assertReaderEquals, don't use @BeforeClass, etc.?
public void testBinaryDocValuesField() throws Exception {
for(LeafReaderContext ctx : sortedReader.leaves()) {
LeafReader reader = ctx.reader();
BinaryDocValues dv = reader.getBinaryDocValues(BINARY_DV_FIELD);
boolean isSorted = reader.getIndexSort() != null;
int lastID = Integer.MIN_VALUE;
for (int docID = 0; docID < reader.maxDoc(); docID++) {
BytesRef bytes = dv.get(docID);
String idString = reader.document(docID).get(ID_FIELD);
assertEquals("incorrect binary DocValues for doc " + docID, idString, bytes.utf8ToString());
if (isSorted) {
int id = Integer.parseInt(idString);
assertTrue("lastID=" + lastID + " vs id=" + id, lastID < id);
lastID = id;
}
}
}
}
public void testPostings() throws Exception {
for(LeafReaderContext ctx : sortedReader.leaves()) {
LeafReader reader = ctx.reader();
TermsEnum termsEnum = reader.terms(DOC_POSITIONS_FIELD).iterator();
assertEquals(SeekStatus.FOUND, termsEnum.seekCeil(new BytesRef(DOC_POSITIONS_TERM)));
PostingsEnum sortedPositions = termsEnum.postings(null, PostingsEnum.ALL);
int doc;
boolean isSorted = reader.getIndexSort() != null;
// test nextDoc()
while ((doc = sortedPositions.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
int freq = sortedPositions.freq();
int id = Integer.parseInt(reader.document(doc).get(ID_FIELD));
assertEquals("incorrect freq for doc=" + doc, id / 10 + 1, freq);
for (int i = 0; i < freq; i++) {
assertEquals("incorrect position for doc=" + doc, i, sortedPositions.nextPosition());
assertEquals("incorrect startOffset for doc=" + doc, i, sortedPositions.startOffset());
assertEquals("incorrect endOffset for doc=" + doc, i, sortedPositions.endOffset());
assertEquals("incorrect payload for doc=" + doc, freq - i, Integer.parseInt(sortedPositions.getPayload().utf8ToString()));
}
}
// test advance()
final PostingsEnum reuse = sortedPositions;
sortedPositions = termsEnum.postings(reuse, PostingsEnum.ALL);
doc = 0;
while ((doc = sortedPositions.advance(doc + TestUtil.nextInt(random(), 1, 5))) != DocIdSetIterator.NO_MORE_DOCS) {
int freq = sortedPositions.freq();
int id = Integer.parseInt(reader.document(doc).get(ID_FIELD));
assertEquals("incorrect freq for doc=" + doc, id / 10 + 1, freq);
for (int i = 0; i < freq; i++) {
assertEquals("incorrect position for doc=" + doc, i, sortedPositions.nextPosition());
assertEquals("incorrect startOffset for doc=" + doc, i, sortedPositions.startOffset());
assertEquals("incorrect endOffset for doc=" + doc, i, sortedPositions.endOffset());
assertEquals("incorrect payload for doc=" + doc, freq - i, Integer.parseInt(sortedPositions.getPayload().utf8ToString()));
}
}
}
}
public void testDocsAreSortedByID() throws Exception {
for(LeafReaderContext ctx : sortedReader.leaves()) {
LeafReader reader = ctx.reader();
if (reader.getIndexSort() != null) {
int maxDoc = reader.maxDoc();
int lastID = Integer.MIN_VALUE;
for(int doc=0;doc<maxDoc;doc++) {
int id = Integer.parseInt(reader.document(doc).get(ID_FIELD));
assertTrue(id > lastID);
lastID = id;
}
}
}
}
public void testNormValues() throws Exception {
for(LeafReaderContext ctx : sortedReader.leaves()) {
LeafReader reader = ctx.reader();
NumericDocValues dv = reader.getNormValues(NORMS_FIELD);
int maxDoc = reader.maxDoc();
boolean isSorted = reader.getIndexSort() != null;
for (int doc = 0; doc < maxDoc; doc++) {
int id = Integer.parseInt(reader.document(doc).get(ID_FIELD));
assertEquals("incorrect norm value for doc " + doc, id, dv.get(doc));
}
}
}
public void testNumericDocValuesField() throws Exception {
for(LeafReaderContext ctx : sortedReader.leaves()) {
LeafReader reader = ctx.reader();
NumericDocValues dv = reader.getNumericDocValues(NUMERIC_DV_FIELD);
int maxDoc = reader.maxDoc();
for (int doc = 0; doc < maxDoc; doc++) {
int id = Integer.parseInt(reader.document(doc).get(ID_FIELD));
assertEquals("incorrect numeric DocValues for doc " + doc, id, dv.get(doc));
}
}
}
public void testSortedDocValuesField() throws Exception {
for(LeafReaderContext ctx : sortedReader.leaves()) {
LeafReader reader = ctx.reader();
SortedDocValues dv = reader.getSortedDocValues(SORTED_DV_FIELD);
int maxDoc = reader.maxDoc();
for (int doc = 0; doc < maxDoc; doc++) {
final BytesRef bytes = dv.get(doc);
String id = reader.document(doc).get(ID_FIELD);
assertEquals("incorrect sorted DocValues for doc " + doc, id, bytes.utf8ToString());
}
}
}
public void testSortedSetDocValuesField() throws Exception {
for(LeafReaderContext ctx : sortedReader.leaves()) {
LeafReader reader = ctx.reader();
SortedSetDocValues dv = reader.getSortedSetDocValues(SORTED_SET_DV_FIELD);
int maxDoc = reader.maxDoc();
for (int doc = 0; doc < maxDoc; doc++) {
dv.setDocument(doc);
BytesRef bytes = dv.lookupOrd(dv.nextOrd());
String id = reader.document(doc).get(ID_FIELD);
assertEquals("incorrect sorted-set DocValues for doc " + doc, id, bytes.utf8ToString());
bytes = dv.lookupOrd(dv.nextOrd());
assertEquals("incorrect sorted-set DocValues for doc " + doc, Integer.valueOf(Integer.parseInt(id) + 1).toString(), bytes.utf8ToString());
assertEquals(SortedSetDocValues.NO_MORE_ORDS, dv.nextOrd());
}
}
}
public void testSortedNumericDocValuesField() throws Exception {
for(LeafReaderContext ctx : sortedReader.leaves()) {
LeafReader reader = ctx.reader();
SortedNumericDocValues dv = reader.getSortedNumericDocValues(SORTED_NUMERIC_DV_FIELD);
int maxDoc = reader.maxDoc();
for (int doc = 0; doc < maxDoc; doc++) {
dv.setDocument(doc);
assertEquals(2, dv.count());
int id = Integer.parseInt(reader.document(doc).get(ID_FIELD));
assertEquals("incorrect sorted-numeric DocValues for doc " + doc, id, dv.valueAt(0));
assertEquals("incorrect sorted-numeric DocValues for doc " + doc, id + 1, dv.valueAt(1));
}
}
}
public void testTermVectors() throws Exception {
for(LeafReaderContext ctx : sortedReader.leaves()) {
LeafReader reader = ctx.reader();
int maxDoc = reader.maxDoc();
for (int doc = 0; doc < maxDoc; doc++) {
Terms terms = reader.getTermVector(doc, TERM_VECTORS_FIELD);
assertNotNull("term vectors not found for doc " + doc + " field [" + TERM_VECTORS_FIELD + "]", terms);
String id = reader.document(doc).get(ID_FIELD);
assertEquals("incorrect term vector for doc " + doc, id, terms.iterator().next().utf8ToString());
}
}
}
public void testPoints() throws Exception {
for(LeafReaderContext ctx : sortedReader.leaves()) {
final LeafReader reader = ctx.reader();
PointValues values = reader.getPointValues();
values.intersect(DIMENSIONAL_FIELD,
new IntersectVisitor() {
@Override
public void visit(int docID) {
throw new IllegalStateException();
}
@Override
public void visit(int docID, byte[] packedValues) throws IOException {
int id = Integer.parseInt(reader.document(docID).get(ID_FIELD));
assertEquals(id, NumericUtils.sortableBytesToInt(packedValues, 0));
}
@Override
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
return Relation.CELL_CROSSES_QUERY;
}
});
}
}
}

View File

@ -69,6 +69,8 @@ import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.PhraseQuery;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.BaseDirectoryWrapper;
@ -2759,5 +2761,6 @@ public class TestIndexWriter extends LuceneTestCase {
w.close();
dir.close();
}
}

View File

@ -51,7 +51,7 @@ public class TestSegmentInfos extends LuceneTestCase {
SegmentInfos sis = new SegmentInfos();
SegmentInfo info = new SegmentInfo(dir, Version.LUCENE_5_0_0, "_0", 1, false, Codec.getDefault(),
Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap());
Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
SegmentCommitInfo commitInfo = new SegmentCommitInfo(info, 0, -1, -1, -1);
@ -72,15 +72,25 @@ public class TestSegmentInfos extends LuceneTestCase {
Codec codec = Codec.getDefault();
SegmentInfos sis = new SegmentInfos();
<<<<<<< HEAD
SegmentInfo info = new SegmentInfo(dir, Version.LUCENE_5_0_0, "_0", 1, false, Codec.getDefault(),
Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap());
=======
SegmentInfo info = new SegmentInfo(dir, Version.LUCENE_6_0_0, "_0", 1, false, Codec.getDefault(),
Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
>>>>>>> 54fa7df... LUCENE-6766: initial patch
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
SegmentCommitInfo commitInfo = new SegmentCommitInfo(info, 0, -1, -1, -1);
sis.add(commitInfo);
<<<<<<< HEAD
info = new SegmentInfo(dir, Version.LUCENE_5_1_0, "_1", 1, false, Codec.getDefault(),
Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap());
=======
info = new SegmentInfo(dir, Version.LUCENE_6_0_0, "_1", 1, false, Codec.getDefault(),
Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
>>>>>>> 54fa7df... LUCENE-6766: initial patch
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
commitInfo = new SegmentCommitInfo(info, 0, -1, -1, -1);

View File

@ -35,6 +35,7 @@ import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.Version;
import org.apache.lucene.util.packed.PackedLongValues;
public class TestSegmentMerger extends LuceneTestCase {
//The variables for the new merged segment
@ -83,7 +84,7 @@ public class TestSegmentMerger extends LuceneTestCase {
public void testMerge() throws IOException {
final Codec codec = Codec.getDefault();
final SegmentInfo si = new SegmentInfo(mergedDir, Version.LATEST, mergedSegment, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
final SegmentInfo si = new SegmentInfo(mergedDir, Version.LATEST, mergedSegment, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
SegmentMerger merger = new SegmentMerger(Arrays.<CodecReader>asList(reader1, reader2),
si, InfoStream.getDefault(), mergedDir,
@ -144,22 +145,9 @@ public class TestSegmentMerger extends LuceneTestCase {
mergedReader.close();
}
private static boolean equals(MergeState.DocMap map1, MergeState.DocMap map2) {
if (map1.maxDoc() != map2.maxDoc()) {
return false;
}
for (int i = 0; i < map1.maxDoc(); ++i) {
if (map1.get(i) != map2.get(i)) {
return false;
}
}
return true;
}
public void testBuildDocMap() {
final int maxDoc = TestUtil.nextInt(random(), 1, 128);
final int numDocs = TestUtil.nextInt(random(), 0, maxDoc);
final int numDeletedDocs = maxDoc - numDocs;
final FixedBitSet liveDocs = new FixedBitSet(maxDoc);
for (int i = 0; i < numDocs; ++i) {
while (true) {
@ -171,15 +159,11 @@ public class TestSegmentMerger extends LuceneTestCase {
}
}
final MergeState.DocMap docMap = MergeState.DocMap.build(maxDoc, liveDocs);
final PackedLongValues docMap = MergeState.removeDeletes(maxDoc, liveDocs);
assertEquals(maxDoc, docMap.maxDoc());
assertEquals(numDocs, docMap.numDocs());
assertEquals(numDeletedDocs, docMap.numDeletedDocs());
// assert the mapping is compact
for (int i = 0, del = 0; i < maxDoc; ++i) {
if (!liveDocs.get(i)) {
assertEquals(-1, docMap.get(i));
if (liveDocs.get(i) == false) {
++del;
} else {
assertEquals(i - del, docMap.get(i));

View File

@ -25,6 +25,7 @@ import java.util.BitSet;
import java.util.List;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.PointValues.IntersectVisitor;
import org.apache.lucene.index.PointValues.Relation;
import org.apache.lucene.store.CorruptingIndexOutput;
@ -554,7 +555,7 @@ public class TestBKD extends LuceneTestCase {
}
List<Long> toMerge = null;
List<Integer> docIDBases = null;
List<MergeState.DocMap> docMaps = null;
int seg = 0;
BKDWriter w = new BKDWriter(numValues, dir, "_" + seg, numDims, numBytesPerDim, maxPointsInLeafNode, maxMB, docValues.length, false);
@ -601,9 +602,15 @@ public class TestBKD extends LuceneTestCase {
if (useMerge && segCount == valuesInThisSeg) {
if (toMerge == null) {
toMerge = new ArrayList<>();
docIDBases = new ArrayList<>();
docMaps = new ArrayList<>();
}
docIDBases.add(lastDocIDBase);
final int curDocIDBase = lastDocIDBase;
docMaps.add(new MergeState.DocMap() {
@Override
public int get(int docID) {
return curDocIDBase + docID;
}
});
toMerge.add(w.finish(out));
valuesInThisSeg = TestUtil.nextInt(random(), numValues/10, numValues/2);
segCount = 0;
@ -620,8 +627,14 @@ public class TestBKD extends LuceneTestCase {
if (toMerge != null) {
if (segCount > 0) {
docIDBases.add(lastDocIDBase);
toMerge.add(w.finish(out));
final int curDocIDBase = lastDocIDBase;
docMaps.add(new MergeState.DocMap() {
@Override
public int get(int docID) {
return curDocIDBase + docID;
}
});
}
out.close();
in = dir.openInput("bkd", IOContext.DEFAULT);
@ -633,7 +646,7 @@ public class TestBKD extends LuceneTestCase {
readers.add(new BKDReader(in));
}
out = dir.createOutput("bkd2", IOContext.DEFAULT);
indexFP = w.merge(out, null, readers, docIDBases);
indexFP = w.merge(out, docMaps, readers);
out.close();
in.close();
in = dir.openInput("bkd2", IOContext.DEFAULT);

View File

@ -140,7 +140,7 @@ public class IndexSplitter {
SegmentInfo info = infoPerCommit.info;
// Same info just changing the dir:
SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.name, info.maxDoc(),
info.getUseCompoundFile(), info.getCodec(), info.getDiagnostics(), info.getId(), new HashMap<>());
info.getUseCompoundFile(), info.getCodec(), info.getDiagnostics(), info.getId(), new HashMap<>(), null);
destInfos.add(new SegmentCommitInfo(newInfo, infoPerCommit.getDelCount(),
infoPerCommit.getDelGen(), infoPerCommit.getFieldInfosGen(),
infoPerCommit.getDocValuesGen()));

View File

@ -24,6 +24,7 @@ import java.util.Map;
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.search.Sort;
import org.apache.lucene.util.Bits;
/**
@ -70,6 +71,11 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
if (getFieldInfos().hasPointValues()) {
throw new IllegalArgumentException("cannot wrap points");
}
for(LeafReaderContext context : reader.leaves()) {
if (context.reader().getIndexSort() != null) {
throw new IllegalArgumentException("cannot use index sort");
}
}
fields = MultiFields.getFields(in);
in.registerParentReader(this);
this.merging = merging;
@ -275,4 +281,9 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
ctx.reader().checkIntegrity();
}
}
@Override
public Sort getIndexSort() {
return null;
}
}

View File

@ -1,264 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.MergeTrigger;
import org.apache.lucene.index.MultiReader;
import org.apache.lucene.index.SegmentCommitInfo;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.index.SlowCompositeReaderWrapper;
import org.apache.lucene.search.Sort;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedLongValues;
/** A {@link MergePolicy} that reorders documents according to a {@link Sort}
* before merging them. As a consequence, all segments resulting from a merge
* will be sorted while segments resulting from a flush will be in the order
* in which documents have been added.
* <p><b>NOTE</b>: Never use this policy if you rely on
* {@link IndexWriter#addDocuments(Iterable) IndexWriter.addDocuments}
* to have sequentially-assigned doc IDs, this policy will scatter doc IDs.
* <p><b>NOTE</b>: This policy should only be used with idempotent {@code Sort}s
* so that the order of segments is predictable. For example, using
* {@link Sort#INDEXORDER} in reverse (which is not idempotent) will make
* the order of documents in a segment depend on the number of times the segment
* has been merged.
* @lucene.experimental */
public final class SortingMergePolicy extends MergePolicyWrapper {
/**
* Put in the {@link SegmentInfo#getDiagnostics() diagnostics} to denote that
* this segment is sorted.
*/
public static final String SORTER_ID_PROP = "sorter";
class SortingOneMerge extends OneMerge {
List<CodecReader> unsortedReaders;
Sorter.DocMap docMap;
LeafReader sortedView;
final InfoStream infoStream;
SortingOneMerge(List<SegmentCommitInfo> segments, InfoStream infoStream) {
super(segments);
this.infoStream = infoStream;
}
@Override
public List<CodecReader> getMergeReaders() throws IOException {
if (unsortedReaders == null) {
unsortedReaders = super.getMergeReaders();
if (infoStream.isEnabled("SMP")) {
infoStream.message("SMP", "sorting " + unsortedReaders);
for (LeafReader leaf : unsortedReaders) {
String sortDescription = getSortDescription(leaf);
if (sortDescription == null) {
sortDescription = "not sorted";
}
infoStream.message("SMP", "seg=" + leaf + " " + sortDescription);
}
}
// wrap readers, to be optimal for merge;
List<LeafReader> wrapped = new ArrayList<>(unsortedReaders.size());
for (LeafReader leaf : unsortedReaders) {
if (leaf instanceof SegmentReader) {
leaf = new MergeReaderWrapper((SegmentReader)leaf);
}
wrapped.add(leaf);
}
final LeafReader atomicView;
if (wrapped.size() == 1) {
atomicView = wrapped.get(0);
} else {
final CompositeReader multiReader = new MultiReader(wrapped.toArray(new LeafReader[wrapped.size()]));
atomicView = new SlowCompositeReaderWrapper(multiReader, true);
}
docMap = sorter.sort(atomicView);
sortedView = SortingLeafReader.wrap(atomicView, docMap);
}
// a null doc map means that the readers are already sorted
if (docMap == null) {
if (infoStream.isEnabled("SMP")) {
infoStream.message("SMP", "readers already sorted, omitting sort");
}
return unsortedReaders;
} else {
if (infoStream.isEnabled("SMP")) {
infoStream.message("SMP", "sorting readers by " + sort);
}
return Collections.singletonList(SlowCodecReaderWrapper.wrap(sortedView));
}
}
@Override
public void setMergeInfo(SegmentCommitInfo info) {
Map<String,String> diagnostics = info.info.getDiagnostics();
diagnostics.put(SORTER_ID_PROP, sorter.getID());
super.setMergeInfo(info);
}
private PackedLongValues getDeletes(List<CodecReader> readers) {
PackedLongValues.Builder deletes = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
int deleteCount = 0;
for (LeafReader reader : readers) {
final int maxDoc = reader.maxDoc();
final Bits liveDocs = reader.getLiveDocs();
for (int i = 0; i < maxDoc; ++i) {
if (liveDocs != null && !liveDocs.get(i)) {
++deleteCount;
} else {
deletes.add(deleteCount);
}
}
}
return deletes.build();
}
@Override
public MergePolicy.DocMap getDocMap(final MergeState mergeState) {
if (unsortedReaders == null) {
throw new IllegalStateException();
}
if (docMap == null) {
return super.getDocMap(mergeState);
}
assert mergeState.docMaps.length == 1; // we returned a singleton reader
final PackedLongValues deletes = getDeletes(unsortedReaders);
return new MergePolicy.DocMap() {
@Override
public int map(int old) {
final int oldWithDeletes = old + (int) deletes.get(old);
final int newWithDeletes = docMap.oldToNew(oldWithDeletes);
return mergeState.docMaps[0].get(newWithDeletes);
}
};
}
@Override
public String toString() {
return "SortingMergePolicy.SortingOneMerge(segments=" + segString() + " sort=" + sort + ")";
}
}
class SortingMergeSpecification extends MergeSpecification {
final InfoStream infoStream;
SortingMergeSpecification(InfoStream infoStream) {
this.infoStream = infoStream;
}
@Override
public void add(OneMerge merge) {
super.add(new SortingOneMerge(merge.segments, infoStream));
}
@Override
public String segString(Directory dir) {
return "SortingMergeSpec(" + super.segString(dir) + ", sorter=" + sorter + ")";
}
}
/** Returns {@code true} if the given {@code reader} is sorted by the
* {@code sort} given. Typically the given {@code sort} would be the
* {@link SortingMergePolicy#getSort()} order of a {@link SortingMergePolicy}. */
public static boolean isSorted(LeafReader reader, Sort sort) {
String description = getSortDescription(reader);
if (description != null && description.equals(sort.toString())) {
return true;
}
return false;
}
private static String getSortDescription(LeafReader reader) {
if (reader instanceof SegmentReader) {
final SegmentReader segReader = (SegmentReader) reader;
final Map<String, String> diagnostics = segReader.getSegmentInfo().info.getDiagnostics();
if (diagnostics != null) {
return diagnostics.get(SORTER_ID_PROP);
}
} else if (reader instanceof FilterLeafReader) {
return getSortDescription(FilterLeafReader.unwrap(reader));
}
return null;
}
private MergeSpecification sortedMergeSpecification(MergeSpecification specification, InfoStream infoStream) {
if (specification == null) {
return null;
}
MergeSpecification sortingSpec = new SortingMergeSpecification(infoStream);
for (OneMerge merge : specification.merges) {
sortingSpec.add(merge);
}
return sortingSpec;
}
final Sorter sorter;
final Sort sort;
/** Create a new {@code MergePolicy} that sorts documents with the given {@code sort}. */
public SortingMergePolicy(MergePolicy in, Sort sort) {
super(in);
this.sorter = new Sorter(sort);
this.sort = sort;
}
/** Return the {@link Sort} order that is used to sort segments when merging. */
public Sort getSort() {
return sort;
}
@Override
public MergeSpecification findMerges(MergeTrigger mergeTrigger,
SegmentInfos segmentInfos, IndexWriter writer) throws IOException {
return sortedMergeSpecification(in.findMerges(mergeTrigger, segmentInfos, writer), writer.infoStream);
}
@Override
public MergeSpecification findForcedMerges(SegmentInfos segmentInfos,
int maxSegmentCount, Map<SegmentCommitInfo,Boolean> segmentsToMerge, IndexWriter writer)
throws IOException {
return sortedMergeSpecification(in.findForcedMerges(segmentInfos, maxSegmentCount, segmentsToMerge, writer), writer.infoStream);
}
@Override
public MergeSpecification findForcedDeletesMerges(SegmentInfos segmentInfos, IndexWriter writer)
throws IOException {
return sortedMergeSpecification(in.findForcedDeletesMerges(segmentInfos, writer), writer.infoStream);
}
@Override
public String toString() {
return "SortingMergePolicy(" + in + ", sorter=" + sorter + ")";
}
}

View File

@ -20,13 +20,14 @@ import java.io.IOException;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.ReaderUtil;
import org.apache.lucene.index.SortingMergePolicy;
import org.apache.lucene.util.BitSet;
// nocommit what to do here?
/**
* Helper class to sort readers that contain blocks of documents.
* <p>
* Note that this class is intended to used with {@link SortingMergePolicy},
* Note that this class is intended to used with index sorting,
* and for other purposes has some limitations:
* <ul>
* <li>Cannot yet be used with {@link IndexSearcher#searchAfter(ScoreDoc, Query, int, Sort) IndexSearcher.searchAfter}

View File

@ -20,14 +20,14 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.SortingMergePolicy;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.CollectionTerminatedException;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.FilterLeafCollector;
import org.apache.lucene.search.FilterCollector;
import org.apache.lucene.search.FilterLeafCollector;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.TopDocsCollector;
import org.apache.lucene.search.TotalHitCountCollector;
@ -39,8 +39,7 @@ import org.apache.lucene.search.TotalHitCountCollector;
*
* <p>
* <b>NOTE:</b> the {@code Collector} detects segments sorted according to a
* {@link SortingMergePolicy}'s {@link Sort} and so it's best used in conjunction
* with a {@link SortingMergePolicy}. Also,it collects up to a specified
* an {@link IndexWriterConfig#setIndexSort}. Also, it collects up to a specified
* {@code numDocsToCollect} from each segment, and therefore is mostly suitable
* for use in conjunction with collectors such as {@link TopDocsCollector}, and
* not e.g. {@link TotalHitCountCollector}.
@ -48,24 +47,14 @@ import org.apache.lucene.search.TotalHitCountCollector;
* <b>NOTE</b>: If you wrap a {@code TopDocsCollector} that sorts in the same
* order as the index order, the returned {@link TopDocsCollector#topDocs() TopDocs}
* will be correct. However the total of {@link TopDocsCollector#getTotalHits()
* hit count} will be underestimated since not all matching documents will have
* hit count} will be vastly underestimated since not all matching documents will have
* been collected.
* <p>
* <b>NOTE</b>: This {@code Collector} uses {@link Sort#toString()} to detect
* whether a segment was sorted with the same {@code Sort}. This has
* two implications:
* <ul>
* <li>if a custom comparator is not implemented correctly and returns
* different identifiers for equivalent instances, this collector will not
* detect sorted segments,</li>
* <li>if you suddenly change the {@link IndexWriter}'s
* {@code SortingMergePolicy} to sort according to another criterion and if both
* the old and the new {@code Sort}s have the same identifier, this
* {@code Collector} will incorrectly detect sorted segments.</li>
* </ul>
*
* @lucene.experimental
*/
// nocommit move to core too
public class EarlyTerminatingSortingCollector extends FilterCollector {
/** Returns whether collection can be early-terminated if it sorts with the
@ -85,7 +74,6 @@ public class EarlyTerminatingSortingCollector extends FilterCollector {
protected final Sort sort;
/** Number of documents to collect in each segment */
protected final int numDocsToCollect;
private final Sort mergePolicySort;
private final AtomicBoolean terminatedEarly = new AtomicBoolean(false);
/**
@ -99,27 +87,26 @@ public class EarlyTerminatingSortingCollector extends FilterCollector {
* the number of documents to collect on each segment. When wrapping
* a {@link TopDocsCollector}, this number should be the number of
* hits.
* @param mergePolicySort
* the sort your {@link SortingMergePolicy} uses
* @throws IllegalArgumentException if the sort order doesn't allow for early
* termination with the given merge policy.
*/
public EarlyTerminatingSortingCollector(Collector in, Sort sort, int numDocsToCollect, Sort mergePolicySort) {
public EarlyTerminatingSortingCollector(Collector in, Sort sort, int numDocsToCollect) {
super(in);
if (numDocsToCollect <= 0) {
throw new IllegalArgumentException("numDocsToCollect must always be > 0, got " + numDocsToCollect);
}
if (canEarlyTerminate(sort, mergePolicySort) == false) {
throw new IllegalStateException("Cannot early terminate with sort order " + sort + " if segments are sorted with " + mergePolicySort);
}
this.sort = sort;
this.numDocsToCollect = numDocsToCollect;
this.mergePolicySort = mergePolicySort;
}
@Override
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
if (SortingMergePolicy.isSorted(context.reader(), mergePolicySort)) {
Sort segmentSort = context.reader().getIndexSort();
if (segmentSort != null && canEarlyTerminate(sort, segmentSort) == false) {
throw new IllegalStateException("Cannot early terminate with sort order " + sort + " if segments are sorted with " + segmentSort);
}
if (segmentSort != null) {
// segment is sorted, can early-terminate
return new FilterLeafCollector(super.getLeafCollector(context)) {
private int numCollected;
@ -142,5 +129,4 @@ public class EarlyTerminatingSortingCollector extends FilterCollector {
public boolean terminatedEarly() {
return terminatedEarly.get();
}
}

View File

@ -1,89 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.SlowCompositeReaderWrapper;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.TestUtil;
import org.junit.BeforeClass;
public class IndexSortingTest extends SorterTestBase {
private static final Sort[] SORT = new Sort[] {
new Sort(new SortField(NUMERIC_DV_FIELD, SortField.Type.LONG)),
new Sort(new SortField(null, SortField.Type.DOC, true))
};
@BeforeClass
public static void beforeClassSorterUtilTest() throws Exception {
// NOTE: index was created by by super's @BeforeClass
// only read the values of the undeleted documents, since after addIndexes,
// the deleted ones will be dropped from the index.
Bits liveDocs = unsortedReader.getLiveDocs();
List<Integer> values = new ArrayList<>();
for (int i = 0; i < unsortedReader.maxDoc(); i++) {
if (liveDocs == null || liveDocs.get(i)) {
values.add(Integer.valueOf(unsortedReader.document(i).get(ID_FIELD)));
}
}
int idx = random().nextInt(SORT.length);
Sort sorter = SORT[idx];
if (idx == 1) { // reverse doc sort
Collections.reverse(values);
} else {
Collections.sort(values);
if (random().nextBoolean()) {
sorter = new Sort(new SortField(NUMERIC_DV_FIELD, SortField.Type.LONG, true)); // descending
Collections.reverse(values);
}
}
sortedValues = values.toArray(new Integer[values.size()]);
if (VERBOSE) {
System.out.println("sortedValues: " + sortedValues);
System.out.println("Sorter: " + sorter);
}
Directory target = newDirectory();
IndexWriter writer = new IndexWriter(target, newIndexWriterConfig(null));
LeafReader reader = SortingLeafReader.wrap(unsortedReader, sorter);
writer.addIndexes(SlowCodecReaderWrapper.wrap(reader));
writer.close();
// NOTE: also closes unsortedReader
reader.close();
dir.close();
// CheckIndex the target directory
dir = target;
TestUtil.checkIndex(dir);
// set reader for tests
sortedReader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
assertFalse("index should not have deletions", sortedReader.hasDeletions());
}
}

View File

@ -1,405 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.BinaryPoint;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.PointValues.IntersectVisitor;
import org.apache.lucene.index.PointValues.Relation;
import org.apache.lucene.index.SortingLeafReader.SortingDocsEnum;
import org.apache.lucene.index.TermsEnum.SeekStatus;
import org.apache.lucene.search.CollectionStatistics;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.TermStatistics;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.NumericUtils;
import org.apache.lucene.util.TestUtil;
import org.junit.AfterClass;
import org.junit.BeforeClass;
public abstract class SorterTestBase extends LuceneTestCase {
static final class NormsSimilarity extends Similarity {
private final Similarity in;
public NormsSimilarity(Similarity in) {
this.in = in;
}
@Override
public long computeNorm(FieldInvertState state) {
if (state.getName().equals(NORMS_FIELD)) {
return Float.floatToIntBits(state.getBoost());
} else {
return in.computeNorm(state);
}
}
@Override
public SimWeight computeWeight(CollectionStatistics collectionStats, TermStatistics... termStats) {
return in.computeWeight(collectionStats, termStats);
}
@Override
public SimScorer simScorer(SimWeight weight, LeafReaderContext context) throws IOException {
return in.simScorer(weight, context);
}
}
static final class PositionsTokenStream extends TokenStream {
private final CharTermAttribute term;
private final PayloadAttribute payload;
private final OffsetAttribute offset;
private int pos, off;
public PositionsTokenStream() {
term = addAttribute(CharTermAttribute.class);
payload = addAttribute(PayloadAttribute.class);
offset = addAttribute(OffsetAttribute.class);
}
@Override
public boolean incrementToken() throws IOException {
if (pos == 0) {
return false;
}
clearAttributes();
term.append(DOC_POSITIONS_TERM);
payload.setPayload(new BytesRef(Integer.toString(pos)));
offset.setOffset(off, off);
--pos;
++off;
return true;
}
void setId(int id) {
pos = id / 10 + 1;
off = 0;
}
}
protected static final String ID_FIELD = "id";
protected static final String DOCS_ENUM_FIELD = "docs";
protected static final String DOCS_ENUM_TERM = "$all$";
protected static final String DOC_POSITIONS_FIELD = "positions";
protected static final String DOC_POSITIONS_TERM = "$all$";
protected static final String NUMERIC_DV_FIELD = "numeric";
protected static final String SORTED_NUMERIC_DV_FIELD = "sorted_numeric";
protected static final String NORMS_FIELD = "norm";
protected static final String BINARY_DV_FIELD = "binary";
protected static final String SORTED_DV_FIELD = "sorted";
protected static final String SORTED_SET_DV_FIELD = "sorted_set";
protected static final String TERM_VECTORS_FIELD = "term_vectors";
protected static final String DIMENSIONAL_FIELD = "numeric1d";
private static final FieldType TERM_VECTORS_TYPE = new FieldType(TextField.TYPE_NOT_STORED);
static {
TERM_VECTORS_TYPE.setStoreTermVectors(true);
TERM_VECTORS_TYPE.freeze();
}
private static final FieldType POSITIONS_TYPE = new FieldType(TextField.TYPE_NOT_STORED);
static {
POSITIONS_TYPE.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
POSITIONS_TYPE.freeze();
}
protected static Directory dir;
protected static LeafReader unsortedReader;
protected static LeafReader sortedReader;
protected static Integer[] sortedValues;
private static Document doc(final int id, PositionsTokenStream positions) {
final Document doc = new Document();
doc.add(new StringField(ID_FIELD, Integer.toString(id), Store.YES));
doc.add(new StringField(DOCS_ENUM_FIELD, DOCS_ENUM_TERM, Store.NO));
positions.setId(id);
doc.add(new Field(DOC_POSITIONS_FIELD, positions, POSITIONS_TYPE));
doc.add(new NumericDocValuesField(NUMERIC_DV_FIELD, id));
TextField norms = new TextField(NORMS_FIELD, Integer.toString(id), Store.NO);
norms.setBoost(Float.intBitsToFloat(id));
doc.add(norms);
doc.add(new BinaryDocValuesField(BINARY_DV_FIELD, new BytesRef(Integer.toString(id))));
doc.add(new SortedDocValuesField(SORTED_DV_FIELD, new BytesRef(Integer.toString(id))));
doc.add(new SortedSetDocValuesField(SORTED_SET_DV_FIELD, new BytesRef(Integer.toString(id))));
doc.add(new SortedSetDocValuesField(SORTED_SET_DV_FIELD, new BytesRef(Integer.toString(id + 1))));
doc.add(new SortedNumericDocValuesField(SORTED_NUMERIC_DV_FIELD, id));
doc.add(new SortedNumericDocValuesField(SORTED_NUMERIC_DV_FIELD, id + 1));
doc.add(new Field(TERM_VECTORS_FIELD, Integer.toString(id), TERM_VECTORS_TYPE));
byte[] bytes = new byte[4];
NumericUtils.intToSortableBytes(id, bytes, 0);
// TODO: index time sorting doesn't yet support points
//doc.add(new BinaryPoint(DIMENSIONAL_FIELD, bytes));
return doc;
}
/** Creates an unsorted index; subclasses then sort this index and open sortedReader. */
private static void createIndex(Directory dir, int numDocs, Random random) throws IOException {
List<Integer> ids = new ArrayList<>();
for (int i = 0; i < numDocs; i++) {
ids.add(Integer.valueOf(i * 10));
}
// shuffle them for indexing
Collections.shuffle(ids, random);
if (VERBOSE) {
System.out.println("Shuffled IDs for indexing: " + Arrays.toString(ids.toArray()));
}
PositionsTokenStream positions = new PositionsTokenStream();
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random));
conf.setMaxBufferedDocs(4); // create some segments
conf.setSimilarity(new NormsSimilarity(conf.getSimilarity())); // for testing norms field
RandomIndexWriter writer = new RandomIndexWriter(random, dir, conf);
writer.setDoRandomForceMerge(false);
for (int id : ids) {
writer.addDocument(doc(id, positions));
}
// delete some documents
writer.commit();
for (Integer id : ids) {
if (random.nextDouble() < 0.2) {
if (VERBOSE) {
System.out.println("delete doc_id " + id);
}
writer.deleteDocuments(new Term(ID_FIELD, id.toString()));
}
}
writer.close();
}
@BeforeClass
public static void beforeClassSorterTestBase() throws Exception {
dir = newDirectory();
int numDocs = atLeast(20);
createIndex(dir, numDocs, random());
unsortedReader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
}
@AfterClass
public static void afterClassSorterTestBase() throws Exception {
unsortedReader.close();
sortedReader.close();
dir.close();
unsortedReader = sortedReader = null;
dir = null;
}
public void testBinaryDocValuesField() throws Exception {
BinaryDocValues dv = sortedReader.getBinaryDocValues(BINARY_DV_FIELD);
for (int i = 0; i < sortedReader.maxDoc(); i++) {
final BytesRef bytes = dv.get(i);
assertEquals("incorrect binary DocValues for doc " + i, sortedValues[i].toString(), bytes.utf8ToString());
}
}
public void testDocsAndPositionsEnum() throws Exception {
TermsEnum termsEnum = sortedReader.terms(DOC_POSITIONS_FIELD).iterator();
assertEquals(SeekStatus.FOUND, termsEnum.seekCeil(new BytesRef(DOC_POSITIONS_TERM)));
PostingsEnum sortedPositions = termsEnum.postings(null, PostingsEnum.ALL);
int doc;
// test nextDoc()
while ((doc = sortedPositions.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
int freq = sortedPositions.freq();
assertEquals("incorrect freq for doc=" + doc, sortedValues[doc].intValue() / 10 + 1, freq);
for (int i = 0; i < freq; i++) {
assertEquals("incorrect position for doc=" + doc, i, sortedPositions.nextPosition());
assertEquals("incorrect startOffset for doc=" + doc, i, sortedPositions.startOffset());
assertEquals("incorrect endOffset for doc=" + doc, i, sortedPositions.endOffset());
assertEquals("incorrect payload for doc=" + doc, freq - i, Integer.parseInt(sortedPositions.getPayload().utf8ToString()));
}
}
// test advance()
final PostingsEnum reuse = sortedPositions;
sortedPositions = termsEnum.postings(reuse, PostingsEnum.ALL);
if (sortedPositions instanceof SortingDocsEnum) {
assertTrue(((SortingDocsEnum) sortedPositions).reused(reuse)); // make sure reuse worked
}
doc = 0;
while ((doc = sortedPositions.advance(doc + TestUtil.nextInt(random(), 1, 5))) != DocIdSetIterator.NO_MORE_DOCS) {
int freq = sortedPositions.freq();
assertEquals("incorrect freq for doc=" + doc, sortedValues[doc].intValue() / 10 + 1, freq);
for (int i = 0; i < freq; i++) {
assertEquals("incorrect position for doc=" + doc, i, sortedPositions.nextPosition());
assertEquals("incorrect startOffset for doc=" + doc, i, sortedPositions.startOffset());
assertEquals("incorrect endOffset for doc=" + doc, i, sortedPositions.endOffset());
assertEquals("incorrect payload for doc=" + doc, freq - i, Integer.parseInt(sortedPositions.getPayload().utf8ToString()));
}
}
}
Bits randomLiveDocs(int maxDoc) {
if (rarely()) {
if (random().nextBoolean()) {
return null;
} else {
return new Bits.MatchNoBits(maxDoc);
}
}
final FixedBitSet bits = new FixedBitSet(maxDoc);
final int bitsSet = TestUtil.nextInt(random(), 1, maxDoc - 1);
for (int i = 0; i < bitsSet; ++i) {
while (true) {
final int index = random().nextInt(maxDoc);
if (!bits.get(index)) {
bits.set(index);
break;
}
}
}
return bits;
}
public void testDocsEnum() throws Exception {
TermsEnum termsEnum = sortedReader.terms(DOCS_ENUM_FIELD).iterator();
assertEquals(SeekStatus.FOUND, termsEnum.seekCeil(new BytesRef(DOCS_ENUM_TERM)));
PostingsEnum docs = termsEnum.postings(null);
int doc;
while ((doc = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
assertEquals("incorrect value; doc " + doc, sortedValues[doc].intValue(), Integer.parseInt(sortedReader.document(doc).get(ID_FIELD)));
}
PostingsEnum reuse = docs;
docs = termsEnum.postings(reuse);
if (docs instanceof SortingDocsEnum) {
assertTrue(((SortingDocsEnum) docs).reused(reuse)); // make sure reuse worked
}
doc = -1;
while ((doc = docs.advance(doc + 1)) != DocIdSetIterator.NO_MORE_DOCS) {
assertEquals("incorrect value; doc " + doc, sortedValues[doc].intValue(), Integer.parseInt(sortedReader.document(doc).get(ID_FIELD)));
}
}
public void testNormValues() throws Exception {
NumericDocValues dv = sortedReader.getNormValues(NORMS_FIELD);
int maxDoc = sortedReader.maxDoc();
for (int i = 0; i < maxDoc; i++) {
assertEquals("incorrect norm value for doc " + i, sortedValues[i].intValue(), dv.get(i));
}
}
public void testNumericDocValuesField() throws Exception {
NumericDocValues dv = sortedReader.getNumericDocValues(NUMERIC_DV_FIELD);
int maxDoc = sortedReader.maxDoc();
for (int i = 0; i < maxDoc; i++) {
assertEquals("incorrect numeric DocValues for doc " + i, sortedValues[i].intValue(), dv.get(i));
}
}
public void testSortedDocValuesField() throws Exception {
SortedDocValues dv = sortedReader.getSortedDocValues(SORTED_DV_FIELD);
int maxDoc = sortedReader.maxDoc();
for (int i = 0; i < maxDoc; i++) {
final BytesRef bytes = dv.get(i);
assertEquals("incorrect sorted DocValues for doc " + i, sortedValues[i].toString(), bytes.utf8ToString());
}
}
public void testSortedSetDocValuesField() throws Exception {
SortedSetDocValues dv = sortedReader.getSortedSetDocValues(SORTED_SET_DV_FIELD);
int maxDoc = sortedReader.maxDoc();
for (int i = 0; i < maxDoc; i++) {
dv.setDocument(i);
BytesRef bytes = dv.lookupOrd(dv.nextOrd());
int value = sortedValues[i].intValue();
assertEquals("incorrect sorted-set DocValues for doc " + i, Integer.valueOf(value).toString(), bytes.utf8ToString());
bytes = dv.lookupOrd(dv.nextOrd());
assertEquals("incorrect sorted-set DocValues for doc " + i, Integer.valueOf(value + 1).toString(), bytes.utf8ToString());
assertEquals(SortedSetDocValues.NO_MORE_ORDS, dv.nextOrd());
}
}
public void testSortedNumericDocValuesField() throws Exception {
SortedNumericDocValues dv = sortedReader.getSortedNumericDocValues(SORTED_NUMERIC_DV_FIELD);
int maxDoc = sortedReader.maxDoc();
for (int i = 0; i < maxDoc; i++) {
dv.setDocument(i);
assertEquals(2, dv.count());
int value = sortedValues[i].intValue();
assertEquals("incorrect sorted-numeric DocValues for doc " + i, value, dv.valueAt(0));
assertEquals("incorrect sorted-numeric DocValues for doc " + i, value + 1, dv.valueAt(1));
}
}
public void testTermVectors() throws Exception {
int maxDoc = sortedReader.maxDoc();
for (int i = 0; i < maxDoc; i++) {
Terms terms = sortedReader.getTermVector(i, TERM_VECTORS_FIELD);
assertNotNull("term vectors not found for doc " + i + " field [" + TERM_VECTORS_FIELD + "]", terms);
assertEquals("incorrect term vector for doc " + i, sortedValues[i].toString(), terms.iterator().next().utf8ToString());
}
}
// TODO: index sorting doesn't yet support points
/*
public void testPoints() throws Exception {
PointValues values = sortedReader.getPointValues();
values.intersect(DIMENSIONAL_FIELD,
new IntersectVisitor() {
@Override
public void visit(int docID) {
throw new IllegalStateException();
}
@Override
public void visit(int docID, byte[] packedValues) {
assertEquals(sortedValues[docID].intValue(), NumericUtils.bytesToInt(packedValues, 0));
}
@Override
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
return Relation.CELL_CROSSES_QUERY;
}
});
}
*/
}

View File

@ -1,73 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.util.Arrays;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.TestUtil;
import org.junit.BeforeClass;
public class SortingLeafReaderTest extends SorterTestBase {
@BeforeClass
public static void beforeClassSortingLeafReaderTest() throws Exception {
// NOTE: index was created by by super's @BeforeClass
// sort the index by id (as integer, in NUMERIC_DV_FIELD)
Sort sort = new Sort(new SortField(NUMERIC_DV_FIELD, SortField.Type.INT));
final Sorter.DocMap docMap = new Sorter(sort).sort(unsortedReader);
// Sorter.compute also sorts the values
NumericDocValues dv = unsortedReader.getNumericDocValues(NUMERIC_DV_FIELD);
sortedValues = new Integer[unsortedReader.maxDoc()];
for (int i = 0; i < unsortedReader.maxDoc(); ++i) {
sortedValues[docMap.oldToNew(i)] = (int)dv.get(i);
}
if (VERBOSE) {
System.out.println("docMap: " + docMap);
System.out.println("sortedValues: " + Arrays.toString(sortedValues));
}
// sort the index by id (as integer, in NUMERIC_DV_FIELD)
sortedReader = SortingLeafReader.wrap(unsortedReader, sort);
if (VERBOSE) {
System.out.print("mapped-deleted-docs: ");
Bits mappedLiveDocs = sortedReader.getLiveDocs();
for (int i = 0; i < mappedLiveDocs.length(); i++) {
if (!mappedLiveDocs.get(i)) {
System.out.print(i + " ");
}
}
System.out.println();
}
TestUtil.checkReader(sortedReader);
}
public void testBadSort() throws Exception {
IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
SortingLeafReader.wrap(sortedReader, Sort.RELEVANCE);
});
assertEquals("Cannot sort an index with a Sort that refers to the relevance score", expected.getMessage());
}
}

View File

@ -1,201 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.index;
import java.io.IOException;
import java.lang.reflect.Method;
import java.lang.reflect.Modifier;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Random;
import java.util.Set;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LogMergePolicy;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.SlowCompositeReaderWrapper;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TieredMergePolicy;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
public class TestSortingMergePolicy extends BaseMergePolicyTestCase {
private List<String> terms;
private Directory dir1, dir2;
private Sort sort;
private boolean reversedSort;
private IndexReader reader;
private IndexReader sortedReader;
@Override
public void setUp() throws Exception {
super.setUp();
final Boolean reverse = (random().nextBoolean() ? null : new Boolean(random().nextBoolean()));
final SortField sort_field = (reverse == null
? new SortField("ndv", SortField.Type.LONG)
: new SortField("ndv", SortField.Type.LONG, reverse.booleanValue()));
sort = new Sort(sort_field);
reversedSort = (null != reverse && reverse.booleanValue());
createRandomIndexes();
}
private Document randomDocument() {
final Document doc = new Document();
doc.add(new NumericDocValuesField("ndv", random().nextLong()));
doc.add(new StringField("s", RandomPicks.randomFrom(random(), terms), Store.YES));
return doc;
}
public MergePolicy mergePolicy() {
return newSortingMergePolicy(sort);
}
public static SortingMergePolicy newSortingMergePolicy(Sort sort) {
// usually create a MP with a low merge factor so that many merges happen
MergePolicy mp;
int thingToDo = random().nextInt(3);
if (thingToDo == 0) {
TieredMergePolicy tmp = newTieredMergePolicy(random());
final int numSegs = TestUtil.nextInt(random(), 3, 5);
tmp.setSegmentsPerTier(numSegs);
tmp.setMaxMergeAtOnce(TestUtil.nextInt(random(), 2, numSegs));
mp = tmp;
} else if (thingToDo == 1) {
LogMergePolicy lmp = newLogMergePolicy(random());
lmp.setMergeFactor(TestUtil.nextInt(random(), 3, 5));
mp = lmp;
} else {
// just a regular random one from LTC (could be alcoholic etc)
mp = newMergePolicy();
}
// wrap it with a sorting mp
if (VERBOSE) {
System.out.println("TEST: return SortingMergePolicy(mp=" + mp + " sort=" + sort + ")");
}
return new SortingMergePolicy(mp, sort);
}
private void createRandomIndexes() throws IOException {
dir1 = newDirectory();
dir2 = newDirectory();
final int numDocs = atLeast(150);
final int numTerms = TestUtil.nextInt(random(), 1, numDocs / 5);
Set<String> randomTerms = new HashSet<>();
while (randomTerms.size() < numTerms) {
randomTerms.add(TestUtil.randomSimpleString(random()));
}
terms = new ArrayList<>(randomTerms);
final long seed = random().nextLong();
final IndexWriterConfig iwc1 = newIndexWriterConfig(new MockAnalyzer(new Random(seed)));
final IndexWriterConfig iwc2 = newIndexWriterConfig(new MockAnalyzer(new Random(seed)));
iwc2.setMergePolicy(mergePolicy());
final RandomIndexWriter iw1 = new RandomIndexWriter(new Random(seed), dir1, iwc1);
final RandomIndexWriter iw2 = new RandomIndexWriter(new Random(seed), dir2, iwc2);
for (int i = 0; i < numDocs; ++i) {
if (random().nextInt(5) == 0 && i != numDocs - 1) {
final String term = RandomPicks.randomFrom(random(), terms);
iw1.deleteDocuments(new Term("s", term));
iw2.deleteDocuments(new Term("s", term));
}
final Document doc = randomDocument();
iw1.addDocument(doc);
iw2.addDocument(doc);
if (random().nextInt(8) == 0) {
iw1.commit();
iw2.commit();
}
}
// Make sure we have something to merge
iw1.commit();
iw2.commit();
final Document doc = randomDocument();
// NOTE: don't use RIW.addDocument directly, since it sometimes commits
// which may trigger a merge, at which case forceMerge may not do anything.
// With field updates this is a problem, since the updates can go into the
// single segment in the index, and threefore the index won't be sorted.
// This hurts the assumption of the test later on, that the index is sorted
// by SortingMP.
iw1.w.addDocument(doc);
iw2.w.addDocument(doc);
// update NDV of docs belonging to one term (covers many documents)
final long value = random().nextLong();
final String term = RandomPicks.randomFrom(random(), terms);
iw1.w.updateNumericDocValue(new Term("s", term), "ndv", value);
iw2.w.updateNumericDocValue(new Term("s", term), "ndv", value);
iw1.forceMerge(1);
iw2.forceMerge(1);
iw1.close();
iw2.close();
reader = DirectoryReader.open(dir1);
sortedReader = DirectoryReader.open(dir2);
}
@Override
public void tearDown() throws Exception {
reader.close();
sortedReader.close();
dir1.close();
dir2.close();
super.tearDown();
}
private static void assertSorted(LeafReader reader, boolean reverse) throws IOException {
final NumericDocValues ndv = reader.getNumericDocValues("ndv");
for (int i = 1; i < reader.maxDoc(); ++i) {
final int lhs = (!reverse ? i-1 : i);
final int rhs = (!reverse ? i : i-1);
assertTrue("ndv(" + (i-1) + ")=" + ndv.get(i-1) + ",ndv(" + i + ")=" + ndv.get(i)+",reverse="+reverse, ndv.get(lhs) <= ndv.get(rhs));
}
}
public void testSortingMP() throws IOException {
final LeafReader sortedReader1 = SortingLeafReader.wrap(SlowCompositeReaderWrapper.wrap(reader), sort);
final LeafReader sortedReader2 = SlowCompositeReaderWrapper.wrap(sortedReader);
assertSorted(sortedReader1, reversedSort);
assertSorted(sortedReader2, reversedSort);
assertReaderEquals("", sortedReader1, sortedReader2);
}
public void testBadSort() throws Exception {
IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
new SortingMergePolicy(newMergePolicy(), Sort.RELEVANCE);
});
assertEquals("Cannot sort an index with a Sort that refers to the relevance score", expected.getMessage());
}
}

View File

@ -32,6 +32,7 @@ import org.apache.lucene.index.FieldInvertState;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.MultiDocValues;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.SlowCompositeReaderWrapper;
@ -367,8 +368,7 @@ public class TestDiversifiedTopDocsCollector extends LuceneTestCase {
reader = writer.getReader();
writer.close();
searcher = newSearcher(reader);
LeafReader ar = SlowCompositeReaderWrapper.wrap(reader);
artistDocValues = ar.getSortedDocValues("artist");
artistDocValues = MultiDocValues.getSortedValues(reader, "artist");
// All searches sort by song popularity
final Similarity base = searcher.getSimilarity(true);

View File

@ -25,23 +25,23 @@ import java.util.Random;
import java.util.Set;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.ExitableDirectoryReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.MockRandomMergePolicy;
import org.apache.lucene.index.QueryTimeout;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.SerialMergeScheduler;
import org.apache.lucene.index.SortingMergePolicy;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TestSortingMergePolicy;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.LeafCollector;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
@ -50,8 +50,8 @@ import org.apache.lucene.search.SortField;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TopFieldCollector;
import org.apache.lucene.store.Directory;
import org.apache.lucene.uninverting.UninvertingReader;
import org.apache.lucene.uninverting.UninvertingReader.Type;
import org.apache.lucene.uninverting.UninvertingReader;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
@ -62,18 +62,11 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
private int numDocs;
private List<String> terms;
private Directory dir;
private Sort sort;
private final Sort sort = new Sort(new SortField("ndv1", SortField.Type.LONG));
private RandomIndexWriter iw;
private IndexReader reader;
private SortingMergePolicy mergePolicy;
private final int forceMergeMaxSegmentCount = 5;
@Override
public void setUp() throws Exception {
super.setUp();
sort = new Sort(new SortField("ndv1", SortField.Type.LONG));
}
private Document randomDocument() {
final Document doc = new Document();
doc.add(new NumericDocValuesField("ndv1", random().nextInt(10)));
@ -93,9 +86,14 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
terms = new ArrayList<>(randomTerms);
final long seed = random().nextLong();
final IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(new Random(seed)));
if (iwc.getMergePolicy() instanceof MockRandomMergePolicy) {
// MockRandomMP randomly wraps the leaf readers which makes merging angry
iwc.setMergePolicy(newTieredMergePolicy());
}
iwc.setMergeScheduler(new SerialMergeScheduler()); // for reproducible tests
mergePolicy = TestSortingMergePolicy.newSortingMergePolicy(sort);
iwc.setMergePolicy(mergePolicy);
iwc.setIndexSort(sort);
// nocommit:
iwc.setCodec(Codec.forName("SimpleText"));
iw = new RandomIndexWriter(new Random(seed), dir, iwc);
iw.setDoRandomForceMerge(false); // don't do this, it may happen anyway with MockRandomMP
for (int i = 0; i < numDocs; ++i) {
@ -151,7 +149,7 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
query = new MatchAllDocsQuery();
}
searcher.search(query, collector1);
searcher.search(query, new EarlyTerminatingSortingCollector(collector2, sort, numHits, mergePolicy.getSort()));
searcher.search(query, new EarlyTerminatingSortingCollector(collector2, sort, numHits));
assertTrue(collector1.getTotalHits() >= collector2.getTotalHits());
assertTopDocsEquals(collector1.topDocs().scoreDocs, collector2.topDocs().scoreDocs);
}
@ -190,40 +188,16 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
}
public void testEarlyTerminationDifferentSorter() throws IOException {
createRandomIndex(false);
final int iters = atLeast(3);
for (int i = 0; i < iters; ++i) {
final IndexSearcher searcher = newSearcher(reader);
// test that the collector works correctly when the index was sorted by a
// different sorter than the one specified in the ctor.
final int numHits = TestUtil.nextInt(random(), 1, numDocs);
final Sort sort = new Sort(new SortField("ndv2", SortField.Type.LONG, false));
final boolean fillFields = random().nextBoolean();
final boolean trackDocScores = random().nextBoolean();
final boolean trackMaxScore = random().nextBoolean();
final TopFieldCollector collector1 = TopFieldCollector.create(sort, numHits, fillFields, trackDocScores, trackMaxScore);
final TopFieldCollector collector2 = TopFieldCollector.create(sort, numHits, fillFields, trackDocScores, trackMaxScore);
final Query query;
if (random().nextBoolean()) {
query = new TermQuery(new Term("s", RandomPicks.randomFrom(random(), terms)));
} else {
query = new MatchAllDocsQuery();
}
searcher.search(query, collector1);
Sort different = new Sort(new SortField("ndv2", SortField.Type.LONG));
createRandomIndex(true);
searcher.search(query, new EarlyTerminatingSortingCollector(collector2, different, numHits, different) {
@Override
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
final LeafCollector ret = super.getLeafCollector(context);
assertTrue("segment should not be recognized as sorted as different sorter was used", ret.getClass() == in.getLeafCollector(context).getClass());
return ret;
}
});
assertTrue(collector1.getTotalHits() >= collector2.getTotalHits());
assertTopDocsEquals(collector1.topDocs().scoreDocs, collector2.topDocs().scoreDocs);
}
Sort sort = new Sort(new SortField("ndv2", SortField.Type.LONG, false));
Collector c = new EarlyTerminatingSortingCollector(TopFieldCollector.create(sort, 10, true, true, true), sort, 10);
IndexSearcher searcher = newSearcher(reader);
Exception e = expectThrows(IllegalStateException.class,
() -> {
searcher.search(new MatchAllDocsQuery(), c);
});
assertEquals("Cannot early terminate with sort order <long: \"ndv2\"> if segments are sorted with <long: \"ndv1\">", e.getMessage());
closeIndex();
}
@ -289,7 +263,7 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase {
searcher.search(query, collector1);
final TestTerminatedEarlySimpleCollector collector2 = new TestTerminatedEarlySimpleCollector();
final EarlyTerminatingSortingCollector etsCollector = new EarlyTerminatingSortingCollector(collector2, sort, 1, mergePolicy.getSort());
final EarlyTerminatingSortingCollector etsCollector = new EarlyTerminatingSortingCollector(collector2, sort, 1);
searcher.search(query, etsCollector);
assertTrue("collector1="+collector1.collectedSomething()+" vs. collector2="+collector2.collectedSomething(), collector1.collectedSomething() == collector2.collectedSomething());

View File

@ -638,7 +638,7 @@ public abstract class BaseCompoundFormatTestCase extends BaseIndexFileFormatTest
/** Returns a new fake segment */
protected static SegmentInfo newSegmentInfo(Directory dir, String name) {
return new SegmentInfo(dir, Version.LATEST, name, 10000, false, Codec.getDefault(), Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
return new SegmentInfo(dir, Version.LATEST, name, 10000, false, Codec.getDefault(), Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
}
/** Creates a file of the specified size with random data. */

View File

@ -347,7 +347,7 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes
/** Returns a new fake segment */
protected static SegmentInfo newSegmentInfo(Directory dir, String name) {
return new SegmentInfo(dir, Version.LATEST, name, 10000, false, Codec.getDefault(), Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
return new SegmentInfo(dir, Version.LATEST, name, 10000, false, Codec.getDefault(), Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
}
@Override

View File

@ -303,7 +303,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
Directory dir = newFSDirectory(createTempDir("justSoYouGetSomeChannelErrors"));
Codec codec = getCodec();
SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, "_0", 1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, "_0", 1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
FieldInfo proto = oneDocReader.getFieldInfos().fieldInfo("field");
FieldInfo field = new FieldInfo(proto.name, proto.number, proto.hasVectors(), proto.omitsNorms(), proto.hasPayloads(),
proto.getIndexOptions(), proto.getDocValuesType(), proto.getDocValuesGen(), new HashMap<>(),

View File

@ -26,7 +26,8 @@ import java.util.Set;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.MockDirectoryWrapper;
@ -52,7 +53,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
Codec codec = getCodec();
byte id[] = StringHelper.randomId();
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
Collections.<String,String>emptyMap(), id, new HashMap<>());
Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
@ -66,7 +67,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
Codec codec = getCodec();
byte id[] = StringHelper.randomId();
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
Collections.<String,String>emptyMap(), id, new HashMap<>());
Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
Set<String> originalFiles = Collections.singleton("_123.a");
info.setFiles(originalFiles);
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
@ -95,7 +96,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
diagnostics.put("key1", "value1");
diagnostics.put("key2", "value2");
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
diagnostics, id, new HashMap<>());
diagnostics, id, new HashMap<>(), null);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
@ -118,7 +119,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
attributes.put("key1", "value1");
attributes.put("key2", "value2");
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
Collections.emptyMap(), id, attributes);
Collections.emptyMap(), id, attributes, null);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
@ -138,7 +139,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
Directory dir = newDirectory();
byte id[] = StringHelper.randomId();
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
Collections.<String,String>emptyMap(), id, new HashMap<>());
Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
@ -153,7 +154,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
Directory dir = newDirectory();
byte id[] = StringHelper.randomId();
SegmentInfo info = new SegmentInfo(dir, v, "_123", 1, false, codec,
Collections.<String,String>emptyMap(), id, new HashMap<>());
Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
@ -161,7 +162,51 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
dir.close();
}
}
/** Test sort */
public void testSort() throws IOException {
final int iters = atLeast(5);
for (int i = 0; i < iters; ++i) {
Sort sort;
if (i == 0) {
sort = null;
} else {
final int numSortFields = TestUtil.nextInt(random(), 1, 3);
SortField[] sortFields = new SortField[numSortFields];
for (int j = 0; j < numSortFields; ++j) {
sortFields[j] = new SortField(
TestUtil.randomSimpleString(random()),
random().nextBoolean() ? SortField.Type.LONG : SortField.Type.STRING,
random().nextBoolean());
if (random().nextBoolean()) {
switch (sortFields[j].getType()) {
case LONG:
sortFields[j].setMissingValue(random().nextLong());
break;
case STRING:
sortFields[j].setMissingValue(random().nextBoolean() ? SortField.STRING_FIRST : SortField.STRING_LAST);
break;
default:
fail();
}
}
}
sort = new Sort(sortFields);
}
Directory dir = newDirectory();
Codec codec = getCodec();
byte id[] = StringHelper.randomId();
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
Collections.<String,String>emptyMap(), id, new HashMap<>(), sort);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
assertEquals(sort, info2.getIndexSort());
dir.close();
}
}
/**
* Test segment infos write that hits exception immediately on open.
* make sure we get our exception back, no file handle leaks, etc.
@ -183,7 +228,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
Codec codec = getCodec();
byte id[] = StringHelper.randomId();
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
Collections.<String,String>emptyMap(), id, new HashMap<>());
Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
info.setFiles(Collections.<String>emptySet());
fail.setDoFail();
@ -216,7 +261,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
Codec codec = getCodec();
byte id[] = StringHelper.randomId();
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
Collections.<String,String>emptyMap(), id, new HashMap<>());
Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
info.setFiles(Collections.<String>emptySet());
fail.setDoFail();
@ -249,7 +294,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
Codec codec = getCodec();
byte id[] = StringHelper.randomId();
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
Collections.<String,String>emptyMap(), id, new HashMap<>());
Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
@ -283,7 +328,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
Codec codec = getCodec();
byte id[] = StringHelper.randomId();
SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec,
Collections.<String,String>emptyMap(), id, new HashMap<>());
Collections.<String,String>emptyMap(), id, new HashMap<>(), null);
info.setFiles(Collections.<String>emptySet());
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
@ -332,7 +377,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT
TestUtil.randomUnicodeString(random()));
}
SegmentInfo info = new SegmentInfo(dir, version, name, docCount, isCompoundFile, codec, diagnostics, id, attributes);
SegmentInfo info = new SegmentInfo(dir, version, name, docCount, isCompoundFile, codec, diagnostics, id, attributes, null);
info.setFiles(files);
codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
SegmentInfo info2 = codec.segmentInfoFormat().read(dir, name, id, IOContext.DEFAULT);

View File

@ -138,7 +138,6 @@ public class MockRandomMergePolicy extends MergePolicy {
static class MockRandomOneMerge extends OneMerge {
final Random r;
ArrayList<CodecReader> readers;
MockRandomOneMerge(List<SegmentCommitInfo> segments, long seed) {
super(segments);
@ -146,34 +145,30 @@ public class MockRandomMergePolicy extends MergePolicy {
}
@Override
public List<CodecReader> getMergeReaders() throws IOException {
if (readers == null) {
readers = new ArrayList<CodecReader>(super.getMergeReaders());
for (int i = 0; i < readers.size(); i++) {
// wrap it (e.g. prevent bulk merge etc)
// TODO: cut this over to FilterCodecReader api, we can explicitly
// enable/disable bulk merge for portions of the index we want.
int thingToDo = r.nextInt(7);
if (thingToDo == 0) {
// simple no-op FilterReader
if (LuceneTestCase.VERBOSE) {
System.out.println("NOTE: MockRandomMergePolicy now swaps in a SlowCodecReaderWrapper for merging reader=" + readers.get(i));
}
readers.set(i, SlowCodecReaderWrapper.wrap(new FilterLeafReader(readers.get(i)) {}));
} else if (thingToDo == 1) {
// renumber fields
// NOTE: currently this only "blocks" bulk merges just by
// being a FilterReader. But it might find bugs elsewhere,
// and maybe the situation can be improved in the future.
if (LuceneTestCase.VERBOSE) {
System.out.println("NOTE: MockRandomMergePolicy now swaps in a MismatchedLeafReader for merging reader=" + readers.get(i));
}
readers.set(i, SlowCodecReaderWrapper.wrap(new MismatchedLeafReader(readers.get(i), r)));
}
// otherwise, reader is unchanged
public CodecReader wrapForMerge(CodecReader reader) throws IOException {
// wrap it (e.g. prevent bulk merge etc)
// TODO: cut this over to FilterCodecReader api, we can explicitly
// enable/disable bulk merge for portions of the index we want.
int thingToDo = r.nextInt(7);
if (thingToDo == 0) {
// simple no-op FilterReader
if (LuceneTestCase.VERBOSE) {
System.out.println("NOTE: MockRandomMergePolicy now swaps in a SlowCodecReaderWrapper for merging reader=" + reader);
}
return SlowCodecReaderWrapper.wrap(new FilterLeafReader(reader) {});
} else if (thingToDo == 1) {
// renumber fields
// NOTE: currently this only "blocks" bulk merges just by
// being a FilterReader. But it might find bugs elsewhere,
// and maybe the situation can be improved in the future.
if (LuceneTestCase.VERBOSE) {
System.out.println("NOTE: MockRandomMergePolicy now swaps in a MismatchedLeafReader for merging reader=" + reader);
}
return SlowCodecReaderWrapper.wrap(new MismatchedLeafReader(reader, r));
} else {
// otherwise, reader is unchanged
return reader;
}
return readers;
}
}
}

View File

@ -611,7 +611,7 @@ public class RandomPostingsTester {
// maxAllowed = the "highest" we can index, but we will still
// randomly index at lower IndexOption
public FieldsProducer buildIndex(Codec codec, Directory dir, IndexOptions maxAllowed, boolean allowPayloads, boolean alwaysTestMax) throws IOException {
SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, "_0", maxDoc, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, "_0", maxDoc, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
int maxIndexOption = Arrays.asList(IndexOptions.values()).indexOf(maxAllowed);
if (LuceneTestCase.VERBOSE) {

View File

@ -294,6 +294,11 @@ public class QueryUtils {
@Override
protected void doClose() throws IOException {}
@Override
public Sort getIndexSort() {
return null;
}
};
}