mirror of https://github.com/apache/lucene.git
LUCENE-7579: sort segments at flush too
This commit is contained in:
parent
48ef96d573
commit
4ccb9fbd2b
|
@ -168,6 +168,11 @@ Optimizations
|
|||
|
||||
* LUCENE-7572: Doc values queries now cache their hash code. (Adrien Grand)
|
||||
|
||||
* LUCENE-7579: Segments are now also sorted during flush, and merging
|
||||
on a sorted index is substantially faster by using some of the same
|
||||
bulk merge optimizations that non-sorted merging uses (Jim Ferenczi
|
||||
via Mike McCandless)
|
||||
|
||||
Other
|
||||
|
||||
* LUCENE-7546: Fixed references to benchmark wikipedia data and the Jenkins line-docs file
|
||||
|
|
|
@ -18,13 +18,16 @@ package org.apache.lucene.codecs.compressing;
|
|||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.codecs.compressing.CompressingStoredFieldsReader.SerializedDocument;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.DocIDMerger;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
|
@ -44,6 +47,8 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
|
||||
|
||||
/**
|
||||
* {@link StoredFieldsWriter} impl for {@link CompressingStoredFieldsFormat}.
|
||||
* @lucene.experimental
|
||||
|
@ -487,16 +492,45 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
|
|||
|
||||
@Override
|
||||
public int merge(MergeState mergeState) throws IOException {
|
||||
if (mergeState.needsIndexSort) {
|
||||
// TODO: can we gain back some optos even if index is sorted? E.g. if sort results in large chunks of contiguous docs from one sub
|
||||
// being copied over...?
|
||||
return super.merge(mergeState);
|
||||
}
|
||||
|
||||
int docCount = 0;
|
||||
int numReaders = mergeState.maxDocs.length;
|
||||
|
||||
MatchingReaders matching = new MatchingReaders(mergeState);
|
||||
if (mergeState.needsIndexSort) {
|
||||
/**
|
||||
* If all readers are compressed and they have the same fieldinfos then we can merge the serialized document
|
||||
* directly.
|
||||
*/
|
||||
List<CompressingStoredFieldsMergeSub> subs = new ArrayList<>();
|
||||
for(int i=0;i<mergeState.storedFieldsReaders.length;i++) {
|
||||
if (matching.matchingReaders[i] &&
|
||||
mergeState.storedFieldsReaders[i] instanceof CompressingStoredFieldsReader) {
|
||||
CompressingStoredFieldsReader storedFieldsReader = (CompressingStoredFieldsReader) mergeState.storedFieldsReaders[i];
|
||||
storedFieldsReader.checkIntegrity();
|
||||
subs.add(new CompressingStoredFieldsMergeSub(storedFieldsReader, mergeState.docMaps[i], mergeState.maxDocs[i]));
|
||||
} else {
|
||||
return super.merge(mergeState);
|
||||
}
|
||||
}
|
||||
|
||||
final DocIDMerger<CompressingStoredFieldsMergeSub> docIDMerger =
|
||||
new DocIDMerger<>(subs, true);
|
||||
while (true) {
|
||||
CompressingStoredFieldsMergeSub sub = docIDMerger.next();
|
||||
if (sub == null) {
|
||||
break;
|
||||
}
|
||||
assert sub.mappedDocID == docCount;
|
||||
SerializedDocument doc = sub.reader.document(sub.docID);
|
||||
startDocument();
|
||||
bufferedDocs.copyBytes(doc.in, doc.length);
|
||||
numStoredFieldsInDoc = doc.numStoredFields;
|
||||
finishDocument();
|
||||
++docCount;
|
||||
}
|
||||
finish(mergeState.mergeFieldInfos, docCount);
|
||||
return docCount;
|
||||
}
|
||||
|
||||
for (int readerIndex=0;readerIndex<numReaders;readerIndex++) {
|
||||
MergeVisitor visitor = new MergeVisitor(mergeState, readerIndex);
|
||||
|
@ -630,4 +664,26 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
|
|||
return candidate.getNumDirtyChunks() > 1024 ||
|
||||
candidate.getNumDirtyChunks() * 100 > candidate.getNumChunks();
|
||||
}
|
||||
|
||||
private static class CompressingStoredFieldsMergeSub extends DocIDMerger.Sub {
|
||||
private final CompressingStoredFieldsReader reader;
|
||||
private final int maxDoc;
|
||||
int docID = -1;
|
||||
|
||||
public CompressingStoredFieldsMergeSub(CompressingStoredFieldsReader reader, MergeState.DocMap docMap, int maxDoc) {
|
||||
super(docMap);
|
||||
this.maxDoc = maxDoc;
|
||||
this.reader = reader;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() {
|
||||
docID++;
|
||||
if (docID == maxDoc) {
|
||||
return NO_MORE_DOCS;
|
||||
} else {
|
||||
return docID;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -176,7 +176,7 @@ public final class Lucene50StoredFieldsFormat extends StoredFieldsFormat {
|
|||
@Override
|
||||
public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOContext context) throws IOException {
|
||||
String previous = si.putAttribute(MODE_KEY, mode.name());
|
||||
if (previous != null) {
|
||||
if (previous != null && previous.equals(mode.name()) == false) {
|
||||
throw new IllegalStateException("found existing value for " + MODE_KEY + " for segment: " + si.name +
|
||||
"old=" + previous + ", new=" + mode.name());
|
||||
}
|
||||
|
|
|
@ -134,13 +134,10 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
|
|||
|
||||
@Override
|
||||
public void merge(MergeState mergeState) throws IOException {
|
||||
if (mergeState.needsIndexSort) {
|
||||
// TODO: can we gain back some optos even if index is sorted? E.g. if sort results in large chunks of contiguous docs from one sub
|
||||
// being copied over...?
|
||||
super.merge(mergeState);
|
||||
return;
|
||||
}
|
||||
|
||||
/**
|
||||
* If indexSort is activated and some of the leaves are not sorted the next test will catch that and the non-optimized merge will run.
|
||||
* If the readers are all sorted then it's safe to perform a bulk merge of the points.
|
||||
**/
|
||||
for(PointsReader reader : mergeState.pointsReaders) {
|
||||
if (reader instanceof Lucene60PointsReader == false) {
|
||||
// We can only bulk merge when all to-be-merged segments use our format:
|
||||
|
|
|
@ -21,16 +21,20 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.store.DataInput;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.PagedBytes;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
import org.apache.lucene.util.packed.PackedLongValues;
|
||||
|
||||
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
|
||||
|
||||
/** Buffers up pending byte[] per doc, then flushes when
|
||||
* segment flushes. */
|
||||
class BinaryDocValuesWriter extends DocValuesWriter {
|
||||
|
@ -98,10 +102,37 @@ class BinaryDocValuesWriter extends DocValuesWriter {
|
|||
public void finish(int maxDoc) {
|
||||
}
|
||||
|
||||
private SortingLeafReader.CachedBinaryDVs sortDocValues(int maxDoc, Sorter.DocMap sortMap, BinaryDocValues oldValues) throws IOException {
|
||||
FixedBitSet docsWithField = new FixedBitSet(maxDoc);
|
||||
BytesRef[] values = new BytesRef[maxDoc];
|
||||
while (true) {
|
||||
int docID = oldValues.nextDoc();
|
||||
if (docID == NO_MORE_DOCS) {
|
||||
break;
|
||||
}
|
||||
int newDocID = sortMap.oldToNew(docID);
|
||||
docsWithField.set(newDocID);
|
||||
values[newDocID] = BytesRef.deepCopyOf(oldValues.binaryValue());
|
||||
}
|
||||
return new SortingLeafReader.CachedBinaryDVs(values, docsWithField);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush(SegmentWriteState state, DocValuesConsumer dvConsumer) throws IOException {
|
||||
Sorter.DocComparator getDocComparator(int numDoc, SortField sortField) throws IOException {
|
||||
throw new IllegalArgumentException("It is forbidden to sort on a binary field");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush(SegmentWriteState state, Sorter.DocMap sortMap, DocValuesConsumer dvConsumer) throws IOException {
|
||||
bytes.freeze(false);
|
||||
final PackedLongValues lengths = this.lengths.build();
|
||||
final SortingLeafReader.CachedBinaryDVs sorted;
|
||||
if (sortMap != null) {
|
||||
sorted = sortDocValues(state.segmentInfo.maxDoc(), sortMap,
|
||||
new BufferedBinaryDocValues(lengths, maxLength, bytes.getDataInput(), docsWithField.iterator()));
|
||||
} else {
|
||||
sorted = null;
|
||||
}
|
||||
dvConsumer.addBinaryField(fieldInfo,
|
||||
new EmptyDocValuesProducer() {
|
||||
@Override
|
||||
|
@ -109,7 +140,11 @@ class BinaryDocValuesWriter extends DocValuesWriter {
|
|||
if (fieldInfoIn != fieldInfo) {
|
||||
throw new IllegalArgumentException("wrong fieldInfo");
|
||||
}
|
||||
return new BufferedBinaryDocValues(lengths, maxLength, bytes.getDataInput(), docsWithField.iterator());
|
||||
if (sorted == null) {
|
||||
return new BufferedBinaryDocValues(lengths, maxLength, bytes.getDataInput(), docsWithField.iterator());
|
||||
} else {
|
||||
return new SortingLeafReader.SortingBinaryDocValues(sorted);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -18,9 +18,13 @@ package org.apache.lucene.index;
|
|||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
|
@ -29,8 +33,9 @@ import org.apache.lucene.codecs.NormsConsumer;
|
|||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.PointsFormat;
|
||||
import org.apache.lucene.codecs.PointsWriter;
|
||||
import org.apache.lucene.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.search.similarities.Similarity;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
|
@ -50,10 +55,8 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
|
||||
// Writes postings and term vectors:
|
||||
final TermsHash termsHash;
|
||||
|
||||
// lazy init:
|
||||
private StoredFieldsWriter storedFieldsWriter;
|
||||
private int lastStoredDocID;
|
||||
// Writes stored fields
|
||||
final StoredFieldsConsumer storedFieldsConsumer;
|
||||
|
||||
// NOTE: I tried using Hash Map<String,PerField>
|
||||
// but it was ~2% slower on Wiki and Geonames with Java
|
||||
|
@ -67,54 +70,79 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
// Holds fields seen in each document
|
||||
private PerField[] fields = new PerField[1];
|
||||
|
||||
private final Set<String> finishedDocValues = new HashSet<>();
|
||||
|
||||
public DefaultIndexingChain(DocumentsWriterPerThread docWriter) throws IOException {
|
||||
this.docWriter = docWriter;
|
||||
this.fieldInfos = docWriter.getFieldInfosBuilder();
|
||||
this.docState = docWriter.docState;
|
||||
this.bytesUsed = docWriter.bytesUsed;
|
||||
|
||||
TermsHash termVectorsWriter = new TermVectorsConsumer(docWriter);
|
||||
final TermsHash termVectorsWriter;
|
||||
if (docWriter.getSegmentInfo().getIndexSort() == null) {
|
||||
storedFieldsConsumer = new StoredFieldsConsumer(docWriter);
|
||||
termVectorsWriter = new TermVectorsConsumer(docWriter);
|
||||
} else {
|
||||
storedFieldsConsumer = new SortingStoredFieldsConsumer(docWriter);
|
||||
termVectorsWriter = new SortingTermVectorsConsumer(docWriter);
|
||||
}
|
||||
termsHash = new FreqProxTermsWriter(docWriter, termVectorsWriter);
|
||||
}
|
||||
|
||||
// TODO: can we remove this lazy-init / make cleaner / do it another way...?
|
||||
private void initStoredFieldsWriter() throws IOException {
|
||||
if (storedFieldsWriter == null) {
|
||||
storedFieldsWriter = docWriter.codec.storedFieldsFormat().fieldsWriter(docWriter.directory, docWriter.getSegmentInfo(), IOContext.DEFAULT);
|
||||
|
||||
private Sorter.DocMap maybeSortSegment(SegmentWriteState state) throws IOException {
|
||||
Sort indexSort = state.segmentInfo.getIndexSort();
|
||||
if (indexSort == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
List<Sorter.DocComparator> comparators = new ArrayList<>();
|
||||
for (int i = 0; i < indexSort.getSort().length; i++) {
|
||||
SortField sortField = indexSort.getSort()[i];
|
||||
PerField perField = getPerField(sortField.getField());
|
||||
if (perField != null && perField.docValuesWriter != null &&
|
||||
finishedDocValues.contains(perField.fieldInfo.name) == false) {
|
||||
perField.docValuesWriter.finish(state.segmentInfo.maxDoc());
|
||||
Sorter.DocComparator cmp = perField.docValuesWriter.getDocComparator(state.segmentInfo.maxDoc(), sortField);
|
||||
comparators.add(cmp);
|
||||
finishedDocValues.add(perField.fieldInfo.name);
|
||||
} else {
|
||||
// safe to ignore, sort field with no values or already seen before
|
||||
}
|
||||
}
|
||||
Sorter sorter = new Sorter(indexSort);
|
||||
// returns null if the documents are already sorted
|
||||
return sorter.sort(state.segmentInfo.maxDoc(), comparators.toArray(new Sorter.DocComparator[comparators.size()]));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush(SegmentWriteState state) throws IOException, AbortingException {
|
||||
public Sorter.DocMap flush(SegmentWriteState state) throws IOException, AbortingException {
|
||||
|
||||
// NOTE: caller (DocumentsWriterPerThread) handles
|
||||
// aborting on any exception from this method
|
||||
|
||||
Sorter.DocMap sortMap = maybeSortSegment(state);
|
||||
int maxDoc = state.segmentInfo.maxDoc();
|
||||
long t0 = System.nanoTime();
|
||||
writeNorms(state);
|
||||
writeNorms(state, sortMap);
|
||||
if (docState.infoStream.isEnabled("IW")) {
|
||||
docState.infoStream.message("IW", ((System.nanoTime()-t0)/1000000) + " msec to write norms");
|
||||
}
|
||||
|
||||
t0 = System.nanoTime();
|
||||
writeDocValues(state);
|
||||
writeDocValues(state, sortMap);
|
||||
if (docState.infoStream.isEnabled("IW")) {
|
||||
docState.infoStream.message("IW", ((System.nanoTime()-t0)/1000000) + " msec to write docValues");
|
||||
}
|
||||
|
||||
t0 = System.nanoTime();
|
||||
writePoints(state);
|
||||
writePoints(state, sortMap);
|
||||
if (docState.infoStream.isEnabled("IW")) {
|
||||
docState.infoStream.message("IW", ((System.nanoTime()-t0)/1000000) + " msec to write points");
|
||||
}
|
||||
|
||||
// it's possible all docs hit non-aborting exceptions...
|
||||
t0 = System.nanoTime();
|
||||
initStoredFieldsWriter();
|
||||
fillStoredFields(maxDoc);
|
||||
storedFieldsWriter.finish(state.fieldInfos, maxDoc);
|
||||
storedFieldsWriter.close();
|
||||
storedFieldsConsumer.finish(maxDoc);
|
||||
storedFieldsConsumer.flush(state, sortMap);
|
||||
if (docState.infoStream.isEnabled("IW")) {
|
||||
docState.infoStream.message("IW", ((System.nanoTime()-t0)/1000000) + " msec to finish stored fields");
|
||||
}
|
||||
|
@ -131,7 +159,7 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
}
|
||||
}
|
||||
|
||||
termsHash.flush(fieldsToFlush, state);
|
||||
termsHash.flush(fieldsToFlush, state, sortMap);
|
||||
if (docState.infoStream.isEnabled("IW")) {
|
||||
docState.infoStream.message("IW", ((System.nanoTime()-t0)/1000000) + " msec to write postings and finish vectors");
|
||||
}
|
||||
|
@ -145,10 +173,12 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
if (docState.infoStream.isEnabled("IW")) {
|
||||
docState.infoStream.message("IW", ((System.nanoTime()-t0)/1000000) + " msec to write fieldInfos");
|
||||
}
|
||||
|
||||
return sortMap;
|
||||
}
|
||||
|
||||
/** Writes all buffered points. */
|
||||
private void writePoints(SegmentWriteState state) throws IOException {
|
||||
private void writePoints(SegmentWriteState state, Sorter.DocMap sortMap) throws IOException {
|
||||
PointsWriter pointsWriter = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
|
@ -169,7 +199,7 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
pointsWriter = fmt.fieldsWriter(state);
|
||||
}
|
||||
|
||||
perField.pointValuesWriter.flush(state, pointsWriter);
|
||||
perField.pointValuesWriter.flush(state, sortMap, pointsWriter);
|
||||
perField.pointValuesWriter = null;
|
||||
} else if (perField.fieldInfo.getPointDimensionCount() != 0) {
|
||||
// BUG
|
||||
|
@ -192,7 +222,7 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
}
|
||||
|
||||
/** Writes all buffered doc values (called from {@link #flush}). */
|
||||
private void writeDocValues(SegmentWriteState state) throws IOException {
|
||||
private void writeDocValues(SegmentWriteState state, Sorter.DocMap sortMap) throws IOException {
|
||||
int maxDoc = state.segmentInfo.maxDoc();
|
||||
DocValuesConsumer dvConsumer = null;
|
||||
boolean success = false;
|
||||
|
@ -211,8 +241,10 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
dvConsumer = fmt.fieldsConsumer(state);
|
||||
}
|
||||
|
||||
perField.docValuesWriter.finish(maxDoc);
|
||||
perField.docValuesWriter.flush(state, dvConsumer);
|
||||
if (finishedDocValues.contains(perField.fieldInfo.name) == false) {
|
||||
perField.docValuesWriter.finish(maxDoc);
|
||||
}
|
||||
perField.docValuesWriter.flush(state, sortMap, dvConsumer);
|
||||
perField.docValuesWriter = null;
|
||||
} else if (perField.fieldInfo.getDocValuesType() != DocValuesType.NONE) {
|
||||
// BUG
|
||||
|
@ -246,17 +278,7 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
}
|
||||
}
|
||||
|
||||
/** Catch up for all docs before us that had no stored
|
||||
* fields, or hit non-aborting exceptions before writing
|
||||
* stored fields. */
|
||||
private void fillStoredFields(int docID) throws IOException, AbortingException {
|
||||
while (lastStoredDocID < docID) {
|
||||
startStoredFields();
|
||||
finishStoredFields();
|
||||
}
|
||||
}
|
||||
|
||||
private void writeNorms(SegmentWriteState state) throws IOException {
|
||||
private void writeNorms(SegmentWriteState state, Sorter.DocMap sortMap) throws IOException {
|
||||
boolean success = false;
|
||||
NormsConsumer normsConsumer = null;
|
||||
try {
|
||||
|
@ -274,7 +296,7 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
if (fi.omitsNorms() == false && fi.getIndexOptions() != IndexOptions.NONE) {
|
||||
assert perField.norms != null: "field=" + fi.name;
|
||||
perField.norms.finish(state.segmentInfo.maxDoc());
|
||||
perField.norms.flush(state, normsConsumer);
|
||||
perField.norms.flush(state, sortMap, normsConsumer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -290,7 +312,10 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
|
||||
@Override
|
||||
public void abort() {
|
||||
IOUtils.closeWhileHandlingException(storedFieldsWriter);
|
||||
try {
|
||||
storedFieldsConsumer.abort();
|
||||
} catch (Throwable t) {
|
||||
}
|
||||
|
||||
try {
|
||||
// E.g. close any open files in the term vectors writer:
|
||||
|
@ -326,21 +351,19 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
|
||||
/** Calls StoredFieldsWriter.startDocument, aborting the
|
||||
* segment if it hits any exception. */
|
||||
private void startStoredFields() throws IOException, AbortingException {
|
||||
private void startStoredFields(int docID) throws IOException, AbortingException {
|
||||
try {
|
||||
initStoredFieldsWriter();
|
||||
storedFieldsWriter.startDocument();
|
||||
storedFieldsConsumer.startDocument(docID);
|
||||
} catch (Throwable th) {
|
||||
throw AbortingException.wrap(th);
|
||||
}
|
||||
lastStoredDocID++;
|
||||
}
|
||||
|
||||
/** Calls StoredFieldsWriter.finishDocument, aborting the
|
||||
* segment if it hits any exception. */
|
||||
private void finishStoredFields() throws IOException, AbortingException {
|
||||
try {
|
||||
storedFieldsWriter.finishDocument();
|
||||
storedFieldsConsumer.finishDocument();
|
||||
} catch (Throwable th) {
|
||||
throw AbortingException.wrap(th);
|
||||
}
|
||||
|
@ -364,8 +387,7 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
|
||||
termsHash.startDocument();
|
||||
|
||||
fillStoredFields(docState.docID);
|
||||
startStoredFields();
|
||||
startStoredFields(docState.docID);
|
||||
|
||||
boolean aborting = false;
|
||||
try {
|
||||
|
@ -435,7 +457,7 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
throw new IllegalArgumentException("stored field \"" + field.name() + "\" is too large (" + value.length() + " characters) to store");
|
||||
}
|
||||
try {
|
||||
storedFieldsWriter.writeField(fp.fieldInfo, field);
|
||||
storedFieldsConsumer.writeField(fp.fieldInfo, field);
|
||||
} catch (Throwable th) {
|
||||
throw AbortingException.wrap(th);
|
||||
}
|
||||
|
|
|
@ -21,6 +21,6 @@ import java.io.IOException;
|
|||
|
||||
abstract class DocConsumer {
|
||||
abstract void processDocument() throws IOException, AbortingException;
|
||||
abstract void flush(final SegmentWriteState state) throws IOException, AbortingException;
|
||||
abstract Sorter.DocMap flush(final SegmentWriteState state) throws IOException, AbortingException;
|
||||
abstract void abort();
|
||||
}
|
||||
|
|
|
@ -20,8 +20,10 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.search.SortField;
|
||||
|
||||
abstract class DocValuesWriter {
|
||||
abstract void finish(int numDoc);
|
||||
abstract void flush(SegmentWriteState state, DocValuesConsumer consumer) throws IOException;
|
||||
abstract void flush(SegmentWriteState state, Sorter.DocMap sortMap, DocValuesConsumer consumer) throws IOException;
|
||||
abstract Sorter.DocComparator getDocComparator(int numDoc, SortField sortField) throws IOException;
|
||||
}
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.lucene.store.TrackingDirectoryWrapper;
|
|||
import org.apache.lucene.util.ByteBlockPool.Allocator;
|
||||
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.IntBlockPool;
|
||||
import org.apache.lucene.util.MutableBits;
|
||||
|
@ -177,7 +178,7 @@ class DocumentsWriterPerThread {
|
|||
assert numDocsInRAM == 0 : "num docs " + numDocsInRAM;
|
||||
deleteSlice = deleteQueue.newSlice();
|
||||
|
||||
segmentInfo = new SegmentInfo(directoryOrig, Version.LATEST, segmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
|
||||
segmentInfo = new SegmentInfo(directoryOrig, Version.LATEST, segmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), indexWriterConfig.getIndexSort());
|
||||
assert numDocsInRAM == 0;
|
||||
if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segmentName + " delQueue=" + deleteQueue);
|
||||
|
@ -438,9 +439,9 @@ class DocumentsWriterPerThread {
|
|||
if (infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", "flush postings as segment " + flushState.segmentInfo.name + " numDocs=" + numDocsInRAM);
|
||||
}
|
||||
|
||||
final Sorter.DocMap sortMap;
|
||||
try {
|
||||
consumer.flush(flushState);
|
||||
sortMap = consumer.flush(flushState);
|
||||
pendingUpdates.terms.clear();
|
||||
segmentInfo.setFiles(new HashSet<>(directory.getCreatedFiles()));
|
||||
|
||||
|
@ -477,7 +478,7 @@ class DocumentsWriterPerThread {
|
|||
|
||||
FlushedSegment fs = new FlushedSegment(segmentInfoPerCommit, flushState.fieldInfos,
|
||||
segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush);
|
||||
sealFlushedSegment(fs);
|
||||
sealFlushedSegment(fs, sortMap);
|
||||
if (infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", "flush time " + ((System.nanoTime() - t0)/1000000.0) + " msec");
|
||||
}
|
||||
|
@ -494,11 +495,23 @@ class DocumentsWriterPerThread {
|
|||
public Set<String> pendingFilesToDelete() {
|
||||
return filesToDelete;
|
||||
}
|
||||
|
||||
private MutableBits sortLiveDocs(MutableBits bits, Sorter.DocMap sortMap) {
|
||||
assert bits != null && sortMap != null;
|
||||
FixedBitSet bitSet = new FixedBitSet(bits.length());
|
||||
for (int i = 0; i < bits.length(); i++) {
|
||||
if (bits.get(i)) {
|
||||
bitSet.set(sortMap.oldToNew(i));
|
||||
}
|
||||
}
|
||||
return bitSet;
|
||||
}
|
||||
|
||||
/**
|
||||
* Seals the {@link SegmentInfo} for the new flushed segment and persists
|
||||
* the deleted documents {@link MutableBits}.
|
||||
*/
|
||||
void sealFlushedSegment(FlushedSegment flushedSegment) throws IOException {
|
||||
void sealFlushedSegment(FlushedSegment flushedSegment, Sorter.DocMap sortMap) throws IOException {
|
||||
assert flushedSegment != null;
|
||||
|
||||
SegmentCommitInfo newSegment = flushedSegment.segmentInfo;
|
||||
|
@ -548,7 +561,13 @@ class DocumentsWriterPerThread {
|
|||
|
||||
SegmentCommitInfo info = flushedSegment.segmentInfo;
|
||||
Codec codec = info.info.getCodec();
|
||||
codec.liveDocsFormat().writeLiveDocs(flushedSegment.liveDocs, directory, info, delCount, context);
|
||||
final MutableBits bits;
|
||||
if (sortMap == null) {
|
||||
bits = flushedSegment.liveDocs;
|
||||
} else {
|
||||
bits = sortLiveDocs(flushedSegment.liveDocs, sortMap);
|
||||
}
|
||||
codec.liveDocsFormat().writeLiveDocs(bits, directory, info, delCount, context);
|
||||
newSegment.setDelCount(delCount);
|
||||
newSegment.advanceDelGen();
|
||||
}
|
||||
|
|
|
@ -79,8 +79,8 @@ final class FreqProxTermsWriter extends TermsHash {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void flush(Map<String,TermsHashPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
|
||||
super.flush(fieldsToFlush, state);
|
||||
public void flush(Map<String,TermsHashPerField> fieldsToFlush, final SegmentWriteState state, Sorter.DocMap sortMap) throws IOException {
|
||||
super.flush(fieldsToFlush, state, sortMap);
|
||||
|
||||
// Gather all fields that saw any postings:
|
||||
List<FreqProxTermsWriterPerField> allFields = new ArrayList<>();
|
||||
|
@ -98,8 +98,10 @@ final class FreqProxTermsWriter extends TermsHash {
|
|||
CollectionUtil.introSort(allFields);
|
||||
|
||||
Fields fields = new FreqProxFields(allFields);
|
||||
|
||||
applyDeletes(state, fields);
|
||||
if (sortMap != null) {
|
||||
fields = new SortingLeafReader.SortingFields(fields, state.fieldInfos, sortMap);
|
||||
}
|
||||
|
||||
FieldsConsumer consumer = state.segmentInfo.getCodec().postingsFormat().fieldsConsumer(state);
|
||||
boolean success = false;
|
||||
|
|
|
@ -1032,8 +1032,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
}
|
||||
}
|
||||
|
||||
/** 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. */
|
||||
/** 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 unsorted flushed segments built
|
||||
* before {@link Version#LUCENE_7_0_0} (flushed segments are sorted in Lucene 7.0). */
|
||||
private void validateIndexSort() {
|
||||
Sort indexSort = config.getIndexSort();
|
||||
if (indexSort != null) {
|
||||
|
@ -1041,6 +1042,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
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);
|
||||
} else if (segmentIndexSort == null) {
|
||||
// Flushed segments are not sorted if they were built with a version prior to 7.0
|
||||
assert info.info.getVersion().onOrAfter(Version.LUCENE_7_0_0) == false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -223,7 +223,10 @@ public class MergeState {
|
|||
return originalReaders;
|
||||
}
|
||||
|
||||
// If an incoming reader is not sorted, because it was flushed by IW, we sort it here:
|
||||
/** If an incoming reader is not sorted, because it was flushed by IW older than {@link Version.LUCENE_7_0_0}
|
||||
* or because we add unsorted segments from another index {@link IndexWriter#addIndexes(CodecReader...)} ,
|
||||
* we sort it here:
|
||||
*/
|
||||
final Sorter sorter = new Sorter(indexSort);
|
||||
List<CodecReader> readers = new ArrayList<>(originalReaders.size());
|
||||
|
||||
|
@ -231,9 +234,6 @@ public class MergeState {
|
|||
Sort segmentSort = leaf.getIndexSort();
|
||||
|
||||
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 their index files on each indexed document:
|
||||
|
||||
// This segment was written by flush, so documents are not yet sorted, so we sort them now:
|
||||
long t0 = System.nanoTime();
|
||||
Sorter.DocMap sortDocMap = sorter.sort(leaf);
|
||||
|
|
|
@ -68,10 +68,15 @@ class NormValuesWriter {
|
|||
public void finish(int maxDoc) {
|
||||
}
|
||||
|
||||
public void flush(SegmentWriteState state, NormsConsumer normsConsumer) throws IOException {
|
||||
|
||||
public void flush(SegmentWriteState state, Sorter.DocMap sortMap, NormsConsumer normsConsumer) throws IOException {
|
||||
final PackedLongValues values = pending.build();
|
||||
|
||||
final SortingLeafReader.CachedNumericDVs sorted;
|
||||
if (sortMap != null) {
|
||||
sorted = NumericDocValuesWriter.sortDocValues(state.segmentInfo.maxDoc(), sortMap,
|
||||
new BufferedNorms(values, docsWithField.iterator()));
|
||||
} else {
|
||||
sorted = null;
|
||||
}
|
||||
normsConsumer.addNormsField(fieldInfo,
|
||||
new NormsProducer() {
|
||||
@Override
|
||||
|
@ -79,7 +84,11 @@ class NormValuesWriter {
|
|||
if (fieldInfo != NormValuesWriter.this.fieldInfo) {
|
||||
throw new IllegalArgumentException("wrong fieldInfo");
|
||||
}
|
||||
return new BufferedNorms(values, docsWithField.iterator());
|
||||
if (sorted == null) {
|
||||
return new BufferedNorms(values, docsWithField.iterator());
|
||||
} else {
|
||||
return new SortingLeafReader.SortingNumericDocValues(sorted);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,15 +21,20 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
import org.apache.lucene.util.packed.PackedLongValues;
|
||||
|
||||
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
|
||||
|
||||
/** Buffers up pending long per doc, then flushes when
|
||||
* segment flushes. */
|
||||
class NumericDocValuesWriter extends DocValuesWriter {
|
||||
|
||||
private PackedLongValues.Builder pending;
|
||||
private PackedLongValues finalValues;
|
||||
private final Counter iwBytesUsed;
|
||||
private long bytesUsed;
|
||||
private DocsWithFieldSet docsWithField;
|
||||
|
@ -69,9 +74,45 @@ class NumericDocValuesWriter extends DocValuesWriter {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void flush(SegmentWriteState state, DocValuesConsumer dvConsumer) throws IOException {
|
||||
Sorter.DocComparator getDocComparator(int maxDoc, SortField sortField) throws IOException {
|
||||
assert finalValues == null;
|
||||
finalValues = pending.build();
|
||||
final BufferedNumericDocValues docValues =
|
||||
new BufferedNumericDocValues(finalValues, docsWithField.iterator());
|
||||
return Sorter.getDocComparator(maxDoc, sortField, () -> null, () -> docValues);
|
||||
}
|
||||
|
||||
final PackedLongValues values = pending.build();
|
||||
static SortingLeafReader.CachedNumericDVs sortDocValues(int maxDoc, Sorter.DocMap sortMap, NumericDocValues oldDocValues) throws IOException {
|
||||
FixedBitSet docsWithField = new FixedBitSet(maxDoc);
|
||||
long[] values = new long[maxDoc];
|
||||
while (true) {
|
||||
int docID = oldDocValues.nextDoc();
|
||||
if (docID == NO_MORE_DOCS) {
|
||||
break;
|
||||
}
|
||||
int newDocID = sortMap.oldToNew(docID);
|
||||
docsWithField.set(newDocID);
|
||||
values[newDocID] = oldDocValues.longValue();
|
||||
}
|
||||
return new SortingLeafReader.CachedNumericDVs(values, docsWithField);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush(SegmentWriteState state, Sorter.DocMap sortMap, DocValuesConsumer dvConsumer) throws IOException {
|
||||
final PackedLongValues values;
|
||||
if (finalValues == null) {
|
||||
values = pending.build();
|
||||
} else {
|
||||
values = finalValues;
|
||||
}
|
||||
|
||||
final SortingLeafReader.CachedNumericDVs sorted;
|
||||
if (sortMap != null) {
|
||||
NumericDocValues oldValues = new BufferedNumericDocValues(values, docsWithField.iterator());
|
||||
sorted = sortDocValues(state.segmentInfo.maxDoc(), sortMap, oldValues);
|
||||
} else {
|
||||
sorted = null;
|
||||
}
|
||||
|
||||
dvConsumer.addNumericField(fieldInfo,
|
||||
new EmptyDocValuesProducer() {
|
||||
|
@ -80,7 +121,11 @@ class NumericDocValuesWriter extends DocValuesWriter {
|
|||
if (fieldInfo != NumericDocValuesWriter.this.fieldInfo) {
|
||||
throw new IllegalArgumentException("wrong fieldInfo");
|
||||
}
|
||||
return new BufferedNumericDocValues(values, docsWithField.iterator());
|
||||
if (sorted == null) {
|
||||
return new BufferedNumericDocValues(values, docsWithField.iterator());
|
||||
} else {
|
||||
return new SortingLeafReader.SortingNumericDocValues(sorted);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -69,9 +69,8 @@ class PointValuesWriter {
|
|||
numPoints++;
|
||||
}
|
||||
|
||||
public void flush(SegmentWriteState state, PointsWriter writer) throws IOException {
|
||||
PointValues values = new MutablePointValues() {
|
||||
|
||||
public void flush(SegmentWriteState state, Sorter.DocMap sortMap, PointsWriter writer) throws IOException {
|
||||
PointValues points = new MutablePointValues() {
|
||||
final int[] ords = new int[numPoints];
|
||||
{
|
||||
for (int i = 0; i < numPoints; ++i) {
|
||||
|
@ -147,8 +146,13 @@ class PointValuesWriter {
|
|||
}
|
||||
};
|
||||
|
||||
final PointValues values;
|
||||
if (sortMap == null) {
|
||||
values = points;
|
||||
} else {
|
||||
values = new MutableSortingPointValues((MutablePointValues) points, sortMap);
|
||||
}
|
||||
PointsReader reader = new PointsReader() {
|
||||
|
||||
@Override
|
||||
public PointValues getValues(String fieldName) {
|
||||
if (fieldName.equals(fieldInfo.name) == false) {
|
||||
|
@ -156,7 +160,7 @@ class PointValuesWriter {
|
|||
}
|
||||
return values;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void checkIntegrity() {
|
||||
throw new UnsupportedOperationException();
|
||||
|
@ -171,7 +175,87 @@ class PointValuesWriter {
|
|||
public void close() {
|
||||
}
|
||||
};
|
||||
|
||||
writer.writeField(fieldInfo, reader);
|
||||
}
|
||||
|
||||
static final class MutableSortingPointValues extends MutablePointValues {
|
||||
|
||||
private final MutablePointValues in;
|
||||
private final Sorter.DocMap docMap;
|
||||
|
||||
public MutableSortingPointValues(final MutablePointValues in, Sorter.DocMap docMap) {
|
||||
this.in = in;
|
||||
this.docMap = docMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void intersect(IntersectVisitor visitor) throws IOException {
|
||||
in.intersect(new IntersectVisitor() {
|
||||
@Override
|
||||
public void visit(int docID) throws IOException {
|
||||
visitor.visit(docMap.oldToNew(docID));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(int docID, byte[] packedValue) throws IOException {
|
||||
visitor.visit(docMap.oldToNew(docID), packedValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
return visitor.compare(minPackedValue, maxPackedValue);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getMinPackedValue() throws IOException {
|
||||
return in.getMinPackedValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getMaxPackedValue() throws IOException {
|
||||
return in.getMaxPackedValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNumDimensions() throws IOException {
|
||||
return in.getNumDimensions();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getBytesPerDimension() throws IOException {
|
||||
return in.getBytesPerDimension();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long size() {
|
||||
return in.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getDocCount() {
|
||||
return in.getDocCount();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getValue(int i, BytesRef packedValue) {
|
||||
in.getValue(i, packedValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte getByteAt(int i, int k) {
|
||||
return in.getByteAt(i, k);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getDocID(int i) {
|
||||
return docMap.oldToNew(in.getDocID(i));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void swap(int i, int j) {
|
||||
in.swap(i, j);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,20 +16,23 @@
|
|||
*/
|
||||
package org.apache.lucene.index;
|
||||
|
||||
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.util.ByteBlockPool;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
|
||||
import org.apache.lucene.util.BytesRefHash;
|
||||
import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
import org.apache.lucene.util.packed.PackedLongValues;
|
||||
|
||||
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
|
||||
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
|
||||
|
||||
/** Buffers up pending byte[] per doc, deref and sorting via
|
||||
* int ord, then flushes when segment flushes. */
|
||||
class SortedDocValuesWriter extends DocValuesWriter {
|
||||
|
@ -41,6 +44,10 @@ class SortedDocValuesWriter extends DocValuesWriter {
|
|||
private final FieldInfo fieldInfo;
|
||||
private int lastDocID = -1;
|
||||
|
||||
private PackedLongValues finalOrds;
|
||||
private int[] finalSortedValues;
|
||||
private int[] finalOrdMap;
|
||||
|
||||
public SortedDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
|
||||
this.fieldInfo = fieldInfo;
|
||||
this.iwBytesUsed = iwBytesUsed;
|
||||
|
@ -100,17 +107,59 @@ class SortedDocValuesWriter extends DocValuesWriter {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void flush(SegmentWriteState state, DocValuesConsumer dvConsumer) throws IOException {
|
||||
Sorter.DocComparator getDocComparator(int maxDoc, SortField sortField) throws IOException {
|
||||
assert sortField.getType().equals(SortField.Type.STRING);
|
||||
assert finalSortedValues == null && finalOrdMap == null &&finalOrds == null;
|
||||
int valueCount = hash.size();
|
||||
finalSortedValues = hash.sort();
|
||||
finalOrds = pending.build();
|
||||
finalOrdMap = new int[valueCount];
|
||||
for (int ord = 0; ord < valueCount; ord++) {
|
||||
finalOrdMap[finalSortedValues[ord]] = ord;
|
||||
}
|
||||
final SortedDocValues docValues =
|
||||
new BufferedSortedDocValues(hash, valueCount, finalOrds, finalSortedValues, finalOrdMap,
|
||||
docsWithField.iterator());
|
||||
return Sorter.getDocComparator(maxDoc, sortField, () -> docValues, () -> null);
|
||||
}
|
||||
|
||||
private int[] sortDocValues(int maxDoc, Sorter.DocMap sortMap, SortedDocValues oldValues) throws IOException {
|
||||
int[] ords = new int[maxDoc];
|
||||
Arrays.fill(ords, -1);
|
||||
int docID;
|
||||
while ((docID = oldValues.nextDoc()) != NO_MORE_DOCS) {
|
||||
int newDocID = sortMap.oldToNew(docID);
|
||||
ords[newDocID] = oldValues.ordValue();
|
||||
}
|
||||
return ords;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush(SegmentWriteState state, Sorter.DocMap sortMap, DocValuesConsumer dvConsumer) throws IOException {
|
||||
final int valueCount = hash.size();
|
||||
final PackedLongValues ords = pending.build();
|
||||
|
||||
final int[] sortedValues = hash.sort();
|
||||
final int[] ordMap = new int[valueCount];
|
||||
|
||||
for(int ord=0;ord<valueCount;ord++) {
|
||||
ordMap[sortedValues[ord]] = ord;
|
||||
final PackedLongValues ords;
|
||||
final int[] sortedValues;
|
||||
final int[] ordMap;
|
||||
if (finalOrds == null) {
|
||||
sortedValues = hash.sort();
|
||||
ords = pending.build();
|
||||
ordMap = new int[valueCount];
|
||||
for (int ord = 0; ord < valueCount; ord++) {
|
||||
ordMap[sortedValues[ord]] = ord;
|
||||
}
|
||||
} else {
|
||||
sortedValues = finalSortedValues;
|
||||
ords = finalOrds;
|
||||
ordMap = finalOrdMap;
|
||||
}
|
||||
|
||||
final int[] sorted;
|
||||
if (sortMap != null) {
|
||||
sorted = sortDocValues(state.segmentInfo.maxDoc(), sortMap,
|
||||
new BufferedSortedDocValues(hash, valueCount, ords, sortedValues, ordMap, docsWithField.iterator()));
|
||||
} else {
|
||||
sorted = null;
|
||||
}
|
||||
dvConsumer.addSortedField(fieldInfo,
|
||||
new EmptyDocValuesProducer() {
|
||||
@Override
|
||||
|
@ -118,7 +167,12 @@ class SortedDocValuesWriter extends DocValuesWriter {
|
|||
if (fieldInfoIn != fieldInfo) {
|
||||
throw new IllegalArgumentException("wrong fieldInfo");
|
||||
}
|
||||
return new BufferedSortedDocValues(hash, valueCount, ords, sortedValues, ordMap, docsWithField.iterator());
|
||||
final SortedDocValues buf =
|
||||
new BufferedSortedDocValues(hash, valueCount, ords, sortedValues, ordMap, docsWithField.iterator());
|
||||
if (sorted == null) {
|
||||
return buf;
|
||||
}
|
||||
return new SortingLeafReader.SortingSortedDocValues(buf, sorted);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -22,12 +22,17 @@ import java.util.Arrays;
|
|||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.search.SortedNumericSelector;
|
||||
import org.apache.lucene.search.SortedNumericSortField;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
import org.apache.lucene.util.packed.PackedLongValues;
|
||||
|
||||
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
|
||||
|
||||
/** Buffers up pending long[] per doc, sorts, then flushes when segment flushes. */
|
||||
class SortedNumericDocValuesWriter extends DocValuesWriter {
|
||||
private PackedLongValues.Builder pending; // stream of all values
|
||||
|
@ -40,6 +45,9 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
|
|||
private long currentValues[] = new long[8];
|
||||
private int currentUpto = 0;
|
||||
|
||||
private PackedLongValues finalValues;
|
||||
private PackedLongValues finalValuesCount;
|
||||
|
||||
public SortedNumericDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
|
||||
this.fieldInfo = fieldInfo;
|
||||
this.iwBytesUsed = iwBytesUsed;
|
||||
|
@ -98,9 +106,51 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void flush(SegmentWriteState state, DocValuesConsumer dvConsumer) throws IOException {
|
||||
final PackedLongValues values = pending.build();
|
||||
final PackedLongValues valueCounts = pendingCounts.build();
|
||||
Sorter.DocComparator getDocComparator(int maxDoc, SortField sortField) throws IOException {
|
||||
assert sortField instanceof SortedNumericSortField;
|
||||
assert finalValues == null && finalValuesCount == null;
|
||||
finalValues = pending.build();
|
||||
finalValuesCount = pendingCounts.build();
|
||||
final SortedNumericDocValues docValues =
|
||||
new BufferedSortedNumericDocValues(finalValues, finalValuesCount, docsWithField.iterator());
|
||||
SortedNumericSortField sf = (SortedNumericSortField) sortField;
|
||||
return Sorter.getDocComparator(maxDoc, sf, () -> null,
|
||||
() -> SortedNumericSelector.wrap(docValues, sf.getSelector(), sf.getNumericType()));
|
||||
}
|
||||
|
||||
private long[][] sortDocValues(int maxDoc, Sorter.DocMap sortMap, SortedNumericDocValues oldValues) throws IOException {
|
||||
long[][] values = new long[maxDoc][];
|
||||
int docID;
|
||||
while ((docID = oldValues.nextDoc()) != NO_MORE_DOCS) {
|
||||
int newDocID = sortMap.oldToNew(docID);
|
||||
long[] docValues = new long[oldValues.docValueCount()];
|
||||
for (int i = 0; i < docValues.length; i++) {
|
||||
docValues[i] = oldValues.nextValue();
|
||||
}
|
||||
values[newDocID] = docValues;
|
||||
}
|
||||
return values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush(SegmentWriteState state, Sorter.DocMap sortMap, DocValuesConsumer dvConsumer) throws IOException {
|
||||
final PackedLongValues values;
|
||||
final PackedLongValues valueCounts;
|
||||
if (finalValues == null) {
|
||||
values = pending.build();
|
||||
valueCounts = pendingCounts.build();
|
||||
} else {
|
||||
values = finalValues;
|
||||
valueCounts = finalValuesCount;
|
||||
}
|
||||
|
||||
final long[][] sorted;
|
||||
if (sortMap != null) {
|
||||
sorted = sortDocValues(state.segmentInfo.maxDoc(), sortMap,
|
||||
new BufferedSortedNumericDocValues(values, valueCounts, docsWithField.iterator()));
|
||||
} else {
|
||||
sorted = null;
|
||||
}
|
||||
|
||||
dvConsumer.addSortedNumericField(fieldInfo,
|
||||
new EmptyDocValuesProducer() {
|
||||
|
@ -109,7 +159,13 @@ class SortedNumericDocValuesWriter extends DocValuesWriter {
|
|||
if (fieldInfoIn != fieldInfo) {
|
||||
throw new IllegalArgumentException("wrong fieldInfo");
|
||||
}
|
||||
return new BufferedSortedNumericDocValues(values, valueCounts, docsWithField.iterator());
|
||||
final SortedNumericDocValues buf =
|
||||
new BufferedSortedNumericDocValues(values, valueCounts, docsWithField.iterator());
|
||||
if (sorted == null) {
|
||||
return buf;
|
||||
} else {
|
||||
return new SortingLeafReader.SortingSortedNumericDocValues(buf, sorted);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -16,22 +16,27 @@
|
|||
*/
|
||||
package org.apache.lucene.index;
|
||||
|
||||
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.SortField;
|
||||
import org.apache.lucene.search.SortedSetSelector;
|
||||
import org.apache.lucene.search.SortedSetSortField;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.ByteBlockPool;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
|
||||
import org.apache.lucene.util.BytesRefHash;
|
||||
import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
import org.apache.lucene.util.packed.PackedLongValues;
|
||||
|
||||
import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS;
|
||||
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
|
||||
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
|
||||
|
||||
/** Buffers up pending byte[]s per doc, deref and sorting via
|
||||
* int ord, then flushes when segment flushes. */
|
||||
class SortedSetDocValuesWriter extends DocValuesWriter {
|
||||
|
@ -47,6 +52,12 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
|
|||
private int currentUpto;
|
||||
private int maxCount;
|
||||
|
||||
private PackedLongValues finalOrds;
|
||||
private PackedLongValues finalOrdCounts;
|
||||
private int[] finalSortedValues;
|
||||
private int[] finalOrdMap;
|
||||
|
||||
|
||||
public SortedSetDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
|
||||
this.fieldInfo = fieldInfo;
|
||||
this.iwBytesUsed = iwBytesUsed;
|
||||
|
@ -136,17 +147,76 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
|
|||
bytesUsed = newBytesUsed;
|
||||
}
|
||||
|
||||
private long[][] sortDocValues(int maxDoc, Sorter.DocMap sortMap, SortedSetDocValues oldValues) throws IOException {
|
||||
long[][] ords = new long[maxDoc][];
|
||||
int docID;
|
||||
while ((docID = oldValues.nextDoc()) != NO_MORE_DOCS) {
|
||||
int newDocID = sortMap.oldToNew(docID);
|
||||
long[] docOrds = new long[1];
|
||||
int upto = 0;
|
||||
while (true) {
|
||||
long ord = oldValues.nextOrd();
|
||||
if (ord == NO_MORE_ORDS) {
|
||||
break;
|
||||
}
|
||||
if (upto == docOrds.length) {
|
||||
docOrds = ArrayUtil.grow(docOrds);
|
||||
}
|
||||
docOrds[upto++] = ord;
|
||||
}
|
||||
ords[newDocID] = Arrays.copyOfRange(docOrds, 0, upto);
|
||||
}
|
||||
return ords;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush(SegmentWriteState state, DocValuesConsumer dvConsumer) throws IOException {
|
||||
Sorter.DocComparator getDocComparator(int maxDoc, SortField sortField) throws IOException {
|
||||
assert sortField instanceof SortedSetSortField;
|
||||
assert finalOrds == null && finalOrdCounts == null && finalSortedValues == null && finalOrdMap == null;
|
||||
int valueCount = hash.size();
|
||||
finalOrds = pending.build();
|
||||
finalOrdCounts = pendingCounts.build();
|
||||
finalSortedValues = hash.sort();
|
||||
finalOrdMap = new int[valueCount];
|
||||
for (int ord = 0; ord < valueCount; ord++) {
|
||||
finalOrdMap[finalSortedValues[ord]] = ord;
|
||||
}
|
||||
|
||||
SortedSetSortField sf = (SortedSetSortField) sortField;
|
||||
final SortedSetDocValues dvs =
|
||||
new BufferedSortedSetDocValues(finalSortedValues, finalOrdMap, hash, finalOrds, finalOrdCounts, maxCount, docsWithField.iterator());
|
||||
return Sorter.getDocComparator(maxDoc, sf, () -> SortedSetSelector.wrap(dvs, sf.getSelector()), () -> null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush(SegmentWriteState state, Sorter.DocMap sortMap, DocValuesConsumer dvConsumer) throws IOException {
|
||||
final int valueCount = hash.size();
|
||||
final PackedLongValues ords = pending.build();
|
||||
final PackedLongValues ordCounts = pendingCounts.build();
|
||||
final PackedLongValues ords;
|
||||
final PackedLongValues ordCounts;
|
||||
final int[] sortedValues;
|
||||
final int[] ordMap;
|
||||
|
||||
final int[] sortedValues = hash.sort();
|
||||
final int[] ordMap = new int[valueCount];
|
||||
if (finalOrdCounts == null) {
|
||||
ords = pending.build();
|
||||
ordCounts = pendingCounts.build();
|
||||
sortedValues = hash.sort();
|
||||
ordMap = new int[valueCount];
|
||||
for(int ord=0;ord<valueCount;ord++) {
|
||||
ordMap[sortedValues[ord]] = ord;
|
||||
}
|
||||
} else {
|
||||
ords = finalOrds;
|
||||
ordCounts = finalOrdCounts;
|
||||
sortedValues = finalSortedValues;
|
||||
ordMap = finalOrdMap;
|
||||
}
|
||||
|
||||
for(int ord=0;ord<valueCount;ord++) {
|
||||
ordMap[sortedValues[ord]] = ord;
|
||||
final long[][] sorted;
|
||||
if (sortMap != null) {
|
||||
sorted = sortDocValues(state.segmentInfo.maxDoc(), sortMap,
|
||||
new BufferedSortedSetDocValues(sortedValues, ordMap, hash, ords, ordCounts, maxCount, docsWithField.iterator()));
|
||||
} else {
|
||||
sorted = null;
|
||||
}
|
||||
dvConsumer.addSortedSetField(fieldInfo,
|
||||
new EmptyDocValuesProducer() {
|
||||
|
@ -155,7 +225,13 @@ class SortedSetDocValuesWriter extends DocValuesWriter {
|
|||
if (fieldInfoIn != fieldInfo) {
|
||||
throw new IllegalArgumentException("wrong fieldInfo");
|
||||
}
|
||||
return new BufferedSortedSetDocValues(sortedValues, ordMap, hash, ords, ordCounts, maxCount, docsWithField.iterator());
|
||||
final SortedSetDocValues buf =
|
||||
new BufferedSortedSetDocValues(sortedValues, ordMap, hash, ords, ordCounts, maxCount, docsWithField.iterator());
|
||||
if (sorted == null) {
|
||||
return buf;
|
||||
} else {
|
||||
return new SortingLeafReader.SortingSortedSetDocValues(buf, sorted);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -238,21 +238,37 @@ final class Sorter {
|
|||
}
|
||||
}
|
||||
|
||||
static DocComparator getDocComparator(LeafReader reader, SortField sortField) throws IOException {
|
||||
return getDocComparator(reader.maxDoc(), sortField,
|
||||
() -> getOrWrapSorted(reader, sortField),
|
||||
() -> getOrWrapNumeric(reader, sortField));
|
||||
}
|
||||
|
||||
interface NumericDocValuesSupplier {
|
||||
NumericDocValues get() throws IOException;
|
||||
}
|
||||
|
||||
interface SortedDocValuesSupplier {
|
||||
SortedDocValues get() throws IOException;
|
||||
}
|
||||
|
||||
/** We cannot use the {@link FieldComparator} API because that API requires that you send it docIDs in order. Note that this API
|
||||
* allocates arrays[maxDoc] to hold the native values needed for comparison, but 1) they are transient (only alive while sorting this one
|
||||
* segment), and 2) in the typical index sorting case, they are only used to sort newly flushed segments, which will be smaller than
|
||||
* merged segments. */
|
||||
private static DocComparator getDocComparator(LeafReader reader, SortField sortField) throws IOException {
|
||||
static DocComparator getDocComparator(int maxDoc,
|
||||
SortField sortField,
|
||||
SortedDocValuesSupplier sortedProvider,
|
||||
NumericDocValuesSupplier numericProvider) throws IOException {
|
||||
|
||||
final int maxDoc = reader.maxDoc();
|
||||
final int reverseMul = sortField.getReverse() ? -1 : 1;
|
||||
final SortField.Type sortType = getSortFieldType(sortField);
|
||||
|
||||
switch(sortType) {
|
||||
|
||||
case STRING:
|
||||
case STRING:
|
||||
{
|
||||
final SortedDocValues sorted = getOrWrapSorted(reader, sortField);
|
||||
final SortedDocValues sorted = sortedProvider.get();
|
||||
final int missingOrd;
|
||||
if (sortField.getMissingValue() == SortField.STRING_LAST) {
|
||||
missingOrd = Integer.MAX_VALUE;
|
||||
|
@ -260,13 +276,13 @@ final class Sorter {
|
|||
missingOrd = Integer.MIN_VALUE;
|
||||
}
|
||||
|
||||
final int[] ords = new int[reader.maxDoc()];
|
||||
final int[] ords = new int[maxDoc];
|
||||
Arrays.fill(ords, missingOrd);
|
||||
int docID;
|
||||
while ((docID = sorted.nextDoc()) != NO_MORE_DOCS) {
|
||||
ords[docID] = sorted.ordValue();
|
||||
}
|
||||
|
||||
|
||||
return new DocComparator() {
|
||||
@Override
|
||||
public int compare(int docID1, int docID2) {
|
||||
|
@ -275,9 +291,9 @@ final class Sorter {
|
|||
};
|
||||
}
|
||||
|
||||
case LONG:
|
||||
case LONG:
|
||||
{
|
||||
final NumericDocValues dvs = getOrWrapNumeric(reader, sortField);
|
||||
final NumericDocValues dvs = numericProvider.get();
|
||||
long[] values = new long[maxDoc];
|
||||
if (sortField.getMissingValue() != null) {
|
||||
Arrays.fill(values, (Long) sortField.getMissingValue());
|
||||
|
@ -298,9 +314,9 @@ final class Sorter {
|
|||
};
|
||||
}
|
||||
|
||||
case INT:
|
||||
case INT:
|
||||
{
|
||||
final NumericDocValues dvs = getOrWrapNumeric(reader, sortField);
|
||||
final NumericDocValues dvs = numericProvider.get();
|
||||
int[] values = new int[maxDoc];
|
||||
if (sortField.getMissingValue() != null) {
|
||||
Arrays.fill(values, (Integer) sortField.getMissingValue());
|
||||
|
@ -322,9 +338,9 @@ final class Sorter {
|
|||
};
|
||||
}
|
||||
|
||||
case DOUBLE:
|
||||
case DOUBLE:
|
||||
{
|
||||
final NumericDocValues dvs = getOrWrapNumeric(reader, sortField);
|
||||
final NumericDocValues dvs = numericProvider.get();
|
||||
double[] values = new double[maxDoc];
|
||||
if (sortField.getMissingValue() != null) {
|
||||
Arrays.fill(values, (Double) sortField.getMissingValue());
|
||||
|
@ -345,9 +361,9 @@ final class Sorter {
|
|||
};
|
||||
}
|
||||
|
||||
case FLOAT:
|
||||
case FLOAT:
|
||||
{
|
||||
final NumericDocValues dvs = getOrWrapNumeric(reader, sortField);
|
||||
final NumericDocValues dvs = numericProvider.get();
|
||||
float[] values = new float[maxDoc];
|
||||
if (sortField.getMissingValue() != null) {
|
||||
Arrays.fill(values, (Float) sortField.getMissingValue());
|
||||
|
@ -368,11 +384,12 @@ final class Sorter {
|
|||
};
|
||||
}
|
||||
|
||||
default:
|
||||
throw new IllegalArgumentException("unhandled SortField.getType()=" + sortField.getType());
|
||||
default:
|
||||
throw new IllegalArgumentException("unhandled SortField.getType()=" + sortField.getType());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Returns a mapping from the old document ID to its new location in the
|
||||
* sorted index. Implementations can use the auxiliary
|
||||
|
@ -388,11 +405,15 @@ final class Sorter {
|
|||
DocMap sort(LeafReader reader) throws IOException {
|
||||
SortField fields[] = sort.getSort();
|
||||
final DocComparator comparators[] = new DocComparator[fields.length];
|
||||
|
||||
|
||||
for (int i = 0; i < fields.length; i++) {
|
||||
comparators[i] = getDocComparator(reader, fields[i]);
|
||||
}
|
||||
return sort(reader.maxDoc(), comparators);
|
||||
}
|
||||
|
||||
|
||||
DocMap sort(int maxDoc, DocComparator[] comparators) throws IOException {
|
||||
final DocComparator comparator = new DocComparator() {
|
||||
@Override
|
||||
public int compare(int docID1, int docID2) {
|
||||
|
@ -406,7 +427,7 @@ final class Sorter {
|
|||
}
|
||||
};
|
||||
|
||||
return sort(reader.maxDoc(), comparator);
|
||||
return sort(maxDoc, comparator);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -42,15 +42,14 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
|
|||
|
||||
/**
|
||||
* An {@link org.apache.lucene.index.LeafReader} which supports sorting documents by a given
|
||||
* {@link Sort}. This is package private and is only used by Lucene when it needs to merge
|
||||
* a newly flushed (unsorted) segment.
|
||||
* {@link Sort}. This is package private and is only used by Lucene fo BWC when it needs to merge
|
||||
* an unsorted flushed segment built by an older version (newly flushed segments are sorted since version 7.0).
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
||||
class SortingLeafReader extends FilterLeafReader {
|
||||
|
||||
private static class SortingFields extends FilterFields {
|
||||
static class SortingFields extends FilterFields {
|
||||
|
||||
private final Sorter.DocMap docMap;
|
||||
private final FieldInfos infos;
|
||||
|
@ -154,7 +153,7 @@ class SortingLeafReader extends FilterLeafReader {
|
|||
|
||||
}
|
||||
|
||||
private static class SortingBinaryDocValues extends BinaryDocValues {
|
||||
static class SortingBinaryDocValues extends BinaryDocValues {
|
||||
|
||||
private final CachedBinaryDVs dvs;
|
||||
private int docID = -1;
|
||||
|
@ -204,7 +203,7 @@ class SortingLeafReader extends FilterLeafReader {
|
|||
|
||||
private final Map<String,CachedNumericDVs> cachedNumericDVs = new HashMap<>();
|
||||
|
||||
private static class CachedNumericDVs {
|
||||
static class CachedNumericDVs {
|
||||
private final long[] values;
|
||||
private final BitSet docsWithField;
|
||||
|
||||
|
@ -216,7 +215,7 @@ class SortingLeafReader extends FilterLeafReader {
|
|||
|
||||
private final Map<String,CachedBinaryDVs> cachedBinaryDVs = new HashMap<>();
|
||||
|
||||
private static class CachedBinaryDVs {
|
||||
static class CachedBinaryDVs {
|
||||
// TODO: at least cutover to BytesRefArray here:
|
||||
private final BytesRef[] values;
|
||||
private final BitSet docsWithField;
|
||||
|
@ -229,7 +228,7 @@ class SortingLeafReader extends FilterLeafReader {
|
|||
|
||||
private final Map<String,int[]> cachedSortedDVs = new HashMap<>();
|
||||
|
||||
private static class SortingNumericDocValues extends NumericDocValues {
|
||||
static class SortingNumericDocValues extends NumericDocValues {
|
||||
|
||||
private final CachedNumericDVs dvs;
|
||||
private int docID = -1;
|
||||
|
@ -359,7 +358,7 @@ class SortingLeafReader extends FilterLeafReader {
|
|||
}
|
||||
}
|
||||
|
||||
private static class SortingSortedDocValues extends SortedDocValues {
|
||||
static class SortingSortedDocValues extends SortedDocValues {
|
||||
|
||||
private final SortedDocValues in;
|
||||
private final int[] ords;
|
||||
|
@ -436,7 +435,7 @@ class SortingLeafReader extends FilterLeafReader {
|
|||
// TODO: pack long[][] into an int[] (offset) and long[] instead:
|
||||
private final Map<String,long[][]> cachedSortedSetDVs = new HashMap<>();
|
||||
|
||||
private static class SortingSortedSetDocValues extends SortedSetDocValues {
|
||||
static class SortingSortedSetDocValues extends SortedSetDocValues {
|
||||
|
||||
private final SortedSetDocValues in;
|
||||
private final long[][] ords;
|
||||
|
@ -519,7 +518,7 @@ class SortingLeafReader extends FilterLeafReader {
|
|||
|
||||
private final Map<String,long[][]> cachedSortedNumericDVs = new HashMap<>();
|
||||
|
||||
private static class SortingSortedNumericDocValues extends SortedNumericDocValues {
|
||||
static class SortingSortedNumericDocValues extends SortedNumericDocValues {
|
||||
private final SortedNumericDocValues in;
|
||||
private final long[][] values;
|
||||
private int docID = -1;
|
||||
|
|
|
@ -0,0 +1,206 @@
|
|||
/*
|
||||
* 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.io.Reader;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.document.StoredField;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
final class SortingStoredFieldsConsumer extends StoredFieldsConsumer {
|
||||
TrackingTmpOutputDirectoryWrapper tmpDirectory;
|
||||
|
||||
SortingStoredFieldsConsumer(DocumentsWriterPerThread docWriter) {
|
||||
super(docWriter);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void initStoredFieldsWriter() throws IOException {
|
||||
if (writer == null) {
|
||||
this.tmpDirectory = new TrackingTmpOutputDirectoryWrapper(docWriter.directory);
|
||||
this.writer = docWriter.codec.storedFieldsFormat().fieldsWriter(tmpDirectory, docWriter.getSegmentInfo(),
|
||||
IOContext.DEFAULT);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
void flush(SegmentWriteState state, Sorter.DocMap sortMap) throws IOException {
|
||||
super.flush(state, sortMap);
|
||||
if (sortMap == null) {
|
||||
// we're lucky the index is already sorted, just rename the temporary file and return
|
||||
for (Map.Entry<String, String> entry : tmpDirectory.getTemporaryFiles().entrySet()) {
|
||||
tmpDirectory.rename(entry.getValue(), entry.getKey());
|
||||
}
|
||||
return;
|
||||
}
|
||||
StoredFieldsReader reader = docWriter.codec.storedFieldsFormat()
|
||||
.fieldsReader(tmpDirectory, state.segmentInfo, state.fieldInfos, IOContext.DEFAULT);
|
||||
StoredFieldsReader mergeReader = reader.getMergeInstance();
|
||||
StoredFieldsWriter sortWriter = docWriter.codec.storedFieldsFormat()
|
||||
.fieldsWriter(state.directory, state.segmentInfo, IOContext.DEFAULT);
|
||||
try {
|
||||
reader.checkIntegrity();
|
||||
CopyVisitor visitor = new CopyVisitor(sortWriter);
|
||||
for (int docID = 0; docID < state.segmentInfo.maxDoc(); docID++) {
|
||||
sortWriter.startDocument();
|
||||
mergeReader.visitDocument(sortMap.newToOld(docID), visitor);
|
||||
sortWriter.finishDocument();
|
||||
}
|
||||
sortWriter.finish(state.fieldInfos, state.segmentInfo.maxDoc());
|
||||
} finally {
|
||||
IOUtils.close(reader, sortWriter);
|
||||
IOUtils.deleteFiles(tmpDirectory,
|
||||
tmpDirectory.getTemporaryFiles().values());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
void abort() {
|
||||
try {
|
||||
super.abort();
|
||||
} finally {
|
||||
IOUtils.deleteFilesIgnoringExceptions(tmpDirectory,
|
||||
tmpDirectory.getTemporaryFiles().values());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A visitor that copies every field it sees in the provided {@link StoredFieldsWriter}.
|
||||
*/
|
||||
private static class CopyVisitor extends StoredFieldVisitor implements IndexableField {
|
||||
final StoredFieldsWriter writer;
|
||||
BytesRef binaryValue;
|
||||
String stringValue;
|
||||
Number numericValue;
|
||||
FieldInfo currentField;
|
||||
|
||||
|
||||
CopyVisitor(StoredFieldsWriter writer) {
|
||||
this.writer = writer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException {
|
||||
reset(fieldInfo);
|
||||
// TODO: can we avoid new BR here?
|
||||
binaryValue = new BytesRef(value);
|
||||
write();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stringField(FieldInfo fieldInfo, byte[] value) throws IOException {
|
||||
reset(fieldInfo);
|
||||
// TODO: can we avoid new String here?
|
||||
stringValue = new String(value, StandardCharsets.UTF_8);
|
||||
write();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void intField(FieldInfo fieldInfo, int value) throws IOException {
|
||||
reset(fieldInfo);
|
||||
numericValue = value;
|
||||
write();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void longField(FieldInfo fieldInfo, long value) throws IOException {
|
||||
reset(fieldInfo);
|
||||
numericValue = value;
|
||||
write();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void floatField(FieldInfo fieldInfo, float value) throws IOException {
|
||||
reset(fieldInfo);
|
||||
numericValue = value;
|
||||
write();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doubleField(FieldInfo fieldInfo, double value) throws IOException {
|
||||
reset(fieldInfo);
|
||||
numericValue = value;
|
||||
write();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Status needsField(FieldInfo fieldInfo) throws IOException {
|
||||
return Status.YES;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String name() {
|
||||
return currentField.name;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexableFieldType fieldType() {
|
||||
return StoredField.TYPE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef binaryValue() {
|
||||
return binaryValue;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String stringValue() {
|
||||
return stringValue;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Number numericValue() {
|
||||
return numericValue;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Reader readerValue() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float boost() {
|
||||
return 1F;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) {
|
||||
return null;
|
||||
}
|
||||
|
||||
void reset(FieldInfo field) {
|
||||
currentField = field;
|
||||
binaryValue = null;
|
||||
stringValue = null;
|
||||
numericValue = null;
|
||||
}
|
||||
|
||||
void write() throws IOException {
|
||||
writer.writeField(currentField, this);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,181 @@
|
|||
/*
|
||||
* 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.Iterator;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.TermVectorsReader;
|
||||
import org.apache.lucene.codecs.TermVectorsWriter;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.store.FlushInfo;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
final class SortingTermVectorsConsumer extends TermVectorsConsumer {
|
||||
TrackingTmpOutputDirectoryWrapper tmpDirectory;
|
||||
|
||||
public SortingTermVectorsConsumer(DocumentsWriterPerThread docWriter) {
|
||||
super(docWriter);
|
||||
}
|
||||
|
||||
@Override
|
||||
void flush(Map<String, TermsHashPerField> fieldsToFlush, final SegmentWriteState state, Sorter.DocMap sortMap) throws IOException {
|
||||
super.flush(fieldsToFlush, state, sortMap);
|
||||
if (tmpDirectory != null) {
|
||||
if (sortMap == null) {
|
||||
// we're lucky the index is already sorted, just rename the temporary file and return
|
||||
for (Map.Entry<String, String> entry : tmpDirectory.getTemporaryFiles().entrySet()) {
|
||||
tmpDirectory.rename(entry.getValue(), entry.getKey());
|
||||
}
|
||||
return;
|
||||
}
|
||||
TermVectorsReader reader = docWriter.codec.termVectorsFormat()
|
||||
.vectorsReader(tmpDirectory, state.segmentInfo, state.fieldInfos, IOContext.DEFAULT);
|
||||
TermVectorsReader mergeReader = reader.getMergeInstance();
|
||||
TermVectorsWriter writer = docWriter.codec.termVectorsFormat()
|
||||
.vectorsWriter(state.directory, state.segmentInfo, IOContext.DEFAULT);
|
||||
try {
|
||||
reader.checkIntegrity();
|
||||
for (int docID = 0; docID < state.segmentInfo.maxDoc(); docID++) {
|
||||
Fields vectors = mergeReader.get(sortMap.newToOld(docID));
|
||||
writeTermVectors(writer, vectors, state.fieldInfos);
|
||||
}
|
||||
writer.finish(state.fieldInfos, state.segmentInfo.maxDoc());
|
||||
} finally {
|
||||
IOUtils.close(reader, writer);
|
||||
IOUtils.deleteFiles(tmpDirectory,
|
||||
tmpDirectory.getTemporaryFiles().values());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
void initTermVectorsWriter() throws IOException {
|
||||
if (writer == null) {
|
||||
IOContext context = new IOContext(new FlushInfo(docWriter.getNumDocsInRAM(), docWriter.bytesUsed()));
|
||||
tmpDirectory = new TrackingTmpOutputDirectoryWrapper(docWriter.directory);
|
||||
writer = docWriter.codec.termVectorsFormat().vectorsWriter(tmpDirectory, docWriter.getSegmentInfo(), context);
|
||||
lastDocID = 0;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
try {
|
||||
super.abort();
|
||||
} finally {
|
||||
IOUtils.deleteFilesIgnoringExceptions(tmpDirectory,
|
||||
tmpDirectory.getTemporaryFiles().values());
|
||||
}
|
||||
}
|
||||
|
||||
/** Safe (but, slowish) default method to copy every vector field in the provided {@link TermVectorsWriter}. */
|
||||
private static void writeTermVectors(TermVectorsWriter writer, Fields vectors, FieldInfos fieldInfos) throws IOException {
|
||||
if (vectors == null) {
|
||||
writer.startDocument(0);
|
||||
writer.finishDocument();
|
||||
return;
|
||||
}
|
||||
|
||||
int numFields = vectors.size();
|
||||
if (numFields == -1) {
|
||||
// count manually! TODO: Maybe enforce that Fields.size() returns something valid?
|
||||
numFields = 0;
|
||||
for (final Iterator<String> it = vectors.iterator(); it.hasNext(); ) {
|
||||
it.next();
|
||||
numFields++;
|
||||
}
|
||||
}
|
||||
writer.startDocument(numFields);
|
||||
|
||||
String lastFieldName = null;
|
||||
|
||||
TermsEnum termsEnum = null;
|
||||
PostingsEnum docsAndPositionsEnum = null;
|
||||
|
||||
int fieldCount = 0;
|
||||
for(String fieldName : vectors) {
|
||||
fieldCount++;
|
||||
final FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldName);
|
||||
|
||||
assert lastFieldName == null || fieldName.compareTo(lastFieldName) > 0: "lastFieldName=" + lastFieldName + " fieldName=" + fieldName;
|
||||
lastFieldName = fieldName;
|
||||
|
||||
final Terms terms = vectors.terms(fieldName);
|
||||
if (terms == null) {
|
||||
// FieldsEnum shouldn't lie...
|
||||
continue;
|
||||
}
|
||||
|
||||
final boolean hasPositions = terms.hasPositions();
|
||||
final boolean hasOffsets = terms.hasOffsets();
|
||||
final boolean hasPayloads = terms.hasPayloads();
|
||||
assert !hasPayloads || hasPositions;
|
||||
|
||||
int numTerms = (int) terms.size();
|
||||
if (numTerms == -1) {
|
||||
// count manually. It is stupid, but needed, as Terms.size() is not a mandatory statistics function
|
||||
numTerms = 0;
|
||||
termsEnum = terms.iterator();
|
||||
while(termsEnum.next() != null) {
|
||||
numTerms++;
|
||||
}
|
||||
}
|
||||
|
||||
writer.startField(fieldInfo, numTerms, hasPositions, hasOffsets, hasPayloads);
|
||||
termsEnum = terms.iterator();
|
||||
|
||||
int termCount = 0;
|
||||
while(termsEnum.next() != null) {
|
||||
termCount++;
|
||||
|
||||
final int freq = (int) termsEnum.totalTermFreq();
|
||||
|
||||
writer.startTerm(termsEnum.term(), freq);
|
||||
|
||||
if (hasPositions || hasOffsets) {
|
||||
docsAndPositionsEnum = termsEnum.postings(docsAndPositionsEnum, PostingsEnum.OFFSETS | PostingsEnum.PAYLOADS);
|
||||
assert docsAndPositionsEnum != null;
|
||||
|
||||
final int docID = docsAndPositionsEnum.nextDoc();
|
||||
assert docID != DocIdSetIterator.NO_MORE_DOCS;
|
||||
assert docsAndPositionsEnum.freq() == freq;
|
||||
|
||||
for(int posUpto=0; posUpto<freq; posUpto++) {
|
||||
final int pos = docsAndPositionsEnum.nextPosition();
|
||||
final int startOffset = docsAndPositionsEnum.startOffset();
|
||||
final int endOffset = docsAndPositionsEnum.endOffset();
|
||||
|
||||
final BytesRef payload = docsAndPositionsEnum.getPayload();
|
||||
|
||||
assert !hasPositions || pos >= 0 ;
|
||||
writer.addPosition(pos, startOffset, endOffset, payload);
|
||||
}
|
||||
}
|
||||
writer.finishTerm();
|
||||
}
|
||||
assert termCount == numTerms;
|
||||
writer.finishField();
|
||||
}
|
||||
assert fieldCount == numFields;
|
||||
writer.finishDocument();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,85 @@
|
|||
/*
|
||||
* 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 org.apache.lucene.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
class StoredFieldsConsumer {
|
||||
final DocumentsWriterPerThread docWriter;
|
||||
StoredFieldsWriter writer;
|
||||
int lastDoc;
|
||||
|
||||
StoredFieldsConsumer(DocumentsWriterPerThread docWriter) {
|
||||
this.docWriter = docWriter;
|
||||
this.lastDoc = -1;
|
||||
}
|
||||
|
||||
protected void initStoredFieldsWriter() throws IOException {
|
||||
if (writer == null) {
|
||||
this.writer =
|
||||
docWriter.codec.storedFieldsFormat().fieldsWriter(docWriter.directory, docWriter.getSegmentInfo(),
|
||||
IOContext.DEFAULT);
|
||||
}
|
||||
}
|
||||
|
||||
void startDocument(int docID) throws IOException {
|
||||
assert lastDoc < docID;
|
||||
initStoredFieldsWriter();
|
||||
while (++lastDoc < docID) {
|
||||
writer.startDocument();
|
||||
writer.finishDocument();
|
||||
}
|
||||
writer.startDocument();
|
||||
}
|
||||
|
||||
void writeField(FieldInfo info, IndexableField field) throws IOException {
|
||||
writer.writeField(info, field);
|
||||
}
|
||||
|
||||
void finishDocument() throws IOException {
|
||||
writer.finishDocument();
|
||||
}
|
||||
|
||||
void finish(int maxDoc) throws IOException {
|
||||
while (lastDoc < maxDoc-1) {
|
||||
startDocument(lastDoc);
|
||||
finishDocument();
|
||||
++lastDoc;
|
||||
}
|
||||
}
|
||||
|
||||
void flush(SegmentWriteState state, Sorter.DocMap sortMap) throws IOException {
|
||||
try {
|
||||
writer.finish(state.fieldInfos, state.segmentInfo.maxDoc());
|
||||
} finally {
|
||||
IOUtils.close(writer);
|
||||
writer = null;
|
||||
}
|
||||
}
|
||||
|
||||
void abort() {
|
||||
if (writer != null) {
|
||||
IOUtils.closeWhileHandlingException(writer);
|
||||
writer = null;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -29,8 +29,7 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
final class TermVectorsConsumer extends TermsHash {
|
||||
|
||||
class TermVectorsConsumer extends TermsHash {
|
||||
TermVectorsWriter writer;
|
||||
|
||||
/** Scratch term used by TermVectorsConsumerPerField.finishDocument. */
|
||||
|
@ -54,7 +53,7 @@ final class TermVectorsConsumer extends TermsHash {
|
|||
}
|
||||
|
||||
@Override
|
||||
void flush(Map<String, TermsHashPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
|
||||
void flush(Map<String, TermsHashPerField> fieldsToFlush, final SegmentWriteState state, Sorter.DocMap sortMap) throws IOException {
|
||||
if (writer != null) {
|
||||
int numDocs = state.segmentInfo.maxDoc();
|
||||
assert numDocs > 0;
|
||||
|
@ -82,7 +81,7 @@ final class TermVectorsConsumer extends TermsHash {
|
|||
}
|
||||
}
|
||||
|
||||
private void initTermVectorsWriter() throws IOException {
|
||||
void initTermVectorsWriter() throws IOException {
|
||||
if (writer == null) {
|
||||
IOContext context = new IOContext(new FlushInfo(docWriter.getNumDocsInRAM(), docWriter.bytesUsed()));
|
||||
writer = docWriter.codec.termVectorsFormat().vectorsWriter(docWriter.directory, docWriter.getSegmentInfo(), context);
|
||||
|
|
|
@ -76,13 +76,13 @@ abstract class TermsHash {
|
|||
bytePool.reset(false, false);
|
||||
}
|
||||
|
||||
void flush(Map<String,TermsHashPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
|
||||
void flush(Map<String,TermsHashPerField> fieldsToFlush, final SegmentWriteState state, Sorter.DocMap sortMap) throws IOException {
|
||||
if (nextTermsHash != null) {
|
||||
Map<String,TermsHashPerField> nextChildFields = new HashMap<>();
|
||||
for (final Map.Entry<String,TermsHashPerField> entry : fieldsToFlush.entrySet()) {
|
||||
nextChildFields.put(entry.getKey(), entry.getValue().nextPerField);
|
||||
}
|
||||
nextTermsHash.flush(nextChildFields, state);
|
||||
nextTermsHash.flush(nextChildFields, state, sortMap);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,53 @@
|
|||
/*
|
||||
* 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.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FilterDirectory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
|
||||
final class TrackingTmpOutputDirectoryWrapper extends FilterDirectory {
|
||||
private final Map<String, String> fileNames = new HashMap();
|
||||
|
||||
TrackingTmpOutputDirectoryWrapper(Directory in) {
|
||||
super(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexOutput createOutput(String name, IOContext context) throws IOException {
|
||||
IndexOutput output = super.createTempOutput(name, "", context);
|
||||
fileNames.put(name, output.getName());
|
||||
return output;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexInput openInput(String name, IOContext context) throws IOException {
|
||||
String tmpName = fileNames.get(name);
|
||||
return super.openInput(tmpName, context);
|
||||
}
|
||||
|
||||
public Map<String, String> getTemporaryFiles() {
|
||||
return fileNames;
|
||||
}
|
||||
}
|
|
@ -1511,8 +1511,6 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
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");
|
||||
|
|
|
@ -68,8 +68,13 @@ public class AssertingLiveDocsFormat extends LiveDocsFormat {
|
|||
|
||||
@Override
|
||||
public void writeLiveDocs(MutableBits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException {
|
||||
assert bits instanceof AssertingMutableBits;
|
||||
MutableBits raw = (MutableBits) ((AssertingMutableBits)bits).in;
|
||||
MutableBits raw = bits;
|
||||
/**
|
||||
* bits is not necessarily an AssertingMutableBits because index sorting needs to wrap it in a sorted view.
|
||||
*/
|
||||
if (bits instanceof AssertingMutableBits) {
|
||||
raw = (MutableBits) ((AssertingMutableBits) bits).in;
|
||||
}
|
||||
check(raw, info.info.maxDoc(), info.getDelCount() + newDelCount);
|
||||
in.writeLiveDocs(raw, dir, info, newDelCount, context);
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue