LUCENE-5969: cutover all merge implementations away from LeafReader[] to individual low-level producers

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5969@1628026 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2014-09-27 23:54:29 +00:00
parent d3eb93047b
commit a06c00350b
20 changed files with 633 additions and 328 deletions

View File

@ -131,67 +131,115 @@ public abstract class DocValuesConsumer implements Closeable {
* Implementations can override this method
* for more sophisticated merging (bulk-byte copying, etc). */
public void merge(MergeState mergeState) throws IOException {
for (FieldInfo field : mergeState.fieldInfos) {
DocValuesType type = field.getDocValuesType();
for(DocValuesProducer docValuesProducer : mergeState.docValuesProducers) {
if (docValuesProducer != null) {
docValuesProducer.checkIntegrity();
}
}
for (FieldInfo mergeFieldInfo : mergeState.mergeFieldInfos) {
DocValuesType type = mergeFieldInfo.getDocValuesType();
if (type != null) {
if (type == DocValuesType.NUMERIC) {
List<NumericDocValues> toMerge = new ArrayList<>();
List<Bits> docsWithField = new ArrayList<>();
for (LeafReader reader : mergeState.readers) {
NumericDocValues values = reader.getNumericDocValues(field.name);
Bits bits = reader.getDocsWithField(field.name);
for (int i=0;i<mergeState.docValuesProducers.length;i++) {
NumericDocValues values = null;
Bits bits = null;
DocValuesProducer docValuesProducer = mergeState.docValuesProducers[i];
if (docValuesProducer != null) {
FieldInfo fieldInfo = mergeState.fieldInfos[i].fieldInfo(mergeFieldInfo.name);
if (fieldInfo != null && fieldInfo.getDocValuesType() == DocValuesType.NUMERIC) {
// TODO: use dedicated merge API, so impl can do merge-specific checksumming, and won't cache values in RAM
values = docValuesProducer.getNumeric(fieldInfo);
bits = docValuesProducer.getDocsWithField(fieldInfo);
}
}
if (values == null) {
values = DocValues.emptyNumeric();
bits = new Bits.MatchNoBits(reader.maxDoc());
bits = new Bits.MatchNoBits(mergeState.maxDocs[i]);
}
toMerge.add(values);
docsWithField.add(bits);
}
mergeNumericField(field, mergeState, toMerge, docsWithField);
mergeNumericField(mergeFieldInfo, mergeState, toMerge, docsWithField);
} else if (type == DocValuesType.BINARY) {
List<BinaryDocValues> toMerge = new ArrayList<>();
List<Bits> docsWithField = new ArrayList<>();
for (LeafReader reader : mergeState.readers) {
BinaryDocValues values = reader.getBinaryDocValues(field.name);
Bits bits = reader.getDocsWithField(field.name);
for (int i=0;i<mergeState.docValuesProducers.length;i++) {
BinaryDocValues values = null;
Bits bits = null;
DocValuesProducer docValuesProducer = mergeState.docValuesProducers[i];
if (docValuesProducer != null) {
FieldInfo fieldInfo = mergeState.fieldInfos[i].fieldInfo(mergeFieldInfo.name);
if (fieldInfo != null && fieldInfo.getDocValuesType() == DocValuesType.BINARY) {
// TODO: use dedicated merge API, so impl can do merge-specific checksumming, and won't cache values in RAM
values = docValuesProducer.getBinary(fieldInfo);
bits = docValuesProducer.getDocsWithField(fieldInfo);
}
}
if (values == null) {
values = DocValues.emptyBinary();
bits = new Bits.MatchNoBits(reader.maxDoc());
bits = new Bits.MatchNoBits(mergeState.maxDocs[i]);
}
toMerge.add(values);
docsWithField.add(bits);
}
mergeBinaryField(field, mergeState, toMerge, docsWithField);
mergeBinaryField(mergeFieldInfo, mergeState, toMerge, docsWithField);
} else if (type == DocValuesType.SORTED) {
List<SortedDocValues> toMerge = new ArrayList<>();
for (LeafReader reader : mergeState.readers) {
SortedDocValues values = reader.getSortedDocValues(field.name);
for (int i=0;i<mergeState.docValuesProducers.length;i++) {
SortedDocValues values = null;
DocValuesProducer docValuesProducer = mergeState.docValuesProducers[i];
if (docValuesProducer != null) {
FieldInfo fieldInfo = mergeState.fieldInfos[i].fieldInfo(mergeFieldInfo.name);
if (fieldInfo != null && fieldInfo.getDocValuesType() == DocValuesType.SORTED) {
// TODO: use dedicated merge API, so impl can do merge-specific checksumming, and won't cache values in RAM
values = docValuesProducer.getSorted(fieldInfo);
}
}
if (values == null) {
values = DocValues.emptySorted();
}
toMerge.add(values);
}
mergeSortedField(field, mergeState, toMerge);
mergeSortedField(mergeFieldInfo, mergeState, toMerge);
} else if (type == DocValuesType.SORTED_SET) {
List<SortedSetDocValues> toMerge = new ArrayList<>();
for (LeafReader reader : mergeState.readers) {
SortedSetDocValues values = reader.getSortedSetDocValues(field.name);
for (int i=0;i<mergeState.docValuesProducers.length;i++) {
SortedSetDocValues values = null;
DocValuesProducer docValuesProducer = mergeState.docValuesProducers[i];
if (docValuesProducer != null) {
FieldInfo fieldInfo = mergeState.fieldInfos[i].fieldInfo(mergeFieldInfo.name);
if (fieldInfo != null && fieldInfo.getDocValuesType() == DocValuesType.SORTED_SET) {
// TODO: use dedicated merge API, so impl can do merge-specific checksumming, and won't cache values in RAM
values = docValuesProducer.getSortedSet(fieldInfo);
}
}
if (values == null) {
values = DocValues.emptySortedSet();
}
toMerge.add(values);
}
mergeSortedSetField(field, mergeState, toMerge);
mergeSortedSetField(mergeFieldInfo, mergeState, toMerge);
} else if (type == DocValuesType.SORTED_NUMERIC) {
List<SortedNumericDocValues> toMerge = new ArrayList<>();
for (LeafReader reader : mergeState.readers) {
SortedNumericDocValues values = reader.getSortedNumericDocValues(field.name);
for (int i=0;i<mergeState.docValuesProducers.length;i++) {
SortedNumericDocValues values = null;
DocValuesProducer docValuesProducer = mergeState.docValuesProducers[i];
if (docValuesProducer != null) {
FieldInfo fieldInfo = mergeState.fieldInfos[i].fieldInfo(mergeFieldInfo.name);
if (fieldInfo != null && fieldInfo.getDocValuesType() == DocValuesType.SORTED_NUMERIC) {
// TODO: use dedicated merge API, so impl can do merge-specific checksumming, and won't cache values in RAM
values = docValuesProducer.getSortedNumeric(fieldInfo);
}
}
if (values == null) {
values = DocValues.emptySortedNumeric(reader.maxDoc());
values = DocValues.emptySortedNumeric(mergeState.maxDocs[i]);
}
toMerge.add(values);
}
mergeSortedNumericField(field, mergeState, toMerge);
mergeSortedNumericField(mergeFieldInfo, mergeState, toMerge);
} else {
throw new AssertionError("type=" + type);
}
@ -216,7 +264,7 @@ public abstract class DocValuesConsumer implements Closeable {
int docIDUpto;
long nextValue;
boolean nextHasValue;
LeafReader currentReader;
int currentMaxDoc;
NumericDocValues currentValues;
Bits currentLiveDocs;
Bits currentDocsWithField;
@ -248,13 +296,13 @@ public abstract class DocValuesConsumer implements Closeable {
return false;
}
if (currentReader == null || docIDUpto == currentReader.maxDoc()) {
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < toMerge.size()) {
currentReader = mergeState.readers.get(readerUpto);
currentValues = toMerge.get(readerUpto);
currentLiveDocs = currentReader.getLiveDocs();
currentDocsWithField = docsWithField.get(readerUpto);
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
@ -297,7 +345,7 @@ public abstract class DocValuesConsumer implements Closeable {
int docIDUpto;
BytesRef nextValue;
BytesRef nextPointer; // points to null if missing, or nextValue
LeafReader currentReader;
int currentMaxDoc;
BinaryDocValues currentValues;
Bits currentLiveDocs;
Bits currentDocsWithField;
@ -329,13 +377,13 @@ public abstract class DocValuesConsumer implements Closeable {
return false;
}
if (currentReader == null || docIDUpto == currentReader.maxDoc()) {
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < toMerge.size()) {
currentReader = mergeState.readers.get(readerUpto);
currentValues = toMerge.get(readerUpto);
currentDocsWithField = docsWithField.get(readerUpto);
currentLiveDocs = currentReader.getLiveDocs();
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
@ -368,8 +416,8 @@ 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 LeafReader readers[] = mergeState.readers.toArray(new LeafReader[toMerge.size()]);
final SortedNumericDocValues dvs[] = toMerge.toArray(new SortedNumericDocValues[toMerge.size()]);
final int numReaders = toMerge.size();
final SortedNumericDocValues dvs[] = toMerge.toArray(new SortedNumericDocValues[numReaders]);
// step 3: add field
addSortedNumericField(fieldInfo,
@ -381,7 +429,7 @@ public abstract class DocValuesConsumer implements Closeable {
int readerUpto = -1;
int docIDUpto;
int nextValue;
LeafReader currentReader;
int currentMaxDoc;
Bits currentLiveDocs;
boolean nextIsSet;
@ -407,15 +455,15 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == readers.length) {
if (readerUpto == numReaders) {
return false;
}
if (currentReader == null || docIDUpto == currentReader.maxDoc()) {
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < readers.length) {
currentReader = readers[readerUpto];
currentLiveDocs = currentReader.getLiveDocs();
if (readerUpto < numReaders) {
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
@ -444,7 +492,7 @@ public abstract class DocValuesConsumer implements Closeable {
int readerUpto = -1;
int docIDUpto;
long nextValue;
LeafReader currentReader;
int currentMaxDoc;
Bits currentLiveDocs;
boolean nextIsSet;
int valueUpto;
@ -472,7 +520,7 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == readers.length) {
if (readerUpto == numReaders) {
return false;
}
@ -483,18 +531,18 @@ public abstract class DocValuesConsumer implements Closeable {
return true;
}
if (currentReader == null || docIDUpto == currentReader.maxDoc()) {
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < readers.length) {
currentReader = readers[readerUpto];
currentLiveDocs = currentReader.getLiveDocs();
if (readerUpto < numReaders) {
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
assert docIDUpto < currentReader.maxDoc();
assert docIDUpto < currentMaxDoc;
SortedNumericDocValues dv = dvs[readerUpto];
dv.setDocument(docIDUpto);
valueUpto = 0;
@ -519,22 +567,22 @@ public abstract class DocValuesConsumer implements Closeable {
* an Iterable that merges ordinals and values and filters deleted documents .
*/
public void mergeSortedField(FieldInfo fieldInfo, final MergeState mergeState, List<SortedDocValues> toMerge) throws IOException {
final LeafReader readers[] = mergeState.readers.toArray(new LeafReader[toMerge.size()]);
final SortedDocValues dvs[] = toMerge.toArray(new SortedDocValues[toMerge.size()]);
final int numReaders = toMerge.size();
final SortedDocValues dvs[] = toMerge.toArray(new SortedDocValues[numReaders]);
// step 1: iterate thru each sub and mark terms still in use
TermsEnum liveTerms[] = new TermsEnum[dvs.length];
long[] weights = new long[liveTerms.length];
for (int sub = 0; sub < liveTerms.length; sub++) {
LeafReader reader = readers[sub];
for (int sub=0;sub<numReaders;sub++) {
SortedDocValues dv = dvs[sub];
Bits liveDocs = reader.getLiveDocs();
Bits liveDocs = mergeState.liveDocs[sub];
int maxDoc = mergeState.maxDocs[sub];
if (liveDocs == null) {
liveTerms[sub] = dv.termsEnum();
weights[sub] = dv.getValueCount();
} else {
LongBitSet bitset = new LongBitSet(dv.getValueCount());
for (int i = 0; i < reader.maxDoc(); i++) {
for (int i = 0; i < maxDoc; i++) {
if (liveDocs.get(i)) {
int ord = dv.getOrd(i);
if (ord >= 0) {
@ -591,7 +639,7 @@ public abstract class DocValuesConsumer implements Closeable {
int readerUpto = -1;
int docIDUpto;
int nextValue;
LeafReader currentReader;
int currentMaxDoc;
Bits currentLiveDocs;
LongValues currentMap;
boolean nextIsSet;
@ -619,16 +667,16 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == readers.length) {
if (readerUpto == numReaders) {
return false;
}
if (currentReader == null || docIDUpto == currentReader.maxDoc()) {
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < readers.length) {
currentReader = readers[readerUpto];
currentLiveDocs = currentReader.getLiveDocs();
if (readerUpto < numReaders) {
currentMap = map.getGlobalOrds(readerUpto);
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
@ -658,22 +706,22 @@ 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 LeafReader readers[] = mergeState.readers.toArray(new LeafReader[toMerge.size()]);
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];
long[] weights = new long[liveTerms.length];
for (int sub = 0; sub < liveTerms.length; sub++) {
LeafReader reader = readers[sub];
SortedSetDocValues dv = dvs[sub];
Bits liveDocs = reader.getLiveDocs();
Bits liveDocs = mergeState.liveDocs[sub];
int maxDoc = mergeState.maxDocs[sub];
if (liveDocs == null) {
liveTerms[sub] = dv.termsEnum();
weights[sub] = dv.getValueCount();
} else {
LongBitSet bitset = new LongBitSet(dv.getValueCount());
for (int i = 0; i < reader.maxDoc(); i++) {
for (int i = 0; i < maxDoc; i++) {
if (liveDocs.get(i)) {
dv.setDocument(i);
long ord;
@ -731,7 +779,7 @@ public abstract class DocValuesConsumer implements Closeable {
int readerUpto = -1;
int docIDUpto;
int nextValue;
LeafReader currentReader;
int currentMaxDoc;
Bits currentLiveDocs;
boolean nextIsSet;
@ -758,15 +806,15 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == readers.length) {
if (readerUpto == numReaders) {
return false;
}
if (currentReader == null || docIDUpto == currentReader.maxDoc()) {
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < readers.length) {
currentReader = readers[readerUpto];
currentLiveDocs = currentReader.getLiveDocs();
if (readerUpto < numReaders) {
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
@ -798,7 +846,7 @@ public abstract class DocValuesConsumer implements Closeable {
int readerUpto = -1;
int docIDUpto;
long nextValue;
LeafReader currentReader;
int currentMaxDoc;
Bits currentLiveDocs;
LongValues currentMap;
boolean nextIsSet;
@ -829,7 +877,7 @@ public abstract class DocValuesConsumer implements Closeable {
private boolean setNext() {
while (true) {
if (readerUpto == readers.length) {
if (readerUpto == numReaders) {
return false;
}
@ -840,19 +888,19 @@ public abstract class DocValuesConsumer implements Closeable {
return true;
}
if (currentReader == null || docIDUpto == currentReader.maxDoc()) {
if (docIDUpto == currentMaxDoc) {
readerUpto++;
if (readerUpto < readers.length) {
currentReader = readers[readerUpto];
currentLiveDocs = currentReader.getLiveDocs();
if (readerUpto < numReaders) {
currentMap = map.getGlobalOrds(readerUpto);
currentLiveDocs = mergeState.liveDocs[readerUpto];
currentMaxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
assert docIDUpto < currentReader.maxDoc();
assert docIDUpto < currentMaxDoc;
SortedSetDocValues dv = dvs[readerUpto];
dv.setDocument(docIDUpto);
ordUpto = ordLength = 0;

View File

@ -90,11 +90,12 @@ public abstract class FieldsConsumer implements Closeable {
int docBase = 0;
for(int readerIndex=0;readerIndex<mergeState.readers.size();readerIndex++) {
final LeafReader reader = mergeState.readers.get(readerIndex);
final Fields f = reader.fields();
final int maxDoc = reader.maxDoc();
for(int readerIndex=0;readerIndex<mergeState.fieldsProducers.length;readerIndex++) {
final FieldsProducer f = mergeState.fieldsProducers[readerIndex];
final int maxDoc = mergeState.maxDocs[readerIndex];
if (f != null) {
f.checkIntegrity();
slices.add(new ReaderSlice(docBase, maxDoc, readerIndex));
fields.add(f);
}

View File

@ -24,9 +24,10 @@ import java.util.Iterator;
import java.util.List;
import java.util.NoSuchElementException;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentWriteState;
@ -72,17 +73,30 @@ public abstract class NormsConsumer implements Closeable {
* Implementations can override this method
* for more sophisticated merging (bulk-byte copying, etc). */
public void merge(MergeState mergeState) throws IOException {
for (FieldInfo field : mergeState.fieldInfos) {
if (field.hasNorms()) {
for(NormsProducer normsProducer : mergeState.normsProducers) {
if (normsProducer != null) {
normsProducer.checkIntegrity();
}
}
for (FieldInfo mergeFieldInfo : mergeState.mergeFieldInfos) {
if (mergeFieldInfo.hasNorms()) {
List<NumericDocValues> toMerge = new ArrayList<>();
for (LeafReader reader : mergeState.readers) {
NumericDocValues norms = reader.getNormValues(field.name);
for (int i=0;i<mergeState.normsProducers.length;i++) {
NormsProducer normsProducer = mergeState.normsProducers[i];
NumericDocValues norms = null;
if (normsProducer != null) {
FieldInfo fieldInfo = mergeState.fieldInfos[i].fieldInfo(mergeFieldInfo.name);
if (fieldInfo != null && fieldInfo.hasNorms()) {
// TODO: use dedicated merge API, so impl can do merge-specific checksumming, and won't cache values in RAM
norms = normsProducer.getNorms(fieldInfo);
}
}
if (norms == null) {
norms = DocValues.emptyNumeric();
}
toMerge.add(norms);
}
mergeNormsField(field, mergeState, toMerge);
mergeNormsField(mergeFieldInfo, mergeState, toMerge);
}
}
}
@ -104,7 +118,7 @@ public abstract class NormsConsumer implements Closeable {
int readerUpto = -1;
int docIDUpto;
long nextValue;
LeafReader currentReader;
int maxDoc;
NumericDocValues currentValues;
Bits currentLiveDocs;
boolean nextIsSet;
@ -135,12 +149,12 @@ public abstract class NormsConsumer implements Closeable {
return false;
}
if (currentReader == null || docIDUpto == currentReader.maxDoc()) {
if (currentValues == null || docIDUpto == maxDoc) {
readerUpto++;
if (readerUpto < toMerge.size()) {
currentReader = mergeState.readers.get(readerUpto);
currentValues = toMerge.get(readerUpto);
currentLiveDocs = currentReader.getLiveDocs();
currentLiveDocs = mergeState.liveDocs[readerUpto];
maxDoc = mergeState.maxDocs[readerUpto];
}
docIDUpto = 0;
continue;

View File

@ -36,8 +36,8 @@ public abstract class StoredFieldsReader implements Cloneable, Closeable, Accoun
protected StoredFieldsReader() {
}
/** Visit the stored fields for document <code>n</code> */
public abstract void visitDocument(int n, StoredFieldVisitor visitor) throws IOException;
/** Visit the stored fields for document <code>docID</code> */
public abstract void visitDocument(int docID, StoredFieldVisitor visitor) throws IOException;
@Override
public abstract StoredFieldsReader clone();

View File

@ -19,13 +19,15 @@ package org.apache.lucene.codecs;
import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.document.DocumentStoredFieldVisitor;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.index.StoredDocument;
import org.apache.lucene.util.Bits;
import org.apache.lucene.index.LeafReader;
/**
* Codec API for writing stored fields:
@ -82,11 +84,13 @@ public abstract class StoredFieldsWriter implements Closeable {
* merging (bulk-byte copying, etc). */
public int merge(MergeState mergeState) throws IOException {
int docCount = 0;
for (LeafReader reader : mergeState.readers) {
final int maxDoc = reader.maxDoc();
final Bits liveDocs = reader.getLiveDocs();
for (int i = 0; i < maxDoc; i++) {
if (liveDocs != null && !liveDocs.get(i)) {
for (int i=0;i<mergeState.storedFieldsReaders.length;i++) {
StoredFieldsReader storedFieldsReader = mergeState.storedFieldsReaders[i];
storedFieldsReader.checkIntegrity();
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;
}
@ -96,13 +100,15 @@ public abstract class StoredFieldsWriter implements Closeable {
// on the fly?
// NOTE: it's very important to first assign to doc then pass it to
// fieldsWriter.addDocument; see LUCENE-1282
StoredDocument doc = reader.document(i);
addDocument(doc, mergeState.fieldInfos);
DocumentStoredFieldVisitor visitor = new DocumentStoredFieldVisitor();
storedFieldsReader.visitDocument(docID, visitor);
StoredDocument doc = visitor.getDocument();
addDocument(doc, mergeState.mergeFieldInfos);
docCount++;
mergeState.checkAbort.work(300);
}
}
finish(mergeState.fieldInfos, docCount);
finish(mergeState.mergeFieldInfos, docCount);
return docCount;
}

View File

@ -21,11 +21,12 @@ import java.io.Closeable;
import java.io.IOException;
import java.util.Iterator;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
@ -177,25 +178,34 @@ public abstract class TermVectorsWriter implements Closeable {
* merging (bulk-byte copying, etc). */
public int merge(MergeState mergeState) throws IOException {
int docCount = 0;
for (int i = 0; i < mergeState.readers.size(); i++) {
final LeafReader reader = mergeState.readers.get(i);
final int maxDoc = reader.maxDoc();
final Bits liveDocs = reader.getLiveDocs();
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();
}
for (int docID = 0; docID < maxDoc; docID++) {
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 = reader.getTermVectors(docID);
Fields vectors;
if (termVectorsReader == null) {
vectors = null;
} else {
vectors = termVectorsReader.get(docID);
}
addAllDocVectors(vectors, mergeState);
docCount++;
mergeState.checkAbort.work(300);
}
}
finish(mergeState.fieldInfos, docCount);
finish(mergeState.mergeFieldInfos, docCount);
return docCount;
}
@ -227,7 +237,7 @@ public abstract class TermVectorsWriter implements Closeable {
int fieldCount = 0;
for(String fieldName : vectors) {
fieldCount++;
final FieldInfo fieldInfo = mergeState.fieldInfos.fieldInfo(fieldName);
final FieldInfo fieldInfo = mergeState.mergeFieldInfos.fieldInfo(fieldName);
assert lastFieldName == null || fieldName.compareTo(lastFieldName) > 0: "lastFieldName=" + lastFieldName + " fieldName=" + fieldName;
lastFieldName = fieldName;

View File

@ -24,11 +24,13 @@ import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.compressing.CompressingStoredFieldsReader.ChunkIterator;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.document.DocumentStoredFieldVisitor;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReader;
@ -331,32 +333,33 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
@Override
public int merge(MergeState mergeState) throws IOException {
int docCount = 0;
int idx = 0;
int numReaders = mergeState.maxDocs.length;
MatchingReaders matching = new MatchingReaders(mergeState);
for (LeafReader reader : mergeState.readers) {
final SegmentReader matchingSegmentReader = matching.matchingSegmentReaders[idx++];
for (int readerIndex=0;readerIndex<numReaders;readerIndex++) {
CompressingStoredFieldsReader matchingFieldsReader = null;
if (matchingSegmentReader != null) {
final StoredFieldsReader fieldsReader = matchingSegmentReader.getFieldsReader();
if (matching.matchingReaders[readerIndex]) {
final StoredFieldsReader fieldsReader = mergeState.storedFieldsReaders[readerIndex];
// we can only bulk-copy if the matching reader is also a CompressingStoredFieldsReader
if (fieldsReader != null && fieldsReader instanceof CompressingStoredFieldsReader) {
matchingFieldsReader = (CompressingStoredFieldsReader) fieldsReader;
}
}
final int maxDoc = reader.maxDoc();
final Bits liveDocs = reader.getLiveDocs();
final int maxDoc = mergeState.maxDocs[readerIndex];
final Bits liveDocs = mergeState.liveDocs[readerIndex];
if (matchingFieldsReader == null
|| matchingFieldsReader.getVersion() != VERSION_CURRENT // means reader version is not the same as the writer version
|| matchingFieldsReader.getCompressionMode() != compressionMode
|| matchingFieldsReader.getChunkSize() != chunkSize) { // the way data is decompressed depends on the chunk size
// naive merge...
StoredFieldsReader storedFieldsReader = mergeState.storedFieldsReaders[readerIndex];
for (int i = nextLiveDoc(0, liveDocs, maxDoc); i < maxDoc; i = nextLiveDoc(i + 1, liveDocs, maxDoc)) {
StoredDocument doc = reader.document(i);
addDocument(doc, mergeState.fieldInfos);
DocumentStoredFieldVisitor visitor = new DocumentStoredFieldVisitor();
storedFieldsReader.visitDocument(i, visitor);
addDocument(visitor.getDocument(), mergeState.mergeFieldInfos);
++docCount;
mergeState.checkAbort.work(300);
}
@ -398,7 +401,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
}
}
}
finish(mergeState.fieldInfos, docCount);
finish(mergeState.mergeFieldInfos, docCount);
return docCount;
}

View File

@ -28,11 +28,12 @@ import java.util.TreeSet;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReader;
@ -727,23 +728,22 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
@Override
public int merge(MergeState mergeState) throws IOException {
int docCount = 0;
int idx = 0;
int numReaders = mergeState.maxDocs.length;
MatchingReaders matching = new MatchingReaders(mergeState);
for (LeafReader reader : mergeState.readers) {
final SegmentReader matchingSegmentReader = matching.matchingSegmentReaders[idx++];
for (int readerIndex=0;readerIndex<numReaders;readerIndex++) {
CompressingTermVectorsReader matchingVectorsReader = null;
if (matchingSegmentReader != null) {
final TermVectorsReader vectorsReader = matchingSegmentReader.getTermVectorsReader();
final TermVectorsReader vectorsReader = mergeState.termVectorsReaders[readerIndex];
if (matching.matchingReaders[readerIndex]) {
// we can only bulk-copy if the matching reader is also a CompressingTermVectorsReader
if (vectorsReader != null && vectorsReader instanceof CompressingTermVectorsReader) {
matchingVectorsReader = (CompressingTermVectorsReader) vectorsReader;
}
}
final int maxDoc = reader.maxDoc();
final Bits liveDocs = reader.getLiveDocs();
final int maxDoc = mergeState.maxDocs[readerIndex];
final Bits liveDocs = mergeState.liveDocs[readerIndex];
if (matchingVectorsReader == null
|| matchingVectorsReader.getVersion() != VERSION_CURRENT
@ -752,7 +752,12 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
|| matchingVectorsReader.getPackedIntsVersion() != PackedInts.VERSION_CURRENT) {
// naive merge...
for (int i = nextLiveDoc(0, liveDocs, maxDoc); i < maxDoc; i = nextLiveDoc(i + 1, liveDocs, maxDoc)) {
final Fields vectors = reader.getTermVectors(i);
Fields vectors;
if (vectorsReader == null) {
vectors = null;
} else {
vectors = vectorsReader.get(i);
}
addAllDocVectors(vectors, mergeState);
++docCount;
mergeState.checkAbort.work(300);
@ -774,8 +779,8 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
&& (i == 0 || index.getStartPointer(i - 1) < startPointer)) { // start of a chunk
final int docBase = vectorsStream.readVInt();
final int chunkDocs = vectorsStream.readVInt();
assert docBase + chunkDocs <= matchingSegmentReader.maxDoc();
if (docBase + chunkDocs < matchingSegmentReader.maxDoc()
assert docBase + chunkDocs <= maxDoc;
if (docBase + chunkDocs < maxDoc
&& nextDeletedDoc(docBase, liveDocs, docBase + chunkDocs) == docBase + chunkDocs) {
final long chunkEnd = index.getStartPointer(docBase + chunkDocs);
final long chunkLength = chunkEnd - vectorsStream.getFilePointer();
@ -789,14 +794,24 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
i = nextLiveDoc(docBase + chunkDocs, liveDocs, maxDoc);
} else {
for (; i < docBase + chunkDocs; i = nextLiveDoc(i + 1, liveDocs, maxDoc)) {
final Fields vectors = reader.getTermVectors(i);
Fields vectors;
if (vectorsReader == null) {
vectors = null;
} else {
vectors = vectorsReader.get(i);
}
addAllDocVectors(vectors, mergeState);
++docCount;
mergeState.checkAbort.work(300);
}
}
} else {
final Fields vectors = reader.getTermVectors(i);
Fields vectors;
if (vectorsReader == null) {
vectors = null;
} else {
vectors = vectorsReader.get(i);
}
addAllDocVectors(vectors, mergeState);
++docCount;
mergeState.checkAbort.work(300);
@ -808,7 +823,7 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
CodecUtil.checkFooter(vectorsStream);
}
}
finish(mergeState.fieldInfos, docCount);
finish(mergeState.mergeFieldInfos, docCount);
return docCount;
}

View File

@ -17,9 +17,10 @@ package org.apache.lucene.codecs.compressing;
* limitations under the License.
*/
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.SegmentReader;
@ -32,7 +33,7 @@ class MatchingReaders {
/** {@link SegmentReader}s that have identical field
* name/number mapping, so their stored fields and term
* vectors may be bulk merged. */
final SegmentReader[] matchingSegmentReaders;
final boolean[] matchingReaders;
/** How many {@link #matchingSegmentReaders} are set. */
final int count;
@ -41,45 +42,33 @@ class MatchingReaders {
// If the i'th reader is a SegmentReader and has
// identical fieldName -> number mapping, then this
// array will be non-null at position i:
int numReaders = mergeState.readers.size();
int numReaders = mergeState.maxDocs.length;
int matchedCount = 0;
matchingSegmentReaders = new SegmentReader[numReaders];
matchingReaders = new boolean[numReaders];
// If this reader is a SegmentReader, and all of its
// field name -> number mappings match the "merged"
// FieldInfos, then we can do a bulk copy of the
// stored fields:
nextReader:
for (int i = 0; i < numReaders; i++) {
LeafReader reader = mergeState.readers.get(i);
// TODO: we may be able to broaden this to
// non-SegmentReaders, since FieldInfos is now
// required? But... this'd also require exposing
// bulk-copy (TVs and stored fields) API in foreign
// readers..
if (reader instanceof SegmentReader) {
SegmentReader segmentReader = (SegmentReader) reader;
boolean same = true;
FieldInfos segmentFieldInfos = segmentReader.getFieldInfos();
for (FieldInfo fi : segmentFieldInfos) {
FieldInfo other = mergeState.fieldInfos.fieldInfo(fi.number);
if (other == null || !other.name.equals(fi.name)) {
same = false;
break;
}
}
if (same) {
matchingSegmentReaders[i] = segmentReader;
matchedCount++;
for (FieldInfo fi : mergeState.fieldInfos[i]) {
FieldInfo other = mergeState.mergeFieldInfos.fieldInfo(fi.number);
if (other == null || !other.name.equals(fi.name)) {
continue nextReader;
}
}
matchingReaders[i] = true;
matchedCount++;
}
this.count = matchedCount;
if (mergeState.infoStream.isEnabled("SM")) {
mergeState.infoStream.message("SM", "merge store matchedCount=" + count + " vs " + mergeState.readers.size());
if (count != mergeState.readers.size()) {
mergeState.infoStream.message("SM", "" + (mergeState.readers.size() - count) + " non-bulk merges");
mergeState.infoStream.message("SM", "merge store matchedCount=" + count + " vs " + numReaders);
if (count != numReaders) {
mergeState.infoStream.message("SM", "" + (numReaders - count) + " non-bulk merges");
}
}
}

View File

@ -2504,7 +2504,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
SegmentMerger merger = new SegmentMerger(mergeReaders, info, infoStream, trackingDir,
MergeState.CheckAbort.NONE, globalFieldNumberMap,
context, config.getCheckIntegrityAtMerge());
context);
if (!merger.shouldMerge()) {
return;
@ -2562,7 +2562,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// above:
success = false;
try {
codec.segmentInfoFormat().getSegmentInfoWriter().write(trackingDir, info, mergeState.fieldInfos, context);
codec.segmentInfoFormat().getSegmentInfoWriter().write(trackingDir, info, mergeState.mergeFieldInfos, context);
success = true;
} finally {
if (!success) {
@ -3901,7 +3901,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
final SegmentMerger merger = new SegmentMerger(merge.getMergeReaders(),
merge.info.info, infoStream, dirWrapper,
checkAbort, globalFieldNumberMap,
context, config.getCheckIntegrityAtMerge());
context);
merge.checkAborted(directory);
@ -3933,11 +3933,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
infoStream.message("IW", "merge away fully deleted segments");
} else {
infoStream.message("IW", "merge codec=" + codec + " docCount=" + merge.info.info.getDocCount() + "; merged segment has " +
(mergeState.fieldInfos.hasVectors() ? "vectors" : "no vectors") + "; " +
(mergeState.fieldInfos.hasNorms() ? "norms" : "no norms") + "; " +
(mergeState.fieldInfos.hasDocValues() ? "docValues" : "no docValues") + "; " +
(mergeState.fieldInfos.hasProx() ? "prox" : "no prox") + "; " +
(mergeState.fieldInfos.hasProx() ? "freqs" : "no freqs"));
(mergeState.mergeFieldInfos.hasVectors() ? "vectors" : "no vectors") + "; " +
(mergeState.mergeFieldInfos.hasNorms() ? "norms" : "no norms") + "; " +
(mergeState.mergeFieldInfos.hasDocValues() ? "docValues" : "no docValues") + "; " +
(mergeState.mergeFieldInfos.hasProx() ? "prox" : "no prox") + "; " +
(mergeState.mergeFieldInfos.hasProx() ? "freqs" : "no freqs"));
}
}
@ -4019,7 +4019,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// above:
boolean success2 = false;
try {
codec.segmentInfoFormat().getSegmentInfoWriter().write(directory, merge.info.info, mergeState.fieldInfos, context);
codec.segmentInfoFormat().getSegmentInfoWriter().write(directory, merge.info.info, mergeState.mergeFieldInfos, context);
success2 = true;
} finally {
if (!success2) {

View File

@ -110,11 +110,6 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
* ram buffers use <code>false</code> */
public final static boolean DEFAULT_USE_COMPOUND_FILE_SYSTEM = true;
/** Default value for calling {@link LeafReader#checkIntegrity()} before
* merging segments (set to <code>false</code>). You can set this
* to <code>true</code> for additional safety. */
public final static boolean DEFAULT_CHECK_INTEGRITY_AT_MERGE = false;
/** Default value for whether calls to {@link IndexWriter#close()} include a commit. */
public final static boolean DEFAULT_COMMIT_ON_CLOSE = true;

View File

@ -94,9 +94,6 @@ public class LiveIndexWriterConfig {
/** True if segment flushes should use compound file format */
protected volatile boolean useCompoundFile = IndexWriterConfig.DEFAULT_USE_COMPOUND_FILE_SYSTEM;
/** True if merging should check integrity of segments before merge */
protected volatile boolean checkIntegrityAtMerge = IndexWriterConfig.DEFAULT_CHECK_INTEGRITY_AT_MERGE;
/** True if calls to {@link IndexWriter#close()} should first do a commit. */
protected boolean commitOnClose = IndexWriterConfig.DEFAULT_COMMIT_ON_CLOSE;
@ -463,26 +460,6 @@ public class LiveIndexWriterConfig {
return useCompoundFile ;
}
/**
* Sets if {@link IndexWriter} should call {@link LeafReader#checkIntegrity()}
* on existing segments before merging them into a new one.
* <p>
* Use <code>true</code> to enable this safety check, which can help
* reduce the risk of propagating index corruption from older segments
* into new ones, at the expense of slower merging.
* </p>
*/
public LiveIndexWriterConfig setCheckIntegrityAtMerge(boolean checkIntegrityAtMerge) {
this.checkIntegrityAtMerge = checkIntegrityAtMerge;
return this;
}
/** Returns true if {@link LeafReader#checkIntegrity()} is called before
* merging segments. */
public boolean getCheckIntegrityAtMerge() {
return checkIntegrityAtMerge;
}
/**
* Returns <code>true</code> if {@link IndexWriter#close()} should first commit before closing.
*/
@ -513,7 +490,6 @@ public class LiveIndexWriterConfig {
sb.append("readerPooling=").append(getReaderPooling()).append("\n");
sb.append("perThreadHardLimitMB=").append(getRAMPerThreadHardLimitMB()).append("\n");
sb.append("useCompoundFile=").append(getUseCompoundFile()).append("\n");
sb.append("checkIntegrityAtMerge=").append(getCheckIntegrityAtMerge()).append("\n");
sb.append("commitOnClose=").append(getCommitOnClose()).append("\n");
return sb.toString();
}

View File

@ -17,9 +17,19 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.packed.PackedInts;
@ -30,6 +40,351 @@ import org.apache.lucene.util.packed.PackedLongValues;
* @lucene.experimental */
public class MergeState {
/** {@link SegmentInfo} of the newly merged segment. */
public final SegmentInfo segmentInfo;
/** {@link FieldInfos} of the newly merged segment. */
public FieldInfos mergeFieldInfos;
public final StoredFieldsReader[] storedFieldsReaders;
public final TermVectorsReader[] termVectorsReaders;
public final NormsProducer[] normsProducers;
public final DocValuesProducer[] docValuesProducers;
public final FieldInfos[] fieldInfos;
/** Live docs for each reader */
public final Bits[] liveDocs;
/** Maps docIDs around deletions. */
public final DocMap[] docMaps;
/** Postings to merge */
public final FieldsProducer[] fieldsProducers;
/** New docID base per reader. */
public final int[] docBase;
/** Max docs per reader */
public final int[] maxDocs;
/** Holds the CheckAbort instance, which is invoked
* periodically to see if the merge has been aborted. */
public final CheckAbort checkAbort;
/** InfoStream for debugging messages. */
public final InfoStream infoStream;
/** Counter used for periodic calls to checkAbort
* @lucene.internal */
public int checkAbortCount;
/** Sole constructor. */
MergeState(List<LeafReader> readers, SegmentInfo segmentInfo, InfoStream infoStream, CheckAbort checkAbort) throws IOException {
int numReaders = readers.size();
docMaps = new DocMap[numReaders];
docBase = new int[numReaders];
maxDocs = new int[numReaders];
fieldsProducers = new FieldsProducer[numReaders];
normsProducers = new NormsProducer[numReaders];
storedFieldsReaders = new StoredFieldsReader[numReaders];
termVectorsReaders = new TermVectorsReader[numReaders];
docValuesProducers = new DocValuesProducer[numReaders];
fieldInfos = new FieldInfos[numReaders];
liveDocs = new Bits[numReaders];
for(int i=0;i<numReaders;i++) {
final LeafReader reader = readers.get(i);
maxDocs[i] = reader.maxDoc();
liveDocs[i] = reader.getLiveDocs();
fieldInfos[i] = reader.getFieldInfos();
NormsProducer normsProducer;
DocValuesProducer docValuesProducer;
StoredFieldsReader storedFieldsReader;
TermVectorsReader termVectorsReader;
FieldsProducer fieldsProducer;
if (reader instanceof SegmentReader) {
SegmentReader segmentReader = (SegmentReader) reader;
normsProducer = segmentReader.getNormsReader();
docValuesProducer = segmentReader.getDocValuesReader();
storedFieldsReader = segmentReader.getFieldsReader();
termVectorsReader = segmentReader.getTermVectorsReader();
fieldsProducer = segmentReader.fields();
} else {
// A "foreign" reader
normsProducer = readerToNormsProducer(reader);
docValuesProducer = readerToDocValuesProducer(reader);
storedFieldsReader = readerToStoredFieldsReader(reader);
termVectorsReader = readerToTermVectorsReader(reader);
fieldsProducer = readerToFieldsProducer(reader);
}
normsProducers[i] = normsProducer;
docValuesProducers[i] = docValuesProducer;
storedFieldsReaders[i] = storedFieldsReader;
termVectorsReaders[i] = termVectorsReader;
fieldsProducers[i] = fieldsProducer;
}
this.segmentInfo = segmentInfo;
this.infoStream = infoStream;
this.checkAbort = checkAbort;
setDocMaps(readers);
}
private NormsProducer readerToNormsProducer(final LeafReader reader) {
return new NormsProducer() {
@Override
public NumericDocValues getNorms(FieldInfo field) throws IOException {
return reader.getNormValues(field.name);
}
@Override
public void checkIntegrity() throws IOException {
// We already checkIntegrity the entire reader up front in SegmentMerger
}
@Override
public void close() {
}
@Override
public long ramBytesUsed() {
return 0;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
};
}
private DocValuesProducer readerToDocValuesProducer(final LeafReader reader) {
return new DocValuesProducer() {
@Override
public NumericDocValues getNumeric(FieldInfo field) throws IOException {
return reader.getNumericDocValues(field.name);
}
@Override
public BinaryDocValues getBinary(FieldInfo field) throws IOException {
return reader.getBinaryDocValues(field.name);
}
@Override
public SortedDocValues getSorted(FieldInfo field) throws IOException {
return reader.getSortedDocValues(field.name);
}
@Override
public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
return reader.getSortedNumericDocValues(field.name);
}
@Override
public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
return reader.getSortedSetDocValues(field.name);
}
@Override
public Bits getDocsWithField(FieldInfo field) throws IOException {
return reader.getDocsWithField(field.name);
}
@Override
public void checkIntegrity() throws IOException {
// We already checkIntegrity the entire reader up front in SegmentMerger
}
@Override
public void close() {
}
@Override
public long ramBytesUsed() {
return 0;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
};
}
private StoredFieldsReader readerToStoredFieldsReader(final LeafReader reader) {
return new StoredFieldsReader() {
@Override
public void visitDocument(int docID, StoredFieldVisitor visitor) throws IOException {
reader.document(docID, visitor);
}
@Override
public StoredFieldsReader clone() {
return readerToStoredFieldsReader(reader);
}
@Override
public void checkIntegrity() throws IOException {
// We already checkIntegrity the entire reader up front in SegmentMerger
}
@Override
public void close() {
}
@Override
public long ramBytesUsed() {
return 0;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
};
}
private TermVectorsReader readerToTermVectorsReader(final LeafReader reader) {
return new TermVectorsReader() {
@Override
public Fields get(int docID) throws IOException {
return reader.getTermVectors(docID);
}
@Override
public TermVectorsReader clone() {
return readerToTermVectorsReader(reader);
}
@Override
public void checkIntegrity() throws IOException {
// We already checkIntegrity the entire reader up front in SegmentMerger
}
@Override
public void close() {
}
@Override
public long ramBytesUsed() {
return 0;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
};
}
private FieldsProducer readerToFieldsProducer(final LeafReader reader) throws IOException {
final Fields fields = reader.fields();
return new FieldsProducer() {
@Override
public Iterator<String> iterator() {
return fields.iterator();
}
@Override
public Terms terms(String field) throws IOException {
return fields.terms(field);
}
@Override
public int size() {
return fields.size();
}
@Override
public void checkIntegrity() throws IOException {
// We already checkIntegrity the entire reader up front in SegmentMerger
}
@Override
public void close() {
}
@Override
public long ramBytesUsed() {
return 0;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
return Collections.emptyList();
}
};
}
// NOTE: removes any "all deleted" readers from mergeState.readers
private void setDocMaps(List<LeafReader> readers) throws IOException {
final int numReaders = maxDocs.length;
// Remap docIDs
int docBase = 0;
for(int i=0;i<numReaders;i++) {
final LeafReader reader = readers.get(i);
this.docBase[i] = docBase;
final DocMap docMap = DocMap.build(reader);
docMaps[i] = docMap;
docBase += docMap.numDocs();
}
segmentInfo.setDocCount(docBase);
}
/**
* Class for recording units of work when merging segments.
*/
public static class CheckAbort {
private double workCount;
private final MergePolicy.OneMerge merge;
private final Directory dir;
/** Creates a #CheckAbort instance. */
public CheckAbort(MergePolicy.OneMerge merge, Directory dir) {
this.merge = merge;
this.dir = dir;
}
/**
* Records the fact that roughly units amount of work
* have been done since this method was last called.
* When adding time-consuming code into SegmentMerger,
* you should test different values for units to ensure
* that the time in between calls to merge.checkAborted
* is up to ~ 1 second.
*/
public void work(double units) throws MergePolicy.MergeAbortedException {
workCount += units;
if (workCount >= 10000.0) {
merge.checkAborted(dir);
workCount = 0;
}
}
/** If you use this: IW.close(false) cannot abort your merge!
* @lucene.internal */
static final MergeState.CheckAbort NONE = new MergeState.CheckAbort(null, null) {
@Override
public void work(double units) {
// do nothing
}
};
}
/**
* Remaps docids around deletes during merge
*/
@ -100,10 +455,8 @@ public class MergeState {
public int numDeletedDocs() {
return numDeletedDocs;
}
};
}
}
private static final class NoDelDocMap extends DocMap {
@ -129,78 +482,4 @@ public class MergeState {
return 0;
}
}
/** {@link SegmentInfo} of the newly merged segment. */
public final SegmentInfo segmentInfo;
/** {@link FieldInfos} of the newly merged segment. */
public FieldInfos fieldInfos;
/** Readers being merged. */
public final List<LeafReader> readers;
/** Maps docIDs around deletions. */
public DocMap[] docMaps;
/** New docID base per reader. */
public int[] docBase;
/** Holds the CheckAbort instance, which is invoked
* periodically to see if the merge has been aborted. */
public final CheckAbort checkAbort;
/** InfoStream for debugging messages. */
public final InfoStream infoStream;
/** Counter used for periodic calls to checkAbort
* @lucene.internal */
public int checkAbortCount;
/** Sole constructor. */
MergeState(List<LeafReader> readers, SegmentInfo segmentInfo, InfoStream infoStream, CheckAbort checkAbort) {
this.readers = readers;
this.segmentInfo = segmentInfo;
this.infoStream = infoStream;
this.checkAbort = checkAbort;
}
/**
* Class for recording units of work when merging segments.
*/
public static class CheckAbort {
private double workCount;
private final MergePolicy.OneMerge merge;
private final Directory dir;
/** Creates a #CheckAbort instance. */
public CheckAbort(MergePolicy.OneMerge merge, Directory dir) {
this.merge = merge;
this.dir = dir;
}
/**
* Records the fact that roughly units amount of work
* have been done since this method was last called.
* When adding time-consuming code into SegmentMerger,
* you should test different values for units to ensure
* that the time in between calls to merge.checkAborted
* is up to ~ 1 second.
*/
public void work(double units) throws MergePolicy.MergeAbortedException {
workCount += units;
if (workCount >= 10000.0) {
merge.checkAborted(dir);
workCount = 0;
}
}
/** If you use this: IW.close(false) cannot abort your merge!
* @lucene.internal */
static final MergeState.CheckAbort NONE = new MergeState.CheckAbort(null, null) {
@Override
public void work(double units) {
// do nothing
}
};
}
}

View File

@ -23,7 +23,7 @@ package org.apache.lucene.index;
public abstract class NumericDocValues {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
* constructors, typically implicit.) */
protected NumericDocValues() {}
/**

View File

@ -148,8 +148,9 @@ public final class SegmentInfo {
// NOTE: leave package private
void setDocCount(int docCount) {
if (this.docCount != -1) {
throw new IllegalStateException("docCount was already set");
// nocommit what to do... this only happens when docCount == 0 ... see IW comment " // would result in a 0 document segment: nothing to merge!"
if (this.docCount != -1 && this.docCount != docCount) {
throw new IllegalStateException("docCount was already set: this.docCount=" + this.docCount + " vs docCount=" + docCount);
}
this.docCount = docCount;
}

View File

@ -51,19 +51,21 @@ final class SegmentMerger {
// note, just like in codec apis Directory 'dir' is NOT the same as segmentInfo.dir!!
SegmentMerger(List<LeafReader> readers, SegmentInfo segmentInfo, InfoStream infoStream, Directory dir,
MergeState.CheckAbort checkAbort, FieldInfos.FieldNumbers fieldNumbers, IOContext context, boolean validate) throws IOException {
MergeState.CheckAbort checkAbort, FieldInfos.FieldNumbers fieldNumbers, IOContext context) throws IOException {
// validate incoming readers
if (validate) {
for (LeafReader reader : readers) {
for (LeafReader reader : readers) {
if ((reader instanceof SegmentReader) == false) {
// We only validate foreign readers up front: each index component
// calls .checkIntegrity itself for each incoming producer
reader.checkIntegrity();
}
}
mergeState = new MergeState(readers, segmentInfo, infoStream, checkAbort);
directory = dir;
this.codec = segmentInfo.getCodec();
this.context = context;
this.fieldInfosBuilder = new FieldInfos.Builder(fieldNumbers);
mergeState.segmentInfo.setDocCount(setDocMaps());
}
/** True if any merging should happen */
@ -86,7 +88,7 @@ final class SegmentMerger {
// method that will spend alot of time. The frequency
// of this check impacts how long
// IndexWriter.close(false) takes to actually stop the
// threads.
// background merge threads.
mergeFieldInfos();
long t0 = 0;
if (mergeState.infoStream.isEnabled("SM")) {
@ -97,10 +99,10 @@ final class SegmentMerger {
long t1 = System.nanoTime();
mergeState.infoStream.message("SM", ((t1-t0)/1000000) + " msec to merge stored fields [" + numMerged + " docs]");
}
assert numMerged == mergeState.segmentInfo.getDocCount();
assert numMerged == mergeState.segmentInfo.getDocCount(): "numMerged=" + numMerged + " vs mergeState.segmentInfo.getDocCount()=" + mergeState.segmentInfo.getDocCount();
final SegmentWriteState segmentWriteState = new SegmentWriteState(mergeState.infoStream, directory, mergeState.segmentInfo,
mergeState.fieldInfos, null, context);
mergeState.mergeFieldInfos, null, context);
if (mergeState.infoStream.isEnabled("SM")) {
t0 = System.nanoTime();
}
@ -113,7 +115,7 @@ final class SegmentMerger {
if (mergeState.infoStream.isEnabled("SM")) {
t0 = System.nanoTime();
}
if (mergeState.fieldInfos.hasDocValues()) {
if (mergeState.mergeFieldInfos.hasDocValues()) {
mergeDocValues(segmentWriteState);
}
if (mergeState.infoStream.isEnabled("SM")) {
@ -121,7 +123,7 @@ final class SegmentMerger {
mergeState.infoStream.message("SM", ((t1-t0)/1000000) + " msec to merge doc values [" + numMerged + " docs]");
}
if (mergeState.fieldInfos.hasNorms()) {
if (mergeState.mergeFieldInfos.hasNorms()) {
if (mergeState.infoStream.isEnabled("SM")) {
t0 = System.nanoTime();
}
@ -132,7 +134,7 @@ final class SegmentMerger {
}
}
if (mergeState.fieldInfos.hasVectors()) {
if (mergeState.mergeFieldInfos.hasVectors()) {
if (mergeState.infoStream.isEnabled("SM")) {
t0 = System.nanoTime();
}
@ -146,7 +148,7 @@ final class SegmentMerger {
// write the merged infos
FieldInfosWriter fieldInfosWriter = codec.fieldInfosFormat().getFieldInfosWriter();
fieldInfosWriter.write(directory, mergeState.segmentInfo, "", mergeState.fieldInfos, context);
fieldInfosWriter.write(directory, mergeState.segmentInfo, "", mergeState.mergeFieldInfos, context);
return mergeState;
}
@ -182,13 +184,12 @@ final class SegmentMerger {
}
public void mergeFieldInfos() throws IOException {
for (LeafReader reader : mergeState.readers) {
FieldInfos readerFieldInfos = reader.getFieldInfos();
for (FieldInfos readerFieldInfos : mergeState.fieldInfos) {
for (FieldInfo fi : readerFieldInfos) {
fieldInfosBuilder.add(fi);
}
}
mergeState.fieldInfos = fieldInfosBuilder.finish();
mergeState.mergeFieldInfos = fieldInfosBuilder.finish();
}
/**
@ -237,32 +238,6 @@ final class SegmentMerger {
return numDocs;
}
// NOTE: removes any "all deleted" readers from mergeState.readers
private int setDocMaps() throws IOException {
final int numReaders = mergeState.readers.size();
// Remap docIDs
mergeState.docMaps = new MergeState.DocMap[numReaders];
mergeState.docBase = new int[numReaders];
int docBase = 0;
int i = 0;
while(i < mergeState.readers.size()) {
final LeafReader reader = mergeState.readers.get(i);
mergeState.docBase[i] = docBase;
final MergeState.DocMap docMap = MergeState.DocMap.build(reader);
mergeState.docMaps[i] = docMap;
docBase += docMap.numDocs();
i++;
}
return docBase;
}
private void mergeTerms(SegmentWriteState segmentWriteState) throws IOException {
FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState);
boolean success = false;

View File

@ -566,7 +566,7 @@ public final class SegmentReader extends LeafReader implements Accountable {
@Override
public void checkIntegrity() throws IOException {
ensureOpen();
// stored fields
getFieldsReader().checkIntegrity();

View File

@ -223,7 +223,7 @@ public class TestDoc extends LuceneTestCase {
SegmentMerger merger = new SegmentMerger(Arrays.<LeafReader>asList(r1, r2),
si, InfoStream.getDefault(), trackingDir,
MergeState.CheckAbort.NONE, new FieldInfos.FieldNumbers(), context, true);
MergeState.CheckAbort.NONE, new FieldInfos.FieldNumbers(), context);
MergeState mergeState = merger.merge();
r1.close();

View File

@ -83,7 +83,7 @@ public class TestSegmentMerger extends LuceneTestCase {
SegmentMerger merger = new SegmentMerger(Arrays.<LeafReader>asList(reader1, reader2),
si, InfoStream.getDefault(), mergedDir,
MergeState.CheckAbort.NONE, new FieldInfos.FieldNumbers(), newIOContext(random()), true);
MergeState.CheckAbort.NONE, new FieldInfos.FieldNumbers(), newIOContext(random()));
MergeState mergeState = merger.merge();
int docsMerged = mergeState.segmentInfo.getDocCount();
assertTrue(docsMerged == 2);

View File

@ -867,7 +867,6 @@ public abstract class LuceneTestCase extends Assert {
}
c.setUseCompoundFile(r.nextBoolean());
c.setReaderPooling(r.nextBoolean());
c.setCheckIntegrityAtMerge(r.nextBoolean());
return c;
}
@ -1040,12 +1039,6 @@ public abstract class LuceneTestCase extends Assert {
didChange = true;
}
if (rarely(r)) {
// change merge integrity check parameters
c.setCheckIntegrityAtMerge(r.nextBoolean());
didChange = true;
}
if (rarely(r)) {
// change CMS merge parameters
MergeScheduler ms = c.getMergeScheduler();