You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2016/11/22 19:10:28 UTC
lucene-solr:master: LUCENE-7568: Optimize merging when index sorting
is used but the index is already sorted
Repository: lucene-solr
Updated Branches:
refs/heads/master bf9db95f2 -> 21735161d
LUCENE-7568: Optimize merging when index sorting is used but the index is already sorted
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/21735161
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/21735161
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/21735161
Branch: refs/heads/master
Commit: 21735161dcbdfcad52220d0389637c43f0d7989d
Parents: bf9db95
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Nov 22 14:10:16 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Nov 22 14:10:16 2016 -0500
----------------------------------------------------------------------
lucene/CHANGES.txt | 5 +
.../apache/lucene/codecs/DocValuesConsumer.java | 10 +-
.../org/apache/lucene/codecs/NormsConsumer.java | 2 +-
.../lucene/codecs/StoredFieldsWriter.java | 2 +-
.../apache/lucene/codecs/TermVectorsWriter.java | 2 +-
.../CompressingStoredFieldsWriter.java | 2 +-
.../CompressingTermVectorsWriter.java | 2 +-
.../codecs/lucene60/Lucene60PointsWriter.java | 2 +-
.../lucene/index/MappingMultiPostingsEnum.java | 2 +-
.../org/apache/lucene/index/MergeState.java | 76 ++++----
.../org/apache/lucene/index/MultiSorter.java | 14 +-
.../apache/lucene/index/TestIndexSorting.java | 184 ++++++++++++++++++-
12 files changed, 257 insertions(+), 46 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/21735161/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index ca39e65..e62a99d 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -117,6 +117,11 @@ Improvements
control how text is analyzed and converted into a query (Matt Weber
via Mike McCandless)
+Optimizations
+
+* LUCENE-7568: Optimize merging when index sorting is used but the
+ index is already sorted (Jim Ferenczi via Mike McCandless)
+
Other
* LUCENE-7546: Fixed references to benchmark wikipedia data and the Jenkins line-docs file
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/21735161/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
index e61724f..ba2f2aa 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
@@ -198,7 +198,7 @@ public abstract class DocValuesConsumer implements Closeable {
}
}
- final DocIDMerger<NumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
+ final DocIDMerger<NumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.needsIndexSort);
final long finalCost = cost;
@@ -296,7 +296,7 @@ public abstract class DocValuesConsumer implements Closeable {
}
}
- final DocIDMerger<BinaryDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
+ final DocIDMerger<BinaryDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.needsIndexSort);
final long finalCost = cost;
return new BinaryDocValues() {
@@ -397,7 +397,7 @@ public abstract class DocValuesConsumer implements Closeable {
final long finalCost = cost;
- final DocIDMerger<SortedNumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
+ final DocIDMerger<SortedNumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.needsIndexSort);
return new SortedNumericDocValues() {
@@ -555,7 +555,7 @@ public abstract class DocValuesConsumer implements Closeable {
final long finalCost = cost;
- final DocIDMerger<SortedDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
+ final DocIDMerger<SortedDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.needsIndexSort);
return new SortedDocValues() {
private int docID = -1;
@@ -721,7 +721,7 @@ public abstract class DocValuesConsumer implements Closeable {
subs.add(new SortedSetDocValuesSub(mergeState.docMaps[i], values, map.getGlobalOrds(i)));
}
- final DocIDMerger<SortedSetDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
+ final DocIDMerger<SortedSetDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.needsIndexSort);
final long finalCost = cost;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/21735161/lucene/core/src/java/org/apache/lucene/codecs/NormsConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/NormsConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/NormsConsumer.java
index 51abb69..7ad7a7c 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/NormsConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/NormsConsumer.java
@@ -130,7 +130,7 @@ public abstract class NormsConsumer implements Closeable {
}
}
- final DocIDMerger<NumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
+ final DocIDMerger<NumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.needsIndexSort);
return new NumericDocValues() {
private int docID = -1;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/21735161/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java
index 1e91271..80a9c49 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java
@@ -117,7 +117,7 @@ public abstract class StoredFieldsWriter implements Closeable {
subs.add(new StoredFieldsMergeSub(new MergeVisitor(mergeState, i), mergeState.docMaps[i], storedFieldsReader, mergeState.maxDocs[i]));
}
- final DocIDMerger<StoredFieldsMergeSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
+ final DocIDMerger<StoredFieldsMergeSub> docIDMerger = new DocIDMerger<>(subs, mergeState.needsIndexSort);
int docCount = 0;
while (true) {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/21735161/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
index 50cfca8..c8ad9f6 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
@@ -205,7 +205,7 @@ public abstract class TermVectorsWriter implements Closeable {
subs.add(new TermVectorsMergeSub(mergeState.docMaps[i], reader, mergeState.maxDocs[i]));
}
- final DocIDMerger<TermVectorsMergeSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
+ final DocIDMerger<TermVectorsMergeSub> docIDMerger = new DocIDMerger<>(subs, mergeState.needsIndexSort);
int docCount = 0;
while (true) {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/21735161/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
index d5bf4ad..1956ab7 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
@@ -486,7 +486,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
@Override
public int merge(MergeState mergeState) throws IOException {
- if (mergeState.segmentInfo.getIndexSort() != null) {
+ 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);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/21735161/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
index 9f8f44e..46a289a 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
@@ -730,7 +730,7 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
@Override
public int merge(MergeState mergeState) throws IOException {
- if (mergeState.segmentInfo.getIndexSort() != null) {
+ 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);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/21735161/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
index 0f3af85..6e404a1 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
@@ -134,7 +134,7 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
@Override
public void merge(MergeState mergeState) throws IOException {
- if (mergeState.segmentInfo.getIndexSort() != null) {
+ 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);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/21735161/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java b/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
index adadc40..d93c771 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
@@ -62,7 +62,7 @@ final class MappingMultiPostingsEnum extends PostingsEnum {
for(int i=0;i<allSubs.length;i++) {
allSubs[i] = new MappingPostingsSub(mergeState.docMaps[i]);
}
- this.docIDMerger = new DocIDMerger<MappingPostingsSub>(subs, allSubs.length, mergeState.segmentInfo.getIndexSort() != null);
+ this.docIDMerger = new DocIDMerger<MappingPostingsSub>(subs, allSubs.length, mergeState.needsIndexSort);
}
MappingMultiPostingsEnum reset(MultiPostingsEnum postingsEnum) throws IOException {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/21735161/lucene/core/src/java/org/apache/lucene/index/MergeState.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MergeState.java b/lucene/core/src/java/org/apache/lucene/index/MergeState.java
index fcaad51..fdedf3e 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MergeState.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MergeState.java
@@ -42,7 +42,7 @@ public class MergeState {
/** Maps document IDs from old segments to document IDs in the new segment */
public final DocMap[] docMaps;
- // Only used by IW when it must remap deletes that arrived against the merging segmetns while a merge was running:
+ // Only used by IW when it must remap deletes that arrived against the merging segments while a merge was running:
final DocMap[] leafDocMaps;
/** {@link SegmentInfo} of the newly merged segment. */
@@ -81,6 +81,9 @@ public class MergeState {
/** InfoStream for debugging messages. */
public final InfoStream infoStream;
+ /** Indicates if the index needs to be sorted **/
+ public boolean needsIndexSort;
+
/** Sole constructor. */
MergeState(List<CodecReader> originalReaders, SegmentInfo segmentInfo, InfoStream infoStream) throws IOException {
@@ -143,50 +146,58 @@ public class MergeState {
this.docMaps = buildDocMaps(readers, indexSort);
}
- private DocMap[] buildDocMaps(List<CodecReader> readers, Sort indexSort) throws IOException {
+ // Remap docIDs around deletions
+ private DocMap[] buildDeletionDocMaps(List<CodecReader> readers) {
+ int totalDocs = 0;
int numReaders = readers.size();
+ DocMap[] docMaps = new DocMap[numReaders];
- if (indexSort == null) {
- // no index sort ... we only must map around deletions, and rebase to the merged segment's docID space
-
- int totalDocs = 0;
- DocMap[] docMaps = new DocMap[numReaders];
+ for (int i = 0; i < numReaders; i++) {
+ LeafReader reader = readers.get(i);
+ Bits liveDocs = reader.getLiveDocs();
- // Remap docIDs around deletions:
- for (int i = 0; i < numReaders; i++) {
- LeafReader reader = readers.get(i);
- Bits liveDocs = reader.getLiveDocs();
+ final PackedLongValues delDocMap;
+ if (liveDocs != null) {
+ delDocMap = removeDeletes(reader.maxDoc(), liveDocs);
+ } else {
+ delDocMap = null;
+ }
- final PackedLongValues delDocMap;
- if (liveDocs != null) {
- delDocMap = removeDeletes(reader.maxDoc(), liveDocs);
- } else {
- delDocMap = null;
+ final int docBase = totalDocs;
+ docMaps[i] = new DocMap() {
+ @Override
+ public int get(int docID) {
+ if (liveDocs == null) {
+ return docBase + docID;
+ } else if (liveDocs.get(docID)) {
+ return docBase + (int) delDocMap.get(docID);
+ } else {
+ return -1;
+ }
}
+ };
+ totalDocs += reader.numDocs();
+ }
- final int docBase = totalDocs;
- docMaps[i] = new DocMap() {
- @Override
- public int get(int docID) {
- if (liveDocs == null) {
- return docBase + docID;
- } else if (liveDocs.get(docID)) {
- return docBase + (int) delDocMap.get(docID);
- } else {
- return -1;
- }
- }
- };
- totalDocs += reader.numDocs();
- }
+ return docMaps;
+ }
- return docMaps;
+ private DocMap[] buildDocMaps(List<CodecReader> readers, Sort indexSort) throws IOException {
+ if (indexSort == null) {
+ // no index sort ... we only must map around deletions, and rebase to the merged segment's docID space
+ return buildDeletionDocMaps(readers);
} else {
// do a merge sort of the incoming leaves:
long t0 = System.nanoTime();
DocMap[] result = MultiSorter.sort(indexSort, readers);
+ if (result == null) {
+ // already sorted so we can switch back to map around deletions
+ return buildDeletionDocMaps(readers);
+ } else {
+ needsIndexSort = true;
+ }
long t1 = System.nanoTime();
if (infoStream.isEnabled("SM")) {
infoStream.message("SM", String.format(Locale.ROOT, "%.2f msec to build merge sorted DocMaps", (t1-t0)/1000000.0));
@@ -233,6 +244,7 @@ public class MergeState {
if (infoStream.isEnabled("SM")) {
infoStream.message("SM", String.format(Locale.ROOT, "segment %s is not sorted; wrapping for sort %s now (%.2f msec to sort)", leaf, indexSort, msec));
}
+ needsIndexSort = true;
leaf = SlowCodecReaderWrapper.wrap(SortingLeafReader.wrap(new MergeReaderWrapper(leaf), sortDocMap));
leafDocMaps[readers.size()] = new DocMap() {
@Override
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/21735161/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java b/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
index 5ca6b65..630b65c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MultiSorter.java
@@ -33,7 +33,9 @@ import org.apache.lucene.util.packed.PackedLongValues;
final class MultiSorter {
/** Does a merge sort of the leaves of the incoming reader, returning {@link DocMap} to map each leaf's
- * documents into the merged segment. The documents for each incoming leaf reader must already be sorted by the same sort! */
+ * documents into the merged segment. The documents for each incoming leaf reader must already be sorted by the same sort!
+ * Returns null if the merge sort is not needed (segments are already in index sort order).
+ **/
static MergeState.DocMap[] sort(Sort sort, List<CodecReader> readers) throws IOException {
// TODO: optimize if only 1 reader is incoming, though that's a rare case
@@ -80,8 +82,15 @@ final class MultiSorter {
// merge sort:
int mappedDocID = 0;
+ int lastReaderIndex = 0;
+ boolean isSorted = true;
while (queue.size() != 0) {
LeafAndDocID top = queue.top();
+ if (lastReaderIndex > top.readerIndex) {
+ // merge sort is needed
+ isSorted = false;
+ }
+ lastReaderIndex = top.readerIndex;
builders[top.readerIndex].add(mappedDocID);
if (top.liveDocs == null || top.liveDocs.get(top.docID)) {
mappedDocID++;
@@ -97,6 +106,9 @@ final class MultiSorter {
queue.pop();
}
}
+ if (isSorted) {
+ return null;
+ }
MergeState.DocMap[] docMaps = new MergeState.DocMap[leafCount];
for(int i=0;i<leafCount;i++) {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/21735161/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
index 1bcac1a..26c4b95 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java
@@ -28,6 +28,7 @@ import java.util.Random;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer;
@@ -37,14 +38,19 @@ import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
+import org.apache.lucene.codecs.FilterCodec;
+import org.apache.lucene.codecs.PointsFormat;
+import org.apache.lucene.codecs.PointsReader;
+import org.apache.lucene.codecs.PointsWriter;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.BinaryPoint;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.DoubleDocValuesField;
-import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.FloatDocValuesField;
+import org.apache.lucene.document.IntPoint;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.document.SortedNumericDocValuesField;
@@ -79,6 +85,182 @@ import org.apache.lucene.util.TestUtil;
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
public class TestIndexSorting extends LuceneTestCase {
+ static class AssertingNeedsIndexSortCodec extends FilterCodec {
+ boolean needsIndexSort;
+ int numCalls;
+
+ AssertingNeedsIndexSortCodec() {
+ super(TestUtil.getDefaultCodec().getName(), TestUtil.getDefaultCodec());
+ }
+
+ @Override
+ public PointsFormat pointsFormat() {
+ final PointsFormat pf = delegate.pointsFormat();
+ return new PointsFormat() {
+ @Override
+ public PointsWriter fieldsWriter(SegmentWriteState state) throws IOException {
+ final PointsWriter writer = pf.fieldsWriter(state);
+ return new PointsWriter() {
+ @Override
+ public void merge(MergeState mergeState) throws IOException {
+ assertEquals(needsIndexSort, mergeState.needsIndexSort);
+ ++ numCalls;
+ writer.merge(mergeState);
+ }
+
+ @Override
+ public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
+ writer.writeField(fieldInfo, values);
+ }
+
+ @Override
+ public void finish() throws IOException {
+ writer.finish();
+ }
+
+ @Override
+ public void close() throws IOException {
+ writer.close();
+ }
+ };
+ }
+
+ @Override
+ public PointsReader fieldsReader(SegmentReadState state) throws IOException {
+ return pf.fieldsReader(state);
+ }
+ };
+ }
+ }
+
+ private static void assertNeedsIndexSortMerge(SortField sortField, Consumer<Document> defaultValueConsumer, Consumer<Document> randomValueConsumer) throws Exception {
+ Directory dir = newDirectory();
+ IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+ AssertingNeedsIndexSortCodec codec = new AssertingNeedsIndexSortCodec();
+ iwc.setCodec(codec);
+ Sort indexSort = new Sort(sortField,
+ new SortField("id", SortField.Type.INT));
+ iwc.setIndexSort(indexSort);
+ iwc.setMergePolicy(newLogMergePolicy());
+
+ // add already sorted documents
+ codec.numCalls = 0;
+ codec.needsIndexSort = false;
+ IndexWriter w = new IndexWriter(dir, iwc);
+ boolean withValues = random().nextBoolean();
+ for (int i = 100; i < 200; i++) {
+ Document doc = new Document();
+ doc.add(new StringField("id", Integer.toString(i), Store.YES));
+ doc.add(new NumericDocValuesField("id", i));
+ doc.add(new IntPoint("point", random().nextInt()));
+ if (withValues) {
+ defaultValueConsumer.accept(doc);
+ }
+ w.addDocument(doc);
+ if (i % 10 == 0) {
+ w.commit();
+ }
+ }
+ Set<Integer> deletedDocs = new HashSet<> ();
+ int num = random().nextInt(20);
+ for (int i = 0; i < num; i++) {
+ int nextDoc = random().nextInt(100);
+ w.deleteDocuments(new Term("id", Integer.toString(nextDoc)));
+ deletedDocs.add(nextDoc);
+ }
+ w.commit();
+ w.waitForMerges();
+ w.forceMerge(1);
+ assertTrue(codec.numCalls > 0);
+
+
+ // merge sort is needed
+ codec.numCalls = 0;
+ codec.needsIndexSort = true;
+ for (int i = 10; i >= 0; i--) {
+ Document doc = new Document();
+ doc.add(new StringField("id", Integer.toString(i), Store.YES));
+ doc.add(new NumericDocValuesField("id", i));
+ doc.add(new IntPoint("point", random().nextInt()));
+ if (withValues) {
+ defaultValueConsumer.accept(doc);
+ }
+ w.addDocument(doc);
+ w.commit();
+ }
+ w.commit();
+ w.waitForMerges();
+ w.forceMerge(1);
+ assertTrue(codec.numCalls > 0);
+
+ // segment sort is needed
+ codec.needsIndexSort = true;
+ codec.numCalls = 0;
+ for (int i = 200; i < 300; i++) {
+ Document doc = new Document();
+ doc.add(new StringField("id", Integer.toString(i), Store.YES));
+ doc.add(new NumericDocValuesField("id", i));
+ doc.add(new IntPoint("point", random().nextInt()));
+ randomValueConsumer.accept(doc);
+ w.addDocument(doc);
+ if (i % 10 == 0) {
+ w.commit();
+ }
+ }
+ w.commit();
+ w.waitForMerges();
+ w.forceMerge(1);
+ assertTrue(codec.numCalls > 0);
+
+ w.close();
+ dir.close();
+ }
+
+ public void testNumericAlreadySorted() throws Exception {
+ assertNeedsIndexSortMerge(new SortField("foo", SortField.Type.INT),
+ (doc) -> doc.add(new NumericDocValuesField("foo", 0)),
+ (doc) -> doc.add(new NumericDocValuesField("foo", random().nextInt())));
+ }
+
+ public void testStringAlreadySorted() throws Exception {
+ assertNeedsIndexSortMerge(new SortField("foo", SortField.Type.STRING),
+ (doc) -> doc.add(new SortedDocValuesField("foo", new BytesRef("default"))),
+ (doc) -> doc.add(new SortedDocValuesField("foo", TestUtil.randomBinaryTerm(random()))));
+ }
+
+ public void testMultiValuedNumericAlreadySorted() throws Exception {
+ assertNeedsIndexSortMerge(new SortedNumericSortField("foo", SortField.Type.INT),
+ (doc) -> {
+ doc.add(new SortedNumericDocValuesField("foo", Integer.MIN_VALUE));
+ int num = random().nextInt(5);
+ for (int j = 0; j < num; j++) {
+ doc.add(new SortedNumericDocValuesField("foo", random().nextInt()));
+ }
+ },
+ (doc) -> {
+ int num = random().nextInt(5);
+ for (int j = 0; j < num; j++) {
+ doc.add(new SortedNumericDocValuesField("foo", random().nextInt()));
+ }
+ });
+ }
+
+ public void testMultiValuedStringAlreadySorted() throws Exception {
+ assertNeedsIndexSortMerge(new SortedSetSortField("foo", false),
+ (doc) -> {
+ doc.add(new SortedSetDocValuesField("foo", new BytesRef("")));
+ int num = random().nextInt(5);
+ for (int j = 0; j < num; j++) {
+ doc.add(new SortedSetDocValuesField("foo", TestUtil.randomBinaryTerm(random())));
+ }
+ },
+ (doc) -> {
+ int num = random().nextInt(5);
+ for (int j = 0; j < num; j++) {
+ doc.add(new SortedSetDocValuesField("foo", TestUtil.randomBinaryTerm(random())));
+ }
+ });
+ }
public void testBasicString() throws Exception {
Directory dir = newDirectory();