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/06/11 15:49:21 UTC
[05/21] lucene-solr:branch_6x: LUCENE-6766: initial patch
LUCENE-6766: initial patch
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/fdc220ee
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/fdc220ee
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/fdc220ee
Branch: refs/heads/branch_6x
Commit: fdc220eeb423f97a3aa9b469f8bf8dbb118cc715
Parents: 7448abb
Author: Mike McCandless <mi...@apache.org>
Authored: Wed May 4 05:37:37 2016 -0400
Committer: Mike McCandless <mi...@apache.org>
Committed: Sat Jun 11 11:48:39 2016 -0400
----------------------------------------------------------------------
.../simpletext/SimpleTextFieldsWriter.java | 2 +
.../simpletext/SimpleTextSegmentInfoFormat.java | 144 ++-
.../SimpleTextStoredFieldsWriter.java | 1 -
.../apache/lucene/codecs/DocValuesConsumer.java | 510 +++++-----
.../org/apache/lucene/codecs/PointsWriter.java | 3 +-
.../lucene/codecs/StoredFieldsWriter.java | 60 +-
.../apache/lucene/codecs/TermVectorsWriter.java | 78 +-
.../lucene50/Lucene50SegmentInfoFormat.java | 7 +-
.../lucene/codecs/lucene60/Lucene60Codec.java | 2 +
.../codecs/lucene60/Lucene60PointsWriter.java | 4 +-
.../org/apache/lucene/index/CheckIndex.java | 3 +
.../org/apache/lucene/index/DocIDMerger.java | 173 ++++
.../lucene/index/DocumentsWriterPerThread.java | 2 +-
.../apache/lucene/index/FilterCodecReader.java | 6 +
.../apache/lucene/index/FilterLeafReader.java | 7 +
.../org/apache/lucene/index/IndexWriter.java | 109 ++-
.../apache/lucene/index/IndexWriterConfig.java | 26 +-
.../org/apache/lucene/index/LeafReader.java | 4 +
.../lucene/index/LiveIndexWriterConfig.java | 13 +
.../lucene/index/MappingMultiPostingsEnum.java | 121 ++-
.../org/apache/lucene/index/MergePolicy.java | 61 +-
.../apache/lucene/index/MergeReaderWrapper.java | 266 ++++++
.../org/apache/lucene/index/MergeState.java | 222 +++--
.../org/apache/lucene/index/MultiFields.java | 2 +
.../apache/lucene/index/MultiPostingsEnum.java | 7 +-
.../org/apache/lucene/index/MultiSorter.java | 221 +++++
.../apache/lucene/index/ParallelLeafReader.java | 18 +
.../org/apache/lucene/index/SegmentInfo.java | 25 +-
.../org/apache/lucene/index/SegmentReader.java | 6 +
.../lucene/index/SlowCodecReaderWrapper.java | 11 +
.../java/org/apache/lucene/index/Sorter.java | 289 ++++++
.../apache/lucene/index/SortingLeafReader.java | 918 ++++++++++++++++++
.../src/java/org/apache/lucene/search/Sort.java | 3 +
.../org/apache/lucene/util/bkd/BKDWriter.java | 16 +-
.../org/apache/lucene/index/TestCodecs.java | 4 +-
.../index/TestDemoParallelLeafReader.java | 38 +-
.../test/org/apache/lucene/index/TestDoc.java | 2 +-
.../apache/lucene/index/TestDocIDMerger.java | 179 ++++
.../apache/lucene/index/TestIndexSorting.java | 792 ++++++++++++++++
.../apache/lucene/index/TestIndexWriter.java | 3 +
.../apache/lucene/index/TestSegmentInfos.java | 12 +-
.../apache/lucene/index/TestSegmentMerger.java | 24 +-
.../org/apache/lucene/util/bkd/TestBKD.java | 23 +-
.../org/apache/lucene/index/IndexSplitter.java | 2 +-
.../apache/lucene/index/MergeReaderWrapper.java | 259 -----
.../index/SlowCompositeReaderWrapper.java | 11 +
.../java/org/apache/lucene/index/Sorter.java | 287 ------
.../apache/lucene/index/SortingLeafReader.java | 940 -------------------
.../apache/lucene/index/SortingMergePolicy.java | 264 ------
.../search/BlockJoinComparatorSource.java | 5 +-
.../EarlyTerminatingSortingCollector.java | 46 +-
.../apache/lucene/index/IndexSortingTest.java | 89 --
.../org/apache/lucene/index/SorterTestBase.java | 405 --------
.../lucene/index/SortingLeafReaderTest.java | 73 --
.../lucene/index/TestSortingMergePolicy.java | 201 ----
.../search/TestDiversifiedTopDocsCollector.java | 4 +-
.../TestEarlyTerminatingSortingCollector.java | 74 +-
.../index/BaseCompoundFormatTestCase.java | 2 +-
.../index/BaseFieldInfoFormatTestCase.java | 2 +-
.../index/BaseIndexFileFormatTestCase.java | 2 +-
.../index/BaseSegmentInfoFormatTestCase.java | 71 +-
.../lucene/index/MockRandomMergePolicy.java | 49 +-
.../lucene/index/RandomPostingsTester.java | 2 +-
.../org/apache/lucene/search/QueryUtils.java | 5 +
64 files changed, 3932 insertions(+), 3278 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java
index 3b026be..2023552 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java
@@ -36,6 +36,7 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
private IndexOutput out;
private final BytesRefBuilder scratch = new BytesRefBuilder();
private final SegmentWriteState writeState;
+ final String segment;
final static BytesRef END = new BytesRef("END");
final static BytesRef FIELD = new BytesRef("field ");
@@ -49,6 +50,7 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
public SimpleTextFieldsWriter(SegmentWriteState writeState) throws IOException {
final String fileName = SimpleTextPostingsFormat.getPostingsFileName(writeState.segmentInfo.name, writeState.segmentSuffix);
+ segment = writeState.segmentInfo.name;
out = writeState.directory.createOutput(fileName, writeState.context);
this.writeState = writeState;
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
index 0823a88..594fccf 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
@@ -31,6 +31,8 @@ import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
@@ -59,6 +61,11 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
final static BytesRef SI_NUM_FILES = new BytesRef(" files ");
final static BytesRef SI_FILE = new BytesRef(" file ");
final static BytesRef SI_ID = new BytesRef(" id ");
+ final static BytesRef SI_SORT = new BytesRef(" sort ");
+ final static BytesRef SI_SORT_FIELD = new BytesRef(" field ");
+ final static BytesRef SI_SORT_TYPE = new BytesRef(" type ");
+ final static BytesRef SI_SORT_REVERSE = new BytesRef(" reverse ");
+ final static BytesRef SI_SORT_MISSING = new BytesRef(" missing ");
public static final String SI_EXTENSION = "si";
@@ -137,10 +144,93 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
+ ", got: " + StringHelper.idToString(id), input);
}
+ SimpleTextUtil.readLine(input, scratch);
+ assert StringHelper.startsWith(scratch.get(), SI_SORT);
+ final int numSortFields = Integer.parseInt(readString(SI_SORT.length, scratch));
+ SortField[] sortField = new SortField[numSortFields];
+ for (int i = 0; i < numSortFields; ++i) {
+ SimpleTextUtil.readLine(input, scratch);
+ assert StringHelper.startsWith(scratch.get(), SI_SORT_FIELD);
+ final String field = readString(SI_SORT_FIELD.length, scratch);
+
+ SimpleTextUtil.readLine(input, scratch);
+ assert StringHelper.startsWith(scratch.get(), SI_SORT_TYPE);
+ final String typeAsString = readString(SI_SORT_TYPE.length, scratch);
+
+ final SortField.Type type;
+ switch (typeAsString) {
+ case "string":
+ type = SortField.Type.STRING;
+ break;
+ case "long":
+ type = SortField.Type.LONG;
+ break;
+ case "int":
+ type = SortField.Type.INT;
+ break;
+ default:
+ throw new CorruptIndexException("unable to parse sort type string: " + typeAsString, input);
+ }
+
+ SimpleTextUtil.readLine(input, scratch);
+ assert StringHelper.startsWith(scratch.get(), SI_SORT_REVERSE);
+ final boolean reverse = Boolean.parseBoolean(readString(SI_SORT_REVERSE.length, scratch));
+
+ SimpleTextUtil.readLine(input, scratch);
+ assert StringHelper.startsWith(scratch.get(), SI_SORT_MISSING);
+ final String missingLastAsString = readString(SI_SORT_MISSING.length, scratch);
+ final Object missingValue;
+ switch (type) {
+ case STRING:
+ switch (missingLastAsString) {
+ case "null":
+ missingValue = null;
+ break;
+ case "first":
+ missingValue = SortField.STRING_FIRST;
+ break;
+ case "last":
+ missingValue = SortField.STRING_LAST;
+ break;
+ default:
+ throw new CorruptIndexException("unable to parse missing string: " + typeAsString, input);
+ }
+ break;
+ case LONG:
+ switch (missingLastAsString) {
+ case "null":
+ missingValue = null;
+ break;
+ default:
+ missingValue = Long.parseLong(missingLastAsString);
+ break;
+ }
+ break;
+ case INT:
+ switch (missingLastAsString) {
+ case "null":
+ missingValue = null;
+ break;
+ default:
+ missingValue = Integer.parseInt(missingLastAsString);
+ break;
+ }
+ break;
+ default:
+ throw new AssertionError();
+ }
+ sortField[i] = new SortField(field, type, reverse);
+ if (missingValue != null) {
+ sortField[i].setMissingValue(missingValue);
+ }
+ }
+ Sort indexSort = sortField.length == 0 ? null : new Sort(sortField);
+
SimpleTextUtil.checkFooter(input);
SegmentInfo info = new SegmentInfo(directory, version, segmentName, docCount,
- isCompoundFile, null, Collections.unmodifiableMap(diagnostics), id, Collections.unmodifiableMap(attributes));
+ isCompoundFile, null, Collections.unmodifiableMap(diagnostics),
+ id, Collections.unmodifiableMap(attributes), indexSort);
info.setFiles(files);
return info;
}
@@ -223,6 +313,58 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat {
SimpleTextUtil.write(output, new BytesRef(si.getId()));
SimpleTextUtil.writeNewline(output);
+ Sort indexSort = si.getIndexSort();
+ SimpleTextUtil.write(output, SI_SORT);
+ final int numSortFields = indexSort == null ? 0 : indexSort.getSort().length;
+ SimpleTextUtil.write(output, Integer.toString(numSortFields), scratch);
+ SimpleTextUtil.writeNewline(output);
+ for (int i = 0; i < numSortFields; ++i) {
+ final SortField sortField = indexSort.getSort()[i];
+
+ SimpleTextUtil.write(output, SI_SORT_FIELD);
+ SimpleTextUtil.write(output, sortField.getField(), scratch);
+ SimpleTextUtil.writeNewline(output);
+
+ SimpleTextUtil.write(output, SI_SORT_TYPE);
+ final String sortType;
+ switch (sortField.getType()) {
+ case STRING:
+ sortType = "string";
+ break;
+ case LONG:
+ sortType = "long";
+ break;
+ case INT:
+ sortType = "int";
+ break;
+ default:
+ throw new IllegalStateException("Unexpected sort type: " + sortField.getType());
+ }
+ SimpleTextUtil.write(output, sortType, scratch);
+ SimpleTextUtil.writeNewline(output);
+
+ SimpleTextUtil.write(output, SI_SORT_REVERSE);
+ SimpleTextUtil.write(output, Boolean.toString(sortField.getReverse()), scratch);
+ SimpleTextUtil.writeNewline(output);
+
+ SimpleTextUtil.write(output, SI_SORT_MISSING);
+ final Object missingValue = sortField.getMissingValue();
+ final String missing;
+ if (missingValue == null) {
+ missing = "null";
+ } else if (missingValue == SortField.STRING_FIRST) {
+ missing = "first";
+ } else if (missingValue == SortField.STRING_LAST) {
+ missing = "last";
+ } else if (missingValue instanceof Long) {
+ missing = Long.toString((Long) missingValue);
+ } else {
+ throw new IllegalStateException("Unexpected missing sort value: " + missingValue);
+ }
+ SimpleTextUtil.write(output, missing, scratch);
+ SimpleTextUtil.writeNewline(output);
+ }
+
SimpleTextUtil.writeChecksum(output, scratch);
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java
index 00259b8..b59114a 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java
@@ -143,7 +143,6 @@ public class SimpleTextStoredFieldsWriter extends StoredFieldsWriter {
} else {
write(TYPE_STRING);
newLine();
-
write(VALUE);
write(field.stringValue());
newLine();
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/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 90abf2a..52bf9b2 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
@@ -16,7 +16,6 @@
*/
package org.apache.lucene.codecs;
-
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
@@ -25,12 +24,13 @@ import java.util.List;
import java.util.NoSuchElementException;
import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DocIDMerger;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FilteredTermsEnum;
import org.apache.lucene.index.MergeState;
-import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.MultiDocValues.OrdinalMap;
-import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentWriteState; // javadocs
import org.apache.lucene.index.SortedDocValues;
@@ -44,6 +44,8 @@ import org.apache.lucene.util.LongBitSet;
import org.apache.lucene.util.LongValues;
import org.apache.lucene.util.packed.PackedInts;
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
/**
* Abstract API that consumes numeric, binary and
* sorted docvalues. Concrete implementations of this
@@ -240,6 +242,32 @@ public abstract class DocValuesConsumer implements Closeable {
}
}
}
+
+ /** Tracks state of one numeric sub-reader that we are merging */
+ private static class NumericDocValuesSub extends DocIDMerger.Sub {
+
+ private final NumericDocValues values;
+ private final Bits docsWithField;
+ private int docID = -1;
+ private final int maxDoc;
+
+ public NumericDocValuesSub(MergeState.DocMap docMap, Bits liveDocs, NumericDocValues values, Bits docsWithField, int maxDoc) {
+ super(docMap, liveDocs);
+ this.values = values;
+ this.docsWithField = docsWithField;
+ this.maxDoc = maxDoc;
+ }
+
+ @Override
+ public int nextDoc() {
+ docID++;
+ if (docID == maxDoc) {
+ return NO_MORE_DOCS;
+ } else {
+ return docID;
+ }
+ }
+ }
/**
* Merges the numeric docvalues from <code>toMerge</code>.
@@ -248,20 +276,23 @@ public abstract class DocValuesConsumer implements Closeable {
* an Iterable that merges and filters deleted documents on the fly.
*/
public void mergeNumericField(final FieldInfo fieldInfo, final MergeState mergeState, final List<NumericDocValues> toMerge, final List<Bits> docsWithField) throws IOException {
-
addNumericField(fieldInfo,
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
+
+ // We must make a new DocIDMerger for each iterator:
+ List<NumericDocValuesSub> subs = new ArrayList<>();
+ assert mergeState.docMaps.length == toMerge.size();
+ for(int i=0;i<toMerge.size();i++) {
+ subs.add(new NumericDocValuesSub(mergeState.docMaps[i], mergeState.liveDocs[i], toMerge.get(i), docsWithField.get(i), mergeState.maxDocs[i]));
+ }
+
+ final DocIDMerger<NumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
+
return new Iterator<Number>() {
- int readerUpto = -1;
- int docIDUpto;
long nextValue;
boolean nextHasValue;
- int currentMaxDoc;
- NumericDocValues currentValues;
- Bits currentLiveDocs;
- Bits currentDocsWithField;
boolean nextIsSet;
@Override
@@ -276,7 +307,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public Number next() {
- if (!hasNext()) {
+ if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@@ -286,35 +317,14 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
- if (readerUpto == toMerge.size()) {
+ NumericDocValuesSub sub = docIDMerger.next();
+ if (sub == null) {
return false;
}
-
- if (docIDUpto == currentMaxDoc) {
- readerUpto++;
- if (readerUpto < toMerge.size()) {
- currentValues = toMerge.get(readerUpto);
- currentDocsWithField = docsWithField.get(readerUpto);
- currentLiveDocs = mergeState.liveDocs[readerUpto];
- currentMaxDoc = mergeState.maxDocs[readerUpto];
- }
- docIDUpto = 0;
- continue;
- }
-
- if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
- nextIsSet = true;
- nextValue = currentValues.get(docIDUpto);
- if (nextValue == 0 && currentDocsWithField.get(docIDUpto) == false) {
- nextHasValue = false;
- } else {
- nextHasValue = true;
- }
- docIDUpto++;
- return true;
- }
-
- docIDUpto++;
+ nextIsSet = true;
+ nextValue = sub.values.get(sub.docID);
+ nextHasValue = nextValue != 0 || sub.docsWithField.get(sub.docID);
+ return true;
}
}
};
@@ -322,6 +332,32 @@ public abstract class DocValuesConsumer implements Closeable {
});
}
+ /** Tracks state of one binary sub-reader that we are merging */
+ private static class BinaryDocValuesSub extends DocIDMerger.Sub {
+
+ private final BinaryDocValues values;
+ private final Bits docsWithField;
+ private int docID = -1;
+ private final int maxDoc;
+
+ public BinaryDocValuesSub(MergeState.DocMap docMap, Bits liveDocs, BinaryDocValues values, Bits docsWithField, int maxDoc) {
+ super(docMap, liveDocs);
+ this.values = values;
+ this.docsWithField = docsWithField;
+ this.maxDoc = maxDoc;
+ }
+
+ @Override
+ public int nextDoc() {
+ docID++;
+ if (docID == maxDoc) {
+ return NO_MORE_DOCS;
+ } else {
+ return docID;
+ }
+ }
+ }
+
/**
* Merges the binary docvalues from <code>toMerge</code>.
* <p>
@@ -329,20 +365,23 @@ public abstract class DocValuesConsumer implements Closeable {
* an Iterable that merges and filters deleted documents on the fly.
*/
public void mergeBinaryField(FieldInfo fieldInfo, final MergeState mergeState, final List<BinaryDocValues> toMerge, final List<Bits> docsWithField) throws IOException {
-
addBinaryField(fieldInfo,
new Iterable<BytesRef>() {
@Override
public Iterator<BytesRef> iterator() {
+
+ // We must make a new DocIDMerger for each iterator:
+ List<BinaryDocValuesSub> subs = new ArrayList<>();
+ assert mergeState.docMaps.length == toMerge.size();
+ for(int i=0;i<toMerge.size();i++) {
+ subs.add(new BinaryDocValuesSub(mergeState.docMaps[i], mergeState.liveDocs[i], toMerge.get(i), docsWithField.get(i), mergeState.maxDocs[i]));
+ }
+
+ final DocIDMerger<BinaryDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
+
return new Iterator<BytesRef>() {
- int readerUpto = -1;
- int docIDUpto;
BytesRef nextValue;
BytesRef nextPointer; // points to null if missing, or nextValue
- int currentMaxDoc;
- BinaryDocValues currentValues;
- Bits currentLiveDocs;
- Bits currentDocsWithField;
boolean nextIsSet;
@Override
@@ -357,7 +396,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public BytesRef next() {
- if (!hasNext()) {
+ if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@@ -367,42 +406,49 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
- if (readerUpto == toMerge.size()) {
- return false;
- }
-
- if (docIDUpto == currentMaxDoc) {
- readerUpto++;
- if (readerUpto < toMerge.size()) {
- currentValues = toMerge.get(readerUpto);
- currentDocsWithField = docsWithField.get(readerUpto);
- currentLiveDocs = mergeState.liveDocs[readerUpto];
- currentMaxDoc = mergeState.maxDocs[readerUpto];
- }
- docIDUpto = 0;
- continue;
- }
-
- if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
- nextIsSet = true;
- if (currentDocsWithField.get(docIDUpto)) {
- nextValue = currentValues.get(docIDUpto);
- nextPointer = nextValue;
- } else {
- nextPointer = null;
- }
- docIDUpto++;
- return true;
+ BinaryDocValuesSub sub = docIDMerger.next();
+ if (sub == null) {
+ return false;
+ }
+ nextIsSet = true;
+ if (sub.docsWithField.get(sub.docID)) {
+ nextPointer = nextValue = sub.values.get(sub.docID);
+ } else {
+ nextPointer = null;
+ }
+ return true;
}
-
- docIDUpto++;
}
- }
};
}
});
}
+ /** Tracks state of one sorted numeric sub-reader that we are merging */
+ private static class SortedNumericDocValuesSub extends DocIDMerger.Sub {
+
+ private final SortedNumericDocValues values;
+ private int docID = -1;
+ private final int maxDoc;
+
+ public SortedNumericDocValuesSub(MergeState.DocMap docMap, Bits liveDocs, SortedNumericDocValues values, int maxDoc) {
+ super(docMap, liveDocs);
+ this.values = values;
+ this.maxDoc = maxDoc;
+ }
+
+ @Override
+ public int nextDoc() {
+ docID++;
+ if (docID == maxDoc) {
+ return NO_MORE_DOCS;
+ } else {
+ values.setDocument(docID);
+ return docID;
+ }
+ }
+ }
+
/**
* Merges the sorted docvalues from <code>toMerge</code>.
* <p>
@@ -410,21 +456,24 @@ public abstract class DocValuesConsumer implements Closeable {
* iterables that filter deleted documents.
*/
public void mergeSortedNumericField(FieldInfo fieldInfo, final MergeState mergeState, List<SortedNumericDocValues> toMerge) throws IOException {
- final int numReaders = toMerge.size();
- final SortedNumericDocValues dvs[] = toMerge.toArray(new SortedNumericDocValues[numReaders]);
- // step 3: add field
addSortedNumericField(fieldInfo,
// doc -> value count
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
+
+ // We must make a new DocIDMerger for each iterator:
+ List<SortedNumericDocValuesSub> subs = new ArrayList<>();
+ assert mergeState.docMaps.length == toMerge.size();
+ for(int i=0;i<toMerge.size();i++) {
+ subs.add(new SortedNumericDocValuesSub(mergeState.docMaps[i], mergeState.liveDocs[i], toMerge.get(i), mergeState.maxDocs[i]));
+ }
+
+ final DocIDMerger<SortedNumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
+
return new Iterator<Number>() {
- int readerUpto = -1;
- int docIDUpto;
int nextValue;
- int currentMaxDoc;
- Bits currentLiveDocs;
boolean nextIsSet;
@Override
@@ -439,7 +488,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public Number next() {
- if (!hasNext()) {
+ if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@@ -449,30 +498,13 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
- if (readerUpto == numReaders) {
+ SortedNumericDocValuesSub sub = docIDMerger.next();
+ if (sub == null) {
return false;
}
-
- if (docIDUpto == currentMaxDoc) {
- readerUpto++;
- if (readerUpto < numReaders) {
- currentLiveDocs = mergeState.liveDocs[readerUpto];
- currentMaxDoc = mergeState.maxDocs[readerUpto];
- }
- docIDUpto = 0;
- continue;
- }
-
- if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
- nextIsSet = true;
- SortedNumericDocValues dv = dvs[readerUpto];
- dv.setDocument(docIDUpto);
- nextValue = dv.count();
- docIDUpto++;
- return true;
- }
-
- docIDUpto++;
+ nextIsSet = true;
+ nextValue = sub.values.count();
+ return true;
}
}
};
@@ -482,15 +514,21 @@ public abstract class DocValuesConsumer implements Closeable {
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
+ // We must make a new DocIDMerger for each iterator:
+ List<SortedNumericDocValuesSub> subs = new ArrayList<>();
+ assert mergeState.docMaps.length == toMerge.size();
+ for(int i=0;i<toMerge.size();i++) {
+ subs.add(new SortedNumericDocValuesSub(mergeState.docMaps[i], mergeState.liveDocs[i], toMerge.get(i), mergeState.maxDocs[i]));
+ }
+
+ final DocIDMerger<SortedNumericDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
+
return new Iterator<Number>() {
- int readerUpto = -1;
- int docIDUpto;
long nextValue;
- int currentMaxDoc;
- Bits currentLiveDocs;
boolean nextIsSet;
int valueUpto;
int valueLength;
+ SortedNumericDocValuesSub current;
@Override
public boolean hasNext() {
@@ -504,7 +542,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public Number next() {
- if (!hasNext()) {
+ if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@@ -514,38 +552,21 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
- if (readerUpto == numReaders) {
- return false;
- }
if (valueUpto < valueLength) {
- nextValue = dvs[readerUpto].valueAt(valueUpto);
+ nextValue = current.values.valueAt(valueUpto);
valueUpto++;
nextIsSet = true;
return true;
}
- if (docIDUpto == currentMaxDoc) {
- readerUpto++;
- if (readerUpto < numReaders) {
- currentLiveDocs = mergeState.liveDocs[readerUpto];
- currentMaxDoc = mergeState.maxDocs[readerUpto];
- }
- docIDUpto = 0;
- continue;
- }
-
- if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
- assert docIDUpto < currentMaxDoc;
- SortedNumericDocValues dv = dvs[readerUpto];
- dv.setDocument(docIDUpto);
- valueUpto = 0;
- valueLength = dv.count();
- docIDUpto++;
- continue;
+ current = docIDMerger.next();
+ if (current == null) {
+ return false;
}
-
- docIDUpto++;
+ valueUpto = 0;
+ valueLength = current.values.count();
+ continue;
}
}
};
@@ -554,6 +575,32 @@ public abstract class DocValuesConsumer implements Closeable {
);
}
+ /** Tracks state of one sorted sub-reader that we are merging */
+ private static class SortedDocValuesSub extends DocIDMerger.Sub {
+
+ private final SortedDocValues values;
+ private int docID = -1;
+ private final int maxDoc;
+ private final LongValues map;
+
+ public SortedDocValuesSub(MergeState.DocMap docMap, Bits liveDocs, SortedDocValues values, int maxDoc, LongValues map) {
+ super(docMap, liveDocs);
+ this.values = values;
+ this.maxDoc = maxDoc;
+ this.map = map;
+ }
+
+ @Override
+ public int nextDoc() {
+ docID++;
+ if (docID == maxDoc) {
+ return NO_MORE_DOCS;
+ } else {
+ return docID;
+ }
+ }
+ }
+
/**
* Merges the sorted docvalues from <code>toMerge</code>.
* <p>
@@ -608,7 +655,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public BytesRef next() {
- if (!hasNext()) {
+ if (hasNext() == false) {
throw new NoSuchElementException();
}
int segmentNumber = map.getFirstSegmentNumber(currentOrd);
@@ -629,13 +676,17 @@ public abstract class DocValuesConsumer implements Closeable {
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
+ // We must make a new DocIDMerger for each iterator:
+ List<SortedDocValuesSub> subs = new ArrayList<>();
+ assert mergeState.docMaps.length == toMerge.size();
+ for(int i=0;i<toMerge.size();i++) {
+ subs.add(new SortedDocValuesSub(mergeState.docMaps[i], mergeState.liveDocs[i], toMerge.get(i), mergeState.maxDocs[i], map.getGlobalOrds(i)));
+ }
+
+ final DocIDMerger<SortedDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
+
return new Iterator<Number>() {
- int readerUpto = -1;
- int docIDUpto;
int nextValue;
- int currentMaxDoc;
- Bits currentLiveDocs;
- LongValues currentMap;
boolean nextIsSet;
@Override
@@ -650,7 +701,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public Number next() {
- if (!hasNext()) {
+ if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@@ -661,30 +712,15 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
- if (readerUpto == numReaders) {
+ SortedDocValuesSub sub = docIDMerger.next();
+ if (sub == null) {
return false;
}
- if (docIDUpto == currentMaxDoc) {
- readerUpto++;
- if (readerUpto < numReaders) {
- currentMap = map.getGlobalOrds(readerUpto);
- currentLiveDocs = mergeState.liveDocs[readerUpto];
- currentMaxDoc = mergeState.maxDocs[readerUpto];
- }
- docIDUpto = 0;
- continue;
- }
-
- if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
- nextIsSet = true;
- int segOrd = dvs[readerUpto].getOrd(docIDUpto);
- nextValue = segOrd == -1 ? -1 : (int) currentMap.get(segOrd);
- docIDUpto++;
- return true;
- }
-
- docIDUpto++;
+ nextIsSet = true;
+ int segOrd = sub.values.getOrd(sub.docID);
+ nextValue = segOrd == -1 ? -1 : (int) sub.map.get(segOrd);
+ return true;
}
}
};
@@ -693,6 +729,37 @@ public abstract class DocValuesConsumer implements Closeable {
);
}
+ /** Tracks state of one sorted set sub-reader that we are merging */
+ private static class SortedSetDocValuesSub extends DocIDMerger.Sub {
+
+ private final SortedSetDocValues values;
+ int docID = -1;
+ private final int maxDoc;
+ private final LongValues map;
+
+ public SortedSetDocValuesSub(MergeState.DocMap docMap, Bits liveDocs, SortedSetDocValues values, int maxDoc, LongValues map) {
+ super(docMap, liveDocs);
+ this.values = values;
+ this.maxDoc = maxDoc;
+ this.map = map;
+ }
+
+ @Override
+ public int nextDoc() {
+ docID++;
+ if (docID == maxDoc) {
+ return NO_MORE_DOCS;
+ } else {
+ return docID;
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "SortedSetDocValuesSub(docID=" + docID + " mappedDocID=" + mappedDocID + " values=" + values + ")";
+ }
+ }
+
/**
* Merges the sortedset docvalues from <code>toMerge</code>.
* <p>
@@ -700,14 +767,12 @@ public abstract class DocValuesConsumer implements Closeable {
* an Iterable that merges ordinals and values and filters deleted documents .
*/
public void mergeSortedSetField(FieldInfo fieldInfo, final MergeState mergeState, List<SortedSetDocValues> toMerge) throws IOException {
- final SortedSetDocValues dvs[] = toMerge.toArray(new SortedSetDocValues[toMerge.size()]);
- final int numReaders = mergeState.maxDocs.length;
// step 1: iterate thru each sub and mark terms still in use
- TermsEnum liveTerms[] = new TermsEnum[dvs.length];
+ TermsEnum liveTerms[] = new TermsEnum[toMerge.size()];
long[] weights = new long[liveTerms.length];
for (int sub = 0; sub < liveTerms.length; sub++) {
- SortedSetDocValues dv = dvs[sub];
+ SortedSetDocValues dv = toMerge.get(sub);
Bits liveDocs = mergeState.liveDocs[sub];
int maxDoc = mergeState.maxDocs[sub];
if (liveDocs == null) {
@@ -748,12 +813,12 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public BytesRef next() {
- if (!hasNext()) {
+ if (hasNext() == false) {
throw new NoSuchElementException();
}
int segmentNumber = map.getFirstSegmentNumber(currentOrd);
long segmentOrd = map.getFirstSegmentOrd(currentOrd);
- final BytesRef term = dvs[segmentNumber].lookupOrd(segmentOrd);
+ final BytesRef term = toMerge.get(segmentNumber).lookupOrd(segmentOrd);
currentOrd++;
return term;
}
@@ -769,12 +834,18 @@ public abstract class DocValuesConsumer implements Closeable {
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
+
+ // We must make a new DocIDMerger for each iterator:
+ List<SortedSetDocValuesSub> subs = new ArrayList<>();
+ assert mergeState.docMaps.length == toMerge.size();
+ for(int i=0;i<toMerge.size();i++) {
+ subs.add(new SortedSetDocValuesSub(mergeState.docMaps[i], mergeState.liveDocs[i], toMerge.get(i), mergeState.maxDocs[i], map.getGlobalOrds(i)));
+ }
+
+ final DocIDMerger<SortedSetDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
+
return new Iterator<Number>() {
- int readerUpto = -1;
- int docIDUpto;
int nextValue;
- int currentMaxDoc;
- Bits currentLiveDocs;
boolean nextIsSet;
@Override
@@ -789,7 +860,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public Number next() {
- if (!hasNext()) {
+ if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@@ -800,33 +871,18 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
- if (readerUpto == numReaders) {
+ SortedSetDocValuesSub sub = docIDMerger.next();
+ if (sub == null) {
return false;
}
-
- if (docIDUpto == currentMaxDoc) {
- readerUpto++;
- if (readerUpto < numReaders) {
- currentLiveDocs = mergeState.liveDocs[readerUpto];
- currentMaxDoc = mergeState.maxDocs[readerUpto];
- }
- docIDUpto = 0;
- continue;
- }
-
- if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
- nextIsSet = true;
- SortedSetDocValues dv = dvs[readerUpto];
- dv.setDocument(docIDUpto);
- nextValue = 0;
- while (dv.nextOrd() != SortedSetDocValues.NO_MORE_ORDS) {
- nextValue++;
- }
- docIDUpto++;
- return true;
+ sub.values.setDocument(sub.docID);
+ nextValue = 0;
+ while (sub.values.nextOrd() != SortedSetDocValues.NO_MORE_ORDS) {
+ nextValue++;
}
-
- docIDUpto++;
+ //System.out.println(" doc " + sub + " -> ord count = " + nextValue);
+ nextIsSet = true;
+ return true;
}
}
};
@@ -836,13 +892,18 @@ public abstract class DocValuesConsumer implements Closeable {
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
+
+ // We must make a new DocIDMerger for each iterator:
+ List<SortedSetDocValuesSub> subs = new ArrayList<>();
+ assert mergeState.docMaps.length == toMerge.size();
+ for(int i=0;i<toMerge.size();i++) {
+ subs.add(new SortedSetDocValuesSub(mergeState.docMaps[i], mergeState.liveDocs[i], toMerge.get(i), mergeState.maxDocs[i], map.getGlobalOrds(i)));
+ }
+
+ final DocIDMerger<SortedSetDocValuesSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
+
return new Iterator<Number>() {
- int readerUpto = -1;
- int docIDUpto;
long nextValue;
- int currentMaxDoc;
- Bits currentLiveDocs;
- LongValues currentMap;
boolean nextIsSet;
long ords[] = new long[8];
int ordUpto;
@@ -860,7 +921,7 @@ public abstract class DocValuesConsumer implements Closeable {
@Override
public Number next() {
- if (!hasNext()) {
+ if (hasNext() == false) {
throw new NoSuchElementException();
}
assert nextIsSet;
@@ -871,10 +932,6 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
- if (readerUpto == numReaders) {
- return false;
- }
-
if (ordUpto < ordLength) {
nextValue = ords[ordUpto];
ordUpto++;
@@ -882,35 +939,22 @@ public abstract class DocValuesConsumer implements Closeable {
return true;
}
- if (docIDUpto == currentMaxDoc) {
- readerUpto++;
- if (readerUpto < numReaders) {
- currentMap = map.getGlobalOrds(readerUpto);
- currentLiveDocs = mergeState.liveDocs[readerUpto];
- currentMaxDoc = mergeState.maxDocs[readerUpto];
- }
- docIDUpto = 0;
- continue;
+ SortedSetDocValuesSub sub = docIDMerger.next();
+ if (sub == null) {
+ return false;
}
-
- if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
- assert docIDUpto < currentMaxDoc;
- SortedSetDocValues dv = dvs[readerUpto];
- dv.setDocument(docIDUpto);
- ordUpto = ordLength = 0;
- long ord;
- while ((ord = dv.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
- if (ordLength == ords.length) {
- ords = ArrayUtil.grow(ords, ordLength+1);
- }
- ords[ordLength] = currentMap.get(ord);
- ordLength++;
+ sub.values.setDocument(sub.docID);
+
+ ordUpto = ordLength = 0;
+ long ord;
+ while ((ord = sub.values.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+ if (ordLength == ords.length) {
+ ords = ArrayUtil.grow(ords, ordLength+1);
}
- docIDUpto++;
- continue;
+ ords[ordLength] = sub.map.get(ord);
+ ordLength++;
}
-
- docIDUpto++;
+ continue;
}
}
};
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
index 43b4416..05084db 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
@@ -76,7 +76,6 @@ public abstract class PointsWriter implements Closeable {
}
MergeState.DocMap docMap = mergeState.docMaps[i];
- int docBase = mergeState.docBase[i];
pointsReader.intersect(fieldInfo.name,
new IntersectVisitor() {
@Override
@@ -90,7 +89,7 @@ public abstract class PointsWriter implements Closeable {
int newDocID = docMap.get(docID);
if (newDocID != -1) {
// Not deleted:
- mergedVisitor.visit(docBase + newDocID, packedValue);
+ mergedVisitor.visit(newDocID, packedValue);
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/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 b8cff11..b76247d 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java
@@ -20,10 +20,13 @@ import java.io.Closeable;
import java.io.IOException;
import java.io.Reader;
import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.StoredField;
+import org.apache.lucene.index.DocIDMerger;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexableField;
@@ -33,6 +36,8 @@ import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
/**
* Codec API for writing stored fields:
* <ol>
@@ -73,6 +78,30 @@ public abstract class StoredFieldsWriter implements Closeable {
* check that this is the case to detect the JRE bug described
* in LUCENE-1282. */
public abstract void finish(FieldInfos fis, int numDocs) throws IOException;
+
+ private static class StoredFieldsMergeSub extends DocIDMerger.Sub {
+ private final StoredFieldsReader reader;
+ private final int maxDoc;
+ private final MergeVisitor visitor;
+ int docID = -1;
+
+ public StoredFieldsMergeSub(MergeVisitor visitor, MergeState.DocMap docMap, Bits liveDocs, StoredFieldsReader reader, int maxDoc) {
+ super(docMap, liveDocs);
+ this.maxDoc = maxDoc;
+ this.reader = reader;
+ this.visitor = visitor;
+ }
+
+ @Override
+ public int nextDoc() {
+ docID++;
+ if (docID == maxDoc) {
+ return NO_MORE_DOCS;
+ } else {
+ return docID;
+ }
+ }
+ }
/** Merges in the stored fields from the readers in
* <code>mergeState</code>. The default implementation skips
@@ -82,23 +111,26 @@ public abstract class StoredFieldsWriter implements Closeable {
* Implementations can override this method for more sophisticated
* merging (bulk-byte copying, etc). */
public int merge(MergeState mergeState) throws IOException {
- int docCount = 0;
- for (int i=0;i<mergeState.storedFieldsReaders.length;i++) {
+ List<StoredFieldsMergeSub> subs = new ArrayList<>();
+ for(int i=0;i<mergeState.storedFieldsReaders.length;i++) {
StoredFieldsReader storedFieldsReader = mergeState.storedFieldsReaders[i];
storedFieldsReader.checkIntegrity();
- MergeVisitor visitor = new MergeVisitor(mergeState, i);
- int maxDoc = mergeState.maxDocs[i];
- Bits liveDocs = mergeState.liveDocs[i];
- for (int docID=0;docID<maxDoc;docID++) {
- if (liveDocs != null && !liveDocs.get(docID)) {
- // skip deleted docs
- continue;
- }
- startDocument();
- storedFieldsReader.visitDocument(docID, visitor);
- finishDocument();
- docCount++;
+ subs.add(new StoredFieldsMergeSub(new MergeVisitor(mergeState, i), mergeState.docMaps[i], mergeState.liveDocs[i], storedFieldsReader, mergeState.maxDocs[i]));
+ }
+
+ final DocIDMerger<StoredFieldsMergeSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
+
+ int docCount = 0;
+ while (true) {
+ StoredFieldsMergeSub sub = docIDMerger.next();
+ if (sub == null) {
+ break;
}
+ assert sub.mappedDocID == docCount;
+ startDocument();
+ sub.reader.visitDocument(sub.docID, sub.visitor);
+ finishDocument();
+ docCount++;
}
finish(mergeState.mergeFieldInfos, docCount);
return docCount;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/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 1aff737..6ab1157 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
@@ -16,16 +16,18 @@
*/
package org.apache.lucene.codecs;
-
import java.io.Closeable;
import java.io.IOException;
+import java.util.ArrayList;
import java.util.Iterator;
+import java.util.List;
-import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.DocIDMerger;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.MergeState;
+import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSetIterator;
@@ -34,6 +36,8 @@ import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
/**
* Codec API for writing term vectors:
* <ol>
@@ -160,6 +164,28 @@ public abstract class TermVectorsWriter implements Closeable {
}
}
+ private static class TermVectorsMergeSub extends DocIDMerger.Sub {
+ private final TermVectorsReader reader;
+ private final int maxDoc;
+ int docID = -1;
+
+ public TermVectorsMergeSub(MergeState.DocMap docMap, Bits liveDocs, TermVectorsReader reader, int maxDoc) {
+ super(docMap, liveDocs);
+ this.maxDoc = maxDoc;
+ this.reader = reader;
+ }
+
+ @Override
+ public int nextDoc() {
+ docID++;
+ if (docID == maxDoc) {
+ return NO_MORE_DOCS;
+ } else {
+ return docID;
+ }
+ }
+ }
+
/** Merges in the term vectors from the readers in
* <code>mergeState</code>. The default implementation skips
* over deleted documents, and uses {@link #startDocument(int)},
@@ -170,32 +196,36 @@ public abstract class TermVectorsWriter implements Closeable {
* Implementations can override this method for more sophisticated
* merging (bulk-byte copying, etc). */
public int merge(MergeState mergeState) throws IOException {
+
+ List<TermVectorsMergeSub> subs = new ArrayList<>();
+ for(int i=0;i<mergeState.termVectorsReaders.length;i++) {
+ TermVectorsReader reader = mergeState.termVectorsReaders[i];
+ if (reader != null) {
+ reader.checkIntegrity();
+ }
+ // nocommit make sure the else case tested here
+ subs.add(new TermVectorsMergeSub(mergeState.docMaps[i], mergeState.liveDocs[i], reader, mergeState.maxDocs[i]));
+ }
+
+ final DocIDMerger<TermVectorsMergeSub> docIDMerger = new DocIDMerger<>(subs, mergeState.segmentInfo.getIndexSort() != null);
+
int docCount = 0;
- int numReaders = mergeState.maxDocs.length;
- for (int i = 0; i < numReaders; i++) {
- int maxDoc = mergeState.maxDocs[i];
- Bits liveDocs = mergeState.liveDocs[i];
- TermVectorsReader termVectorsReader = mergeState.termVectorsReaders[i];
- if (termVectorsReader != null) {
- termVectorsReader.checkIntegrity();
+ while (true) {
+ TermVectorsMergeSub sub = docIDMerger.next();
+ if (sub == null) {
+ break;
}
- for (int docID=0;docID<maxDoc;docID++) {
- if (liveDocs != null && !liveDocs.get(docID)) {
- // skip deleted docs
- continue;
- }
- // NOTE: it's very important to first assign to vectors then pass it to
- // termVectorsWriter.addAllDocVectors; see LUCENE-1282
- Fields vectors;
- if (termVectorsReader == null) {
- vectors = null;
- } else {
- vectors = termVectorsReader.get(docID);
- }
- addAllDocVectors(vectors, mergeState);
- docCount++;
+ // NOTE: it's very important to first assign to vectors then pass it to
+ // termVectorsWriter.addAllDocVectors; see LUCENE-1282
+ Fields vectors;
+ if (sub.reader == null) {
+ vectors = null;
+ } else {
+ vectors = sub.reader.get(sub.docID);
}
+ addAllDocVectors(vectors, mergeState);
+ docCount++;
}
finish(mergeState.mergeFieldInfos, docCount);
return docCount;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java
index 68aacc6..9786ec1 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java
@@ -109,7 +109,7 @@ public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
attributes = Collections.unmodifiableMap(input.readStringStringMap());
}
- si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes);
+ si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, null);
si.setFiles(files);
} catch (Throwable exception) {
priorE = exception;
@@ -123,6 +123,10 @@ public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
@Override
public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene50SegmentInfoFormat.SI_EXTENSION);
+ // nocommit indexSort
+ if (si.getIndexSort() != null) {
+ throw new IllegalArgumentException("teach me to write indexSort");
+ }
try (IndexOutput output = dir.createOutput(fileName, ioContext)) {
// Only add the file once we've successfully created it, else IFD assert can trip:
@@ -153,6 +157,7 @@ public class Lucene50SegmentInfoFormat extends SegmentInfoFormat {
}
output.writeSetOfStrings(files);
output.writeMapOfStrings(si.getAttributes());
+
CodecUtil.writeFooter(output);
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
index 9f0d546..c696fbe 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
@@ -16,6 +16,8 @@
*/
package org.apache.lucene.codecs.lucene60;
+// nocommit if index time sorting is in use, don't try to bulk merge ... later we can make crazy bulk merger that looks for long runs from
+// one sub?
import java.util.Objects;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/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 2b1e13d..f6c73bd 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
@@ -171,7 +171,6 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
singleValuePerDoc)) {
List<BKDReader> bkdReaders = new ArrayList<>();
List<MergeState.DocMap> docMaps = new ArrayList<>();
- List<Integer> docIDBases = new ArrayList<>();
for(int i=0;i<mergeState.pointsReaders.length;i++) {
PointsReader reader = mergeState.pointsReaders[i];
@@ -191,7 +190,6 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
if (readerFieldInfo != null) {
BKDReader bkdReader = reader60.readers.get(readerFieldInfo.number);
if (bkdReader != null) {
- docIDBases.add(mergeState.docBase[i]);
bkdReaders.add(bkdReader);
docMaps.add(mergeState.docMaps[i]);
}
@@ -199,7 +197,7 @@ public class Lucene60PointsWriter extends PointsWriter implements Closeable {
}
}
- long fp = writer.merge(dataOut, docMaps, bkdReaders, docIDBases);
+ long fp = writer.merge(dataOut, docMaps, bkdReaders);
if (fp != -1) {
indexFPs.put(fieldInfo.name, fp);
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index 89b36ef..5a68a3d 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -2498,6 +2498,9 @@ public final class CheckIndex implements Closeable {
}
}
+ // nocommit must check index is sorted, if it claims to be
+ // nocommit must check that all segments have the same sort, if any segment is sorted
+
/**
* Parse command line args into fields
* @param args The command line arguments
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java b/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
new file mode 100644
index 0000000..fb1cdf2
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.index;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.lucene.search.DocIdSetIterator; // javadocs
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.PriorityQueue;
+
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
+/** Utility class to help merging documents from sub-readers according to either simple
+ * concatenated (unsorted) order, or by a specified index-time sort, skipping
+ * deleted documents and remapping non-deleted documents. */
+
+public class DocIDMerger<T extends DocIDMerger.Sub> {
+
+ private final List<T> subs;
+
+ // Used when indexSort != null:
+ private final PriorityQueue<T> queue;
+ private boolean first;
+
+ // Used when indexIsSorted
+ private T current;
+ private int nextIndex;
+
+ public static abstract class Sub {
+ public int mappedDocID;
+ final MergeState.DocMap docMap;
+ final Bits liveDocs;
+
+ // nocommit isn't liveDocs redundant? docMap returns -1 for us?
+ public Sub(MergeState.DocMap docMap, Bits liveDocs) {
+ this.docMap = docMap;
+ this.liveDocs = liveDocs;
+ }
+
+ /** Returns the next document ID from this sub reader, and {@link DocIdSetIterator#NO_MORE_DOCS} when done */
+ public abstract int nextDoc();
+ }
+
+ public DocIDMerger(List<T> subs, int maxCount, boolean indexIsSorted) {
+ this.subs = subs;
+
+ if (indexIsSorted) {
+ queue = new PriorityQueue<T>(maxCount) {
+ @Override
+ protected boolean lessThan(Sub a, Sub b) {
+ assert a.mappedDocID != b.mappedDocID;
+ return a.mappedDocID < b.mappedDocID;
+ }
+ };
+ } else {
+ // We simply concatentate
+ queue = null;
+ }
+
+ reset();
+ }
+
+ // nocommit it's awkward that we must pass in this boolean, when the subs should "know" this based on what docMap they have?
+ public DocIDMerger(List<T> subs, boolean indexIsSorted) {
+ this(subs, subs.size(), indexIsSorted);
+ }
+
+ /** Reuse API, currently only used by postings during merge */
+ public void reset() {
+ if (queue != null) {
+ assert queue.size() == 0;
+ for(T sub : subs) {
+ while (true) {
+ int docID = sub.nextDoc();
+ if (docID == NO_MORE_DOCS) {
+ // all docs in this sub were deleted; do not add it to the queue!
+ break;
+ } else if (sub.liveDocs != null && sub.liveDocs.get(docID) == false) {
+ // nocommit is it sub's job to skip deleted docs?
+ continue;
+ } else {
+ sub.mappedDocID = sub.docMap.get(docID);
+ assert sub.mappedDocID != -1;
+ queue.add(sub);
+ break;
+ }
+ }
+ }
+ first = true;
+ } else {
+ if (subs.size() > 0) {
+ current = subs.get(0);
+ nextIndex = 1;
+ } else {
+ current = null;
+ nextIndex = 0;
+ }
+ }
+ }
+
+ /** Returns null when done */
+ public T next() {
+ // Loop until we find a non-deleted document
+ if (queue != null) {
+ T top = queue.top();
+ if (top == null) {
+ // NOTE: it's annoying that caller is allowed to call us again even after we returned null before
+ return null;
+ }
+
+ if (first == false) {
+ while (true) {
+ int docID = top.nextDoc();
+ if (docID == NO_MORE_DOCS) {
+ queue.pop();
+ top = queue.top();
+ break;
+ } else if (top.liveDocs != null && top.liveDocs.get(docID) == false) {
+ continue;
+ } else {
+ top.mappedDocID = top.docMap.get(docID);
+ top = queue.updateTop();
+ break;
+ }
+ }
+ }
+
+ first = false;
+
+ return top;
+
+ } else {
+ while (true) {
+ if (current == null) {
+ // NOTE: it's annoying that caller is allowed to call us again even after we returned null before
+ return null;
+ }
+ int docID = current.nextDoc();
+ if (docID == NO_MORE_DOCS) {
+ if (nextIndex == subs.size()) {
+ current = null;
+ return null;
+ }
+ current = subs.get(nextIndex);
+ nextIndex++;
+ continue;
+ } else if (current.liveDocs != null && current.liveDocs.get(docID) == false) {
+ // Document is deleted
+ continue;
+ }
+
+ current.mappedDocID = current.docMap.get(docID);
+ return current;
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
index 65d6a14..3e8a227 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
@@ -178,7 +178,7 @@ class DocumentsWriterPerThread {
pendingUpdates.clear();
deleteSlice = deleteQueue.newSlice();
- segmentInfo = new SegmentInfo(directoryOrig, Version.LATEST, segmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
+ segmentInfo = new SegmentInfo(directoryOrig, Version.LATEST, segmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
assert numDocsInRAM == 0;
if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segmentName + " delQueue=" + deleteQueue);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/java/org/apache/lucene/index/FilterCodecReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FilterCodecReader.java b/lucene/core/src/java/org/apache/lucene/index/FilterCodecReader.java
index c35dc67..13b6e8d 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FilterCodecReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FilterCodecReader.java
@@ -25,6 +25,7 @@ import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PointsReader;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
+import org.apache.lucene.search.Sort;
import org.apache.lucene.util.Bits;
/**
@@ -102,6 +103,11 @@ public abstract class FilterCodecReader extends CodecReader {
}
@Override
+ public Sort getIndexSort() {
+ return in.getIndexSort();
+ }
+
+ @Override
public void addCoreClosedListener(CoreClosedListener listener) {
in.addCoreClosedListener(listener);
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java b/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java
index f273dba..886c12a 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java
@@ -22,6 +22,7 @@ import java.util.Iterator;
import java.util.Objects;
import org.apache.lucene.search.QueryCache;
+import org.apache.lucene.search.Sort;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@@ -472,6 +473,12 @@ public abstract class FilterLeafReader extends LeafReader {
}
@Override
+ public Sort getIndexSort() {
+ ensureOpen();
+ return in.getIndexSort();
+ }
+
+ @Override
public void checkIntegrity() throws IOException {
ensureOpen();
in.checkIntegrity();
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fdc220ee/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index 2b45b6b..2254ba4 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -16,6 +16,7 @@
*/
package org.apache.lucene.index;
+// nocommit must add sorted indices to back compat tests
import java.io.Closeable;
import java.io.FileNotFoundException;
@@ -32,8 +33,8 @@ import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Locale;
-import java.util.Map;
import java.util.Map.Entry;
+import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
@@ -49,6 +50,7 @@ import org.apache.lucene.index.FieldInfos.FieldNumbers;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Sort;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
@@ -937,6 +939,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// NOTE: this is correct even for an NRT reader because we'll pull FieldInfos even for the un-committed segments:
globalFieldNumberMap = getFieldNumberMap();
+ validateIndexSort();
+
config.getFlushPolicy().init(config);
docWriter = new DocumentsWriter(this, config, directoryOrig, directory);
eventQueue = docWriter.eventQueue();
@@ -1000,6 +1004,22 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
}
+ // nocommit can we improve this from just best effort?
+
+ /** Confirms that the incoming index sort (if any) matches the existing index sort (if any). This is unfortunately just best effort,
+ * because it could be the old index only has flushed segments. */
+ private void validateIndexSort() {
+ Sort indexSort = config.getIndexSort();
+ if (indexSort != null) {
+ for(SegmentCommitInfo info : segmentInfos) {
+ Sort segmentIndexSort = info.info.getIndexSort();
+ if (segmentIndexSort != null && indexSort.equals(segmentIndexSort) == false) {
+ throw new IllegalArgumentException("cannot change previous indexSort=" + segmentIndexSort + " (from segment=" + info + ") to new indexSort=" + indexSort);
+ }
+ }
+ }
+ }
+
// reads latest field infos for the commit
// this is used on IW init and addIndexes(Dir) to create/update the global field map.
// TODO: fix tests abusing this method!
@@ -2474,6 +2494,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* @throws IllegalArgumentException if addIndexes would cause
* the index to exceed {@link #MAX_DOCS}
*/
+ // nocommit doesn't support index sorting? or sorts must be the same?
public void addIndexes(Directory... dirs) throws IOException {
ensureOpen();
@@ -2603,6 +2624,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* @throws IllegalArgumentException
* if addIndexes would cause the index to exceed {@link #MAX_DOCS}
*/
+ // nocommit make sure if you add "sorted by X" to "sorted by Y" index, we catch it
public void addIndexes(CodecReader... readers) throws IOException {
ensureOpen();
@@ -2630,7 +2652,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(directory);
SegmentInfo info = new SegmentInfo(directoryOrig, Version.LATEST, mergedName, -1,
- false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
+ false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), config.getIndexSort());
SegmentMerger merger = new SegmentMerger(Arrays.asList(readers), info, infoStream, trackingDir,
globalFieldNumberMap,
@@ -2715,7 +2737,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// Same SI as before but we change directory and name
SegmentInfo newInfo = new SegmentInfo(directoryOrig, info.info.getVersion(), segName, info.info.maxDoc(),
info.info.getUseCompoundFile(), info.info.getCodec(),
- info.info.getDiagnostics(), info.info.getId(), info.info.getAttributes());
+ info.info.getDiagnostics(), info.info.getId(), info.info.getAttributes(), info.info.getIndexSort());
SegmentCommitInfo newInfoPerCommit = new SegmentCommitInfo(newInfo, info.getDelCount(), info.getDelGen(),
info.getFieldInfosGen(), info.getDocValuesGen());
@@ -3243,16 +3265,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
private static class MergedDeletesAndUpdates {
ReadersAndUpdates mergedDeletesAndUpdates = null;
- MergePolicy.DocMap docMap = null;
boolean initializedWritableLiveDocs = false;
MergedDeletesAndUpdates() {}
- final void init(ReaderPool readerPool, MergePolicy.OneMerge merge, MergeState mergeState, boolean initWritableLiveDocs) throws IOException {
+ final void init(ReaderPool readerPool, MergePolicy.OneMerge merge, boolean initWritableLiveDocs) throws IOException {
if (mergedDeletesAndUpdates == null) {
mergedDeletesAndUpdates = readerPool.get(merge.info, true);
- docMap = merge.getDocMap(mergeState);
- assert docMap.isConsistent(merge.info.info.maxDoc());
}
if (initWritableLiveDocs && !initializedWritableLiveDocs) {
mergedDeletesAndUpdates.initWritableLiveDocs();
@@ -3262,18 +3281,18 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
- private void maybeApplyMergedDVUpdates(MergePolicy.OneMerge merge, MergeState mergeState, int docUpto,
+ private void maybeApplyMergedDVUpdates(MergePolicy.OneMerge merge, MergeState mergeState,
MergedDeletesAndUpdates holder, String[] mergingFields, DocValuesFieldUpdates[] dvFieldUpdates,
- DocValuesFieldUpdates.Iterator[] updatesIters, int curDoc) throws IOException {
+ DocValuesFieldUpdates.Iterator[] updatesIters, int segment, int curDoc) throws IOException {
int newDoc = -1;
for (int idx = 0; idx < mergingFields.length; idx++) {
DocValuesFieldUpdates.Iterator updatesIter = updatesIters[idx];
if (updatesIter.doc() == curDoc) { // document has an update
if (holder.mergedDeletesAndUpdates == null) {
- holder.init(readerPool, merge, mergeState, false);
+ holder.init(readerPool, merge, false);
}
if (newDoc == -1) { // map once per all field updates, but only if there are any updates
- newDoc = holder.docMap.map(docUpto);
+ newDoc = mergeState.docMaps[segment].get(curDoc);
}
DocValuesFieldUpdates dvUpdates = dvFieldUpdates[idx];
dvUpdates.add(newDoc, updatesIter.value());
@@ -3306,13 +3325,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// Carefully merge deletes that occurred after we
// started merging:
- int docUpto = 0;
long minGen = Long.MAX_VALUE;
// Lazy init (only when we find a delete to carry over):
final MergedDeletesAndUpdates holder = new MergedDeletesAndUpdates();
final DocValuesFieldUpdates.Container mergedDVUpdates = new DocValuesFieldUpdates.Container();
-
+
+ assert sourceSegments.size() == mergeState.docMaps.length;
for (int i = 0; i < sourceSegments.size(); i++) {
SegmentCommitInfo info = sourceSegments.get(i);
minGen = Math.min(info.getBufferedDeletesGen(), minGen);
@@ -3375,21 +3394,20 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// since we started the merge, so we
// must merge them:
for (int j = 0; j < maxDoc; j++) {
- if (!prevLiveDocs.get(j)) {
- assert !currentLiveDocs.get(j);
- } else {
- if (!currentLiveDocs.get(j)) {
- if (holder.mergedDeletesAndUpdates == null || !holder.initializedWritableLiveDocs) {
- holder.init(readerPool, merge, mergeState, true);
- }
- holder.mergedDeletesAndUpdates.delete(holder.docMap.map(docUpto));
- if (mergingFields != null) { // advance all iters beyond the deleted document
- skipDeletedDoc(updatesIters, j);
- }
- } else if (mergingFields != null) {
- maybeApplyMergedDVUpdates(merge, mergeState, docUpto, holder, mergingFields, dvFieldUpdates, updatesIters, j);
+ if (prevLiveDocs.get(j) == false) {
+ // if the document was deleted before, it better still be deleted!
+ assert currentLiveDocs.get(j) == false;
+ } else if (currentLiveDocs.get(j) == false) {
+ // the document was deleted while we were merging:
+ if (holder.mergedDeletesAndUpdates == null || holder.initializedWritableLiveDocs == false) {
+ holder.init(readerPool, merge, true);
}
- docUpto++;
+ holder.mergedDeletesAndUpdates.delete(mergeState.docMaps[i].get(mergeState.leafDocMaps[i].get(j)));
+ if (mergingFields != null) { // advance all iters beyond the deleted document
+ skipDeletedDoc(updatesIters, j);
+ }
+ } else if (mergingFields != null) {
+ maybeApplyMergedDVUpdates(merge, mergeState, holder, mergingFields, dvFieldUpdates, updatesIters, i, j);
}
}
} else if (mergingFields != null) {
@@ -3397,50 +3415,38 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
for (int j = 0; j < maxDoc; j++) {
if (prevLiveDocs.get(j)) {
// document isn't deleted, check if any of the fields have an update to it
- maybeApplyMergedDVUpdates(merge, mergeState, docUpto, holder, mergingFields, dvFieldUpdates, updatesIters, j);
- // advance docUpto for every non-deleted document
- docUpto++;
+ maybeApplyMergedDVUpdates(merge, mergeState, holder, mergingFields, dvFieldUpdates, updatesIters, i, j);
} else {
// advance all iters beyond the deleted document
skipDeletedDoc(updatesIters, j);
}
}
- } else {
- docUpto += info.info.maxDoc() - info.getDelCount() - rld.getPendingDeleteCount();
}
} else if (currentLiveDocs != null) {
assert currentLiveDocs.length() == maxDoc;
// This segment had no deletes before but now it
// does:
for (int j = 0; j < maxDoc; j++) {
- if (!currentLiveDocs.get(j)) {
+ if (currentLiveDocs.get(j) == false) {
if (holder.mergedDeletesAndUpdates == null || !holder.initializedWritableLiveDocs) {
- holder.init(readerPool, merge, mergeState, true);
+ holder.init(readerPool, merge, true);
}
- holder.mergedDeletesAndUpdates.delete(holder.docMap.map(docUpto));
+ holder.mergedDeletesAndUpdates.delete(mergeState.docMaps[i].get(mergeState.leafDocMaps[i].get(j)));
if (mergingFields != null) { // advance all iters beyond the deleted document
skipDeletedDoc(updatesIters, j);
}
} else if (mergingFields != null) {
- maybeApplyMergedDVUpdates(merge, mergeState, docUpto, holder, mergingFields, dvFieldUpdates, updatesIters, j);
+ maybeApplyMergedDVUpdates(merge, mergeState, holder, mergingFields, dvFieldUpdates, updatesIters, i, j);
}
- docUpto++;
}
} else if (mergingFields != null) {
// no deletions before or after, but there were updates
for (int j = 0; j < maxDoc; j++) {
- maybeApplyMergedDVUpdates(merge, mergeState, docUpto, holder, mergingFields, dvFieldUpdates, updatesIters, j);
- // advance docUpto for every non-deleted document
- docUpto++;
+ maybeApplyMergedDVUpdates(merge, mergeState, holder, mergingFields, dvFieldUpdates, updatesIters, i, j);
}
- } else {
- // No deletes or updates before or after
- docUpto += info.info.maxDoc();
}
}
- assert docUpto == merge.info.info.maxDoc();
-
if (mergedDVUpdates.any()) {
// System.out.println("[" + Thread.currentThread().getName() + "] IW.commitMergedDeletes: mergedDeletes.info=" + mergedDeletes.info + ", mergedFieldUpdates=" + mergedFieldUpdates);
boolean success = false;
@@ -3881,7 +3887,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// ConcurrentMergePolicy we keep deterministic segment
// names.
final String mergeSegmentName = newSegmentName();
- SegmentInfo si = new SegmentInfo(directoryOrig, Version.LATEST, mergeSegmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
+ SegmentInfo si = new SegmentInfo(directoryOrig, Version.LATEST, mergeSegmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), config.getIndexSort());
Map<String,String> details = new HashMap<>();
details.put("mergeMaxNumSegments", "" + merge.maxNumSegments);
details.put("mergeFactor", Integer.toString(merge.segments.size()));
@@ -4082,10 +4088,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
// System.out.println("[" + Thread.currentThread().getName() + "] IW.mergeMiddle: merging " + merge.getMergeReaders());
-
- // we pass merge.getMergeReaders() instead of merge.readers to allow the
- // OneMerge to return a view over the actual segments to merge
- final SegmentMerger merger = new SegmentMerger(merge.getMergeReaders(),
+
+ // Let the merge wrap readers
+ List<CodecReader> mergeReaders = new ArrayList<>();
+ for (SegmentReader reader : merge.readers) {
+ mergeReaders.add(merge.wrapForMerge(reader));
+ }
+ final SegmentMerger merger = new SegmentMerger(mergeReaders,
merge.info.info, infoStream, dirWrapper,
globalFieldNumberMap,
context);