LUCENE-7579: sort segments at flush too

This commit is contained in:
Mike McCandless 2016-12-20 06:45:06 -05:00
parent 48ef96d573
commit 4ccb9fbd2b
28 changed files with 1172 additions and 159 deletions

View File

@ -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

View 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;
}
}
}
}

View File

@ -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());
}

View File

@ -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:

View File

@ -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);
}
}
});
}

View File

@ -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);
}

View File

@ -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();
}

View File

@ -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;
}

View File

@ -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();
}

View File

@ -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;

View File

@ -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;
}
}
}

View File

@ -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);

View File

@ -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

View File

@ -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);
}
}
});
}

View File

@ -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);
}
}
}

View File

@ -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);
}
});
}

View File

@ -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);
}
}
});
}

View File

@ -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);
}
}
});
}

View File

@ -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);
}
/**

View File

@ -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;

View File

@ -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);
}
}
}

View File

@ -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();
}
}

View File

@ -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;
}
}
}

View File

@ -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);

View File

@ -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);
}
}

View File

@ -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;
}
}

View File

@ -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");

View File

@ -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);
}