LUCENE-6165: Change merging APIs from LeafReader to CodecReader

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1650301 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2015-01-08 14:53:03 +00:00
parent b2f3330f7d
commit ec43ca9fd8
31 changed files with 894 additions and 611 deletions

View File

@ -347,8 +347,8 @@ API Changes
be set at the constructor for non-contextual lookup.
(Boon Low, Tomás Fernández Löbbe)
* LUCENE-6158: IndexWriter.addIndexes(IndexReader...) changed to
addIndexes(LeafReader...) (Robert Muir)
* LUCENE-6158, LUCENE-6165: IndexWriter.addIndexes(IndexReader...) changed to
addIndexes(CodecReader...) (Robert Muir)
Bug Fixes

View File

@ -26,13 +26,15 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.CodecReader;
import org.apache.lucene.index.SlowCodecReaderWrapper;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
/**
* Adds an input index to an existing index, using
* {@link IndexWriter#addIndexes(Directory...)} or
* {@link IndexWriter#addIndexes(LeafReader...)}. The location of the input
* {@link IndexWriter#addIndexes(CodecReader...)}. The location of the input
* index is specified by the parameter {@link #ADDINDEXES_INPUT_DIR} and is
* assumed to be a directory on the file system.
* <p>
@ -67,10 +69,10 @@ public class AddIndexesTask extends PerfTask {
writer.addIndexes(inputDir);
} else {
try (IndexReader r = DirectoryReader.open(inputDir)) {
LeafReader leaves[] = new LeafReader[r.leaves().size()];
CodecReader leaves[] = new CodecReader[r.leaves().size()];
int i = 0;
for (LeafReaderContext leaf : r.leaves()) {
leaves[i++] = leaf.reader();
leaves[i++] = SlowCodecReaderWrapper.wrap(leaf.reader());
}
writer.addIndexes(leaves);
}
@ -84,7 +86,7 @@ public class AddIndexesTask extends PerfTask {
* @param params
* {@code useAddIndexesDir=true} for using
* {@link IndexWriter#addIndexes(Directory...)} or {@code false} for
* using {@link IndexWriter#addIndexes(LeafReader...)}. Defaults to
* using {@link IndexWriter#addIndexes(CodecReader...)}. Defaults to
* {@code true}.
*/
@Override

View File

@ -0,0 +1,379 @@
package org.apache.lucene.index;
/*
* 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.
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
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.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.IOUtils;
/**
* LeafReader implemented by codec APIs.
*/
public abstract class CodecReader extends LeafReader implements Accountable {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected CodecReader() {}
/**
* Expert: retrieve thread-private StoredFieldsReader
* @lucene.internal
*/
public abstract StoredFieldsReader getFieldsReader();
/**
* Expert: retrieve thread-private TermVectorsReader
* @lucene.internal
*/
public abstract TermVectorsReader getTermVectorsReader();
/**
* Expert: retrieve underlying NormsProducer
* @lucene.internal
*/
public abstract NormsProducer getNormsReader();
/**
* Expert: retrieve underlying DocValuesProducer
* @lucene.internal
*/
public abstract DocValuesProducer getDocValuesReader();
/**
* Expert: retrieve underlying FieldsProducer
* @lucene.internal
*/
public abstract FieldsProducer getPostingsReader();
@Override
public final void document(int docID, StoredFieldVisitor visitor) throws IOException {
checkBounds(docID);
getFieldsReader().visitDocument(docID, visitor);
}
@Override
public final Fields getTermVectors(int docID) throws IOException {
TermVectorsReader termVectorsReader = getTermVectorsReader();
if (termVectorsReader == null) {
return null;
}
checkBounds(docID);
return termVectorsReader.get(docID);
}
private void checkBounds(int docID) {
if (docID < 0 || docID >= maxDoc()) {
throw new IndexOutOfBoundsException("docID must be >= 0 and < maxDoc=" + maxDoc() + " (got docID=" + docID + ")");
}
}
@Override
public final Fields fields() {
return getPostingsReader();
}
final CloseableThreadLocal<Map<String,Object>> docValuesLocal = new CloseableThreadLocal<Map<String,Object>>() {
@Override
protected Map<String,Object> initialValue() {
return new HashMap<>();
}
};
final CloseableThreadLocal<Map<String,Bits>> docsWithFieldLocal = new CloseableThreadLocal<Map<String,Bits>>() {
@Override
protected Map<String,Bits> initialValue() {
return new HashMap<>();
}
};
// returns the FieldInfo that corresponds to the given field and type, or
// null if the field does not exist, or not indexed as the requested
// DovDocValuesType.
private FieldInfo getDVField(String field, DocValuesType type) {
FieldInfo fi = getFieldInfos().fieldInfo(field);
if (fi == null) {
// Field does not exist
return null;
}
if (fi.getDocValuesType() == DocValuesType.NONE) {
// Field was not indexed with doc values
return null;
}
if (fi.getDocValuesType() != type) {
// Field DocValues are different than requested type
return null;
}
return fi;
}
@Override
public final NumericDocValues getNumericDocValues(String field) throws IOException {
ensureOpen();
Map<String,Object> dvFields = docValuesLocal.get();
Object previous = dvFields.get(field);
if (previous != null && previous instanceof NumericDocValues) {
return (NumericDocValues) previous;
} else {
FieldInfo fi = getDVField(field, DocValuesType.NUMERIC);
if (fi == null) {
return null;
}
NumericDocValues dv = getDocValuesReader().getNumeric(fi);
dvFields.put(field, dv);
return dv;
}
}
@Override
public final Bits getDocsWithField(String field) throws IOException {
ensureOpen();
Map<String,Bits> dvFields = docsWithFieldLocal.get();
Bits previous = dvFields.get(field);
if (previous != null) {
return previous;
} else {
FieldInfo fi = getFieldInfos().fieldInfo(field);
if (fi == null) {
// Field does not exist
return null;
}
if (fi.getDocValuesType() == DocValuesType.NONE) {
// Field was not indexed with doc values
return null;
}
Bits dv = getDocValuesReader().getDocsWithField(fi);
dvFields.put(field, dv);
return dv;
}
}
@Override
public final BinaryDocValues getBinaryDocValues(String field) throws IOException {
ensureOpen();
FieldInfo fi = getDVField(field, DocValuesType.BINARY);
if (fi == null) {
return null;
}
Map<String,Object> dvFields = docValuesLocal.get();
BinaryDocValues dvs = (BinaryDocValues) dvFields.get(field);
if (dvs == null) {
dvs = getDocValuesReader().getBinary(fi);
dvFields.put(field, dvs);
}
return dvs;
}
@Override
public final SortedDocValues getSortedDocValues(String field) throws IOException {
ensureOpen();
Map<String,Object> dvFields = docValuesLocal.get();
Object previous = dvFields.get(field);
if (previous != null && previous instanceof SortedDocValues) {
return (SortedDocValues) previous;
} else {
FieldInfo fi = getDVField(field, DocValuesType.SORTED);
if (fi == null) {
return null;
}
SortedDocValues dv = getDocValuesReader().getSorted(fi);
dvFields.put(field, dv);
return dv;
}
}
@Override
public final SortedNumericDocValues getSortedNumericDocValues(String field) throws IOException {
ensureOpen();
Map<String,Object> dvFields = docValuesLocal.get();
Object previous = dvFields.get(field);
if (previous != null && previous instanceof SortedNumericDocValues) {
return (SortedNumericDocValues) previous;
} else {
FieldInfo fi = getDVField(field, DocValuesType.SORTED_NUMERIC);
if (fi == null) {
return null;
}
SortedNumericDocValues dv = getDocValuesReader().getSortedNumeric(fi);
dvFields.put(field, dv);
return dv;
}
}
@Override
public final SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
ensureOpen();
Map<String,Object> dvFields = docValuesLocal.get();
Object previous = dvFields.get(field);
if (previous != null && previous instanceof SortedSetDocValues) {
return (SortedSetDocValues) previous;
} else {
FieldInfo fi = getDVField(field, DocValuesType.SORTED_SET);
if (fi == null) {
return null;
}
SortedSetDocValues dv = getDocValuesReader().getSortedSet(fi);
dvFields.put(field, dv);
return dv;
}
}
final CloseableThreadLocal<Map<String,NumericDocValues>> normsLocal = new CloseableThreadLocal<Map<String,NumericDocValues>>() {
@Override
protected Map<String,NumericDocValues> initialValue() {
return new HashMap<>();
}
};
@Override
public final NumericDocValues getNormValues(String field) throws IOException {
ensureOpen();
Map<String,NumericDocValues> normFields = normsLocal.get();
NumericDocValues norms = normFields.get(field);
if (norms != null) {
return norms;
} else {
FieldInfo fi = getFieldInfos().fieldInfo(field);
if (fi == null || !fi.hasNorms()) {
// Field does not exist or does not index norms
return null;
}
norms = getNormsReader().getNorms(fi);
normFields.put(field, norms);
return norms;
}
}
@Override
protected void doClose() throws IOException {
IOUtils.close(docValuesLocal, docsWithFieldLocal, normsLocal);
}
@Override
public long ramBytesUsed() {
ensureOpen();
// terms/postings
long ramBytesUsed = getPostingsReader().ramBytesUsed();
// norms
if (getNormsReader() != null) {
ramBytesUsed += getNormsReader().ramBytesUsed();
}
// docvalues
if (getDocValuesReader() != null) {
ramBytesUsed += getDocValuesReader().ramBytesUsed();
}
// stored fields
if (getFieldsReader() != null) {
ramBytesUsed += getFieldsReader().ramBytesUsed();
}
// term vectors
if (getTermVectorsReader() != null) {
ramBytesUsed += getTermVectorsReader().ramBytesUsed();
}
return ramBytesUsed;
}
@Override
public Collection<Accountable> getChildResources() {
ensureOpen();
List<Accountable> resources = new ArrayList<>();
// terms/postings
resources.add(Accountables.namedAccountable("postings", getPostingsReader()));
// norms
if (getNormsReader() != null) {
resources.add(Accountables.namedAccountable("norms", getNormsReader()));
}
// docvalues
if (getDocValuesReader() != null) {
resources.add(Accountables.namedAccountable("docvalues", getDocValuesReader()));
}
// stored fields
if (getFieldsReader() != null) {
resources.add(Accountables.namedAccountable("stored fields", getFieldsReader()));
}
// term vectors
if (getTermVectorsReader() != null) {
resources.add(Accountables.namedAccountable("term vectors", getTermVectorsReader()));
}
return Collections.unmodifiableList(resources);
}
@Override
public void checkIntegrity() throws IOException {
ensureOpen();
// terms/postings
getPostingsReader().checkIntegrity();
// norms
if (getNormsReader() != null) {
getNormsReader().checkIntegrity();
}
// docvalues
if (getDocValuesReader() != null) {
getDocValuesReader().checkIntegrity();
}
// stored fields
if (getFieldsReader() != null) {
getFieldsReader().checkIntegrity();
}
// term vectors
if (getTermVectorsReader() != null) {
getTermVectorsReader().checkIntegrity();
}
}
}

View File

@ -0,0 +1,102 @@
package org.apache.lucene.index;
/*
* 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.
*/
import java.util.Objects;
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.util.Bits;
/**
* A <code>FilterCodecReader</code> contains another CodecReader, which it
* uses as its basic source of data, possibly transforming the data along the
* way or providing additional functionality.
*/
public class FilterCodecReader extends CodecReader {
/**
* The underlying CodecReader instance.
*/
protected final CodecReader in;
/**
* Creates a new FilterCodecReader.
* @param in the underlying CodecReader instance.
*/
public FilterCodecReader(CodecReader in) {
this.in = Objects.requireNonNull(in);
}
@Override
public StoredFieldsReader getFieldsReader() {
return in.getFieldsReader();
}
@Override
public TermVectorsReader getTermVectorsReader() {
return in.getTermVectorsReader();
}
@Override
public NormsProducer getNormsReader() {
return in.getNormsReader();
}
@Override
public DocValuesProducer getDocValuesReader() {
return in.getDocValuesReader();
}
@Override
public FieldsProducer getPostingsReader() {
return in.getPostingsReader();
}
@Override
public Bits getLiveDocs() {
return in.getLiveDocs();
}
@Override
public FieldInfos getFieldInfos() {
return in.getFieldInfos();
}
@Override
public int numDocs() {
return in.numDocs();
}
@Override
public int maxDoc() {
return in.maxDoc();
}
@Override
public void addCoreClosedListener(CoreClosedListener listener) {
in.addCoreClosedListener(listener);
}
@Override
public void removeCoreClosedListener(CoreClosedListener listener) {
in.removeCoreClosedListener(listener);
}
}

View File

@ -235,8 +235,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
public static final String SOURCE_MERGE = "merge";
/** Source of a segment which results from a flush. */
public static final String SOURCE_FLUSH = "flush";
/** Source of a segment which results from a call to {@link #addIndexes(LeafReader...)}. */
public static final String SOURCE_ADDINDEXES_READERS = "addIndexes(LeafReader...)";
/** Source of a segment which results from a call to {@link #addIndexes(CodecReader...)}. */
public static final String SOURCE_ADDINDEXES_READERS = "addIndexes(CodecReader...)";
/**
* Absolute hard maximum length for a term, in bytes once
@ -2099,7 +2099,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
*
* <p>
* NOTE: this method will forcefully abort all merges in progress. If other
* threads are running {@link #forceMerge}, {@link #addIndexes(LeafReader[])}
* threads are running {@link #forceMerge}, {@link #addIndexes(CodecReader[])}
* or {@link #forceMergeDeletes} methods, they may receive
* {@link MergePolicy.MergeAbortedException}s.
*/
@ -2509,18 +2509,18 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* @throws IOException
* if there is a low-level IO error
*/
public void addIndexes(LeafReader... readers) throws IOException {
public void addIndexes(CodecReader... readers) throws IOException {
ensureOpen();
int numDocs = 0;
try {
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "flush at addIndexes(LeafReader...)");
infoStream.message("IW", "flush at addIndexes(CodecReader...)");
}
flush(false, true);
String mergedName = newSegmentName();
for (LeafReader leaf : readers) {
for (CodecReader leaf : readers) {
numDocs += leaf.numDocs();
}

View File

@ -145,12 +145,12 @@ public abstract class MergePolicy {
* reorders doc IDs, it must override {@link #getDocMap} too so that
* deletes that happened during the merge can be applied to the newly
* merged segment. */
public List<LeafReader> getMergeReaders() throws IOException {
public List<CodecReader> getMergeReaders() throws IOException {
if (readers == null) {
throw new IllegalStateException("IndexWriter has not initialized readers from the segment infos yet");
}
final List<LeafReader> readers = new ArrayList<>(this.readers.size());
for (LeafReader reader : this.readers) {
final List<CodecReader> readers = new ArrayList<>(this.readers.size());
for (SegmentReader reader : this.readers) {
if (reader.numDocs() > 0) {
readers.add(reader);
}

View File

@ -18,7 +18,6 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import org.apache.lucene.codecs.DocValuesProducer;
@ -76,7 +75,7 @@ public class MergeState {
public final InfoStream infoStream;
/** Sole constructor. */
MergeState(List<LeafReader> readers, SegmentInfo segmentInfo, InfoStream infoStream) throws IOException {
MergeState(List<CodecReader> readers, SegmentInfo segmentInfo, InfoStream infoStream) throws IOException {
int numReaders = readers.size();
docMaps = new DocMap[numReaders];
@ -91,50 +90,33 @@ public class MergeState {
liveDocs = new Bits[numReaders];
for(int i=0;i<numReaders;i++) {
final LeafReader reader = readers.get(i);
final CodecReader 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();
if (normsProducer != null) {
normsProducer = normsProducer.getMergeInstance();
}
docValuesProducer = segmentReader.getDocValuesReader();
if (docValuesProducer != null) {
docValuesProducer = docValuesProducer.getMergeInstance();
}
storedFieldsReader = segmentReader.getFieldsReader();
if (storedFieldsReader != null) {
storedFieldsReader = storedFieldsReader.getMergeInstance();
}
termVectorsReader = segmentReader.getTermVectorsReader();
if (termVectorsReader != null) {
termVectorsReader = termVectorsReader.getMergeInstance();
}
fieldsProducer = segmentReader.fields().getMergeInstance();
} else {
// A "foreign" reader
normsProducer = readerToNormsProducer(reader);
docValuesProducer = readerToDocValuesProducer(reader);
storedFieldsReader = readerToStoredFieldsReader(reader);
termVectorsReader = readerToTermVectorsReader(reader);
fieldsProducer = readerToFieldsProducer(reader);
normsProducers[i] = reader.getNormsReader();
if (normsProducers[i] != null) {
normsProducers[i] = normsProducers[i].getMergeInstance();
}
normsProducers[i] = normsProducer;
docValuesProducers[i] = docValuesProducer;
storedFieldsReaders[i] = storedFieldsReader;
termVectorsReaders[i] = termVectorsReader;
fieldsProducers[i] = fieldsProducer;
docValuesProducers[i] = reader.getDocValuesReader();
if (docValuesProducers[i] != null) {
docValuesProducers[i] = docValuesProducers[i].getMergeInstance();
}
storedFieldsReaders[i] = reader.getFieldsReader();
if (storedFieldsReaders[i] != null) {
storedFieldsReaders[i] = storedFieldsReaders[i].getMergeInstance();
}
termVectorsReaders[i] = reader.getTermVectorsReader();
if (termVectorsReaders[i] != null) {
termVectorsReaders[i] = termVectorsReaders[i].getMergeInstance();
}
fieldsProducers[i] = reader.getPostingsReader().getMergeInstance();
}
this.segmentInfo = segmentInfo;
@ -143,177 +125,14 @@ public class MergeState {
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;
}
};
}
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;
}
};
}
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;
}
};
}
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;
}
};
}
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;
}
};
}
// NOTE: removes any "all deleted" readers from mergeState.readers
private void setDocMaps(List<LeafReader> readers) throws IOException {
private void setDocMaps(List<CodecReader> 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);
final CodecReader reader = readers.get(i);
this.docBase[i] = docBase;
final DocMap docMap = DocMap.build(reader);
docMaps[i] = docMap;
@ -352,7 +171,7 @@ public class MergeState {
/** Creates a {@link DocMap} instance appropriate for
* this reader. */
public static DocMap build(LeafReader reader) {
public static DocMap build(CodecReader reader) {
final int maxDoc = reader.maxDoc();
if (!reader.hasDeletions()) {
return new NoDelDocMap(maxDoc);

View File

@ -19,9 +19,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
@ -35,16 +33,12 @@ import org.apache.lucene.index.LeafReader.CoreClosedListener;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
/** Holds core readers that are shared (unchanged) when
* SegmentReader is cloned or reopened */
final class SegmentCoreReaders implements Accountable {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(SegmentCoreReaders.class);
final class SegmentCoreReaders {
// Counts how many other readers share the core objects
// (freqStream, proxStream, tis, etc.) of this reader;
@ -84,13 +78,6 @@ final class SegmentCoreReaders implements Accountable {
}
};
final CloseableThreadLocal<Map<String,Object>> normsLocal = new CloseableThreadLocal<Map<String,Object>>() {
@Override
protected Map<String,Object> initialValue() {
return new HashMap<>();
}
};
private final Set<CoreClosedListener> coreClosedListeners =
Collections.synchronizedSet(new LinkedHashSet<CoreClosedListener>());
@ -157,31 +144,12 @@ final class SegmentCoreReaders implements Accountable {
throw new AlreadyClosedException("SegmentCoreReaders is already closed");
}
NumericDocValues getNormValues(FieldInfos infos, String field) throws IOException {
Map<String,Object> normFields = normsLocal.get();
NumericDocValues norms = (NumericDocValues) normFields.get(field);
if (norms != null) {
return norms;
} else {
FieldInfo fi = infos.fieldInfo(field);
if (fi == null || !fi.hasNorms()) {
// Field does not exist or does not index norms
return null;
}
assert normsProducer != null;
norms = normsProducer.getNorms(fi);
normFields.put(field, norms);
return norms;
}
}
void decRef() throws IOException {
if (ref.decrementAndGet() == 0) {
// System.err.println("--- closing core readers");
Throwable th = null;
try {
IOUtils.close(termVectorsLocal, fieldsReaderLocal, normsLocal, fields, termVectorsReaderOrig, fieldsReaderOrig,
IOUtils.close(termVectorsLocal, fieldsReaderLocal, fields, termVectorsReaderOrig, fieldsReaderOrig,
cfsReader, normsProducer);
} catch (Throwable throwable) {
th = throwable;
@ -217,14 +185,4 @@ final class SegmentCoreReaders implements Accountable {
void removeCoreClosedListener(CoreClosedListener listener) {
coreClosedListeners.remove(listener);
}
// TODO: remove this, it can just be on SR
@Override
public long ramBytesUsed() {
return BASE_RAM_BYTES_USED +
((normsProducer!=null) ? normsProducer.ramBytesUsed() : 0) +
((fields!=null) ? fields.ramBytesUsed() : 0) +
((fieldsReaderOrig!=null)? fieldsReaderOrig.ramBytesUsed() : 0) +
((termVectorsReaderOrig!=null) ? termVectorsReaderOrig.ramBytesUsed() : 0);
}
}

View File

@ -48,20 +48,11 @@ final class SegmentMerger {
private final FieldInfos.Builder fieldInfosBuilder;
// 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,
SegmentMerger(List<CodecReader> readers, SegmentInfo segmentInfo, InfoStream infoStream, Directory dir,
FieldInfos.FieldNumbers fieldNumbers, IOContext context) throws IOException {
if (context.context != IOContext.Context.MERGE) {
throw new IllegalArgumentException("IOContext.context should be MERGE; got: " + context.context);
}
// validate incoming 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);
directory = dir;
this.codec = segmentInfo.getCodec();

View File

@ -18,12 +18,7 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesProducer;
@ -34,12 +29,7 @@ import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
/**
* IndexReader implementation over a single segment.
@ -48,12 +38,8 @@ import org.apache.lucene.util.RamUsageEstimator;
* may share the same core data.
* @lucene.experimental
*/
public final class SegmentReader extends LeafReader implements Accountable {
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(SegmentReader.class)
+ RamUsageEstimator.shallowSizeOfInstance(SegmentDocValues.class);
public final class SegmentReader extends CodecReader {
private final SegmentCommitInfo si;
private final Bits liveDocs;
@ -65,20 +51,6 @@ public final class SegmentReader extends LeafReader implements Accountable {
final SegmentCoreReaders core;
final SegmentDocValues segDocValues;
final CloseableThreadLocal<Map<String,Object>> docValuesLocal = new CloseableThreadLocal<Map<String,Object>>() {
@Override
protected Map<String,Object> initialValue() {
return new HashMap<>();
}
};
final CloseableThreadLocal<Map<String,Bits>> docsWithFieldLocal = new CloseableThreadLocal<Map<String,Bits>>() {
@Override
protected Map<String,Bits> initialValue() {
return new HashMap<>();
}
};
final DocValuesProducer docValuesProducer;
final FieldInfos fieldInfos;
@ -197,7 +169,7 @@ public final class SegmentReader extends LeafReader implements Accountable {
core.decRef();
} finally {
try {
IOUtils.close(docValuesLocal, docsWithFieldLocal);
super.doClose();
} finally {
if (docValuesProducer instanceof SegmentDocValuesProducer) {
segDocValues.decRef(((SegmentDocValuesProducer)docValuesProducer).dvGens);
@ -213,18 +185,6 @@ public final class SegmentReader extends LeafReader implements Accountable {
ensureOpen();
return fieldInfos;
}
@Override
public void document(int docID, StoredFieldVisitor visitor) throws IOException {
checkBounds(docID);
getFieldsReader().visitDocument(docID, visitor);
}
@Override
public FieldsProducer fields() {
ensureOpen();
return core.fields;
}
@Override
public int numDocs() {
@ -238,50 +198,34 @@ public final class SegmentReader extends LeafReader implements Accountable {
return si.info.getDocCount();
}
/** Expert: retrieve thread-private {@link
* TermVectorsReader}
* @lucene.internal */
@Override
public TermVectorsReader getTermVectorsReader() {
ensureOpen();
return core.termVectorsLocal.get();
}
/** Expert: retrieve thread-private {@link
* StoredFieldsReader}
* @lucene.internal */
@Override
public StoredFieldsReader getFieldsReader() {
ensureOpen();
return core.fieldsReaderLocal.get();
}
/** Expert: retrieve underlying NormsProducer
* @lucene.internal */
@Override
public NormsProducer getNormsReader() {
ensureOpen();
return core.normsProducer;
}
/** Expert: retrieve underlying DocValuesProducer
* @lucene.internal */
@Override
public DocValuesProducer getDocValuesReader() {
ensureOpen();
return docValuesProducer;
}
@Override
public Fields getTermVectors(int docID) throws IOException {
TermVectorsReader termVectorsReader = getTermVectorsReader();
if (termVectorsReader == null) {
return null;
}
checkBounds(docID);
return termVectorsReader.get(docID);
}
private void checkBounds(int docID) {
if (docID < 0 || docID >= maxDoc()) {
throw new IndexOutOfBoundsException("docID must be >= 0 and < maxDoc=" + maxDoc() + " (got docID=" + docID + ")");
}
public FieldsProducer getPostingsReader() {
ensureOpen();
return core.fields;
}
@Override
@ -329,152 +273,6 @@ public final class SegmentReader extends LeafReader implements Accountable {
return this;
}
// returns the FieldInfo that corresponds to the given field and type, or
// null if the field does not exist, or not indexed as the requested
// DovDocValuesType.
private FieldInfo getDVField(String field, DocValuesType type) {
FieldInfo fi = fieldInfos.fieldInfo(field);
if (fi == null) {
// Field does not exist
return null;
}
if (fi.getDocValuesType() == DocValuesType.NONE) {
// Field was not indexed with doc values
return null;
}
if (fi.getDocValuesType() != type) {
// Field DocValues are different than requested type
return null;
}
return fi;
}
@Override
public NumericDocValues getNumericDocValues(String field) throws IOException {
ensureOpen();
Map<String,Object> dvFields = docValuesLocal.get();
Object previous = dvFields.get(field);
if (previous != null && previous instanceof NumericDocValues) {
return (NumericDocValues) previous;
} else {
FieldInfo fi = getDVField(field, DocValuesType.NUMERIC);
if (fi == null) {
return null;
}
NumericDocValues dv = docValuesProducer.getNumeric(fi);
dvFields.put(field, dv);
return dv;
}
}
@Override
public Bits getDocsWithField(String field) throws IOException {
ensureOpen();
Map<String,Bits> dvFields = docsWithFieldLocal.get();
Bits previous = dvFields.get(field);
if (previous != null) {
return previous;
} else {
FieldInfo fi = fieldInfos.fieldInfo(field);
if (fi == null) {
// Field does not exist
return null;
}
if (fi.getDocValuesType() == DocValuesType.NONE) {
// Field was not indexed with doc values
return null;
}
Bits dv = docValuesProducer.getDocsWithField(fi);
dvFields.put(field, dv);
return dv;
}
}
@Override
public BinaryDocValues getBinaryDocValues(String field) throws IOException {
ensureOpen();
FieldInfo fi = getDVField(field, DocValuesType.BINARY);
if (fi == null) {
return null;
}
Map<String,Object> dvFields = docValuesLocal.get();
BinaryDocValues dvs = (BinaryDocValues) dvFields.get(field);
if (dvs == null) {
dvs = docValuesProducer.getBinary(fi);
dvFields.put(field, dvs);
}
return dvs;
}
@Override
public SortedDocValues getSortedDocValues(String field) throws IOException {
ensureOpen();
Map<String,Object> dvFields = docValuesLocal.get();
Object previous = dvFields.get(field);
if (previous != null && previous instanceof SortedDocValues) {
return (SortedDocValues) previous;
} else {
FieldInfo fi = getDVField(field, DocValuesType.SORTED);
if (fi == null) {
return null;
}
SortedDocValues dv = docValuesProducer.getSorted(fi);
dvFields.put(field, dv);
return dv;
}
}
@Override
public SortedNumericDocValues getSortedNumericDocValues(String field) throws IOException {
ensureOpen();
Map<String,Object> dvFields = docValuesLocal.get();
Object previous = dvFields.get(field);
if (previous != null && previous instanceof SortedNumericDocValues) {
return (SortedNumericDocValues) previous;
} else {
FieldInfo fi = getDVField(field, DocValuesType.SORTED_NUMERIC);
if (fi == null) {
return null;
}
SortedNumericDocValues dv = docValuesProducer.getSortedNumeric(fi);
dvFields.put(field, dv);
return dv;
}
}
@Override
public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
ensureOpen();
Map<String,Object> dvFields = docValuesLocal.get();
Object previous = dvFields.get(field);
if (previous != null && previous instanceof SortedSetDocValues) {
return (SortedSetDocValues) previous;
} else {
FieldInfo fi = getDVField(field, DocValuesType.SORTED_SET);
if (fi == null) {
return null;
}
SortedSetDocValues dv = docValuesProducer.getSortedSet(fi);
dvFields.put(field, dv);
return dv;
}
}
@Override
public NumericDocValues getNormValues(String field) throws IOException {
ensureOpen();
return core.getNormValues(fieldInfos, field);
}
@Override
public void addCoreClosedListener(CoreClosedListener listener) {
ensureOpen();
@ -486,66 +284,4 @@ public final class SegmentReader extends LeafReader implements Accountable {
ensureOpen();
core.removeCoreClosedListener(listener);
}
@Override
public long ramBytesUsed() {
ensureOpen();
long ramBytesUsed = BASE_RAM_BYTES_USED;
if (docValuesProducer != null) {
ramBytesUsed += docValuesProducer.ramBytesUsed();
}
if (core != null) {
ramBytesUsed += core.ramBytesUsed();
}
return ramBytesUsed;
}
@Override
public Collection<Accountable> getChildResources() {
ensureOpen();
List<Accountable> resources = new ArrayList<>();
resources.add(Accountables.namedAccountable("postings", core.fields));
if (core.normsProducer != null) {
resources.add(Accountables.namedAccountable("norms", core.normsProducer));
}
if (docValuesProducer != null) {
resources.add(Accountables.namedAccountable("docvalues", docValuesProducer));
}
if (getFieldsReader() != null) {
resources.add(Accountables.namedAccountable("stored fields", getFieldsReader()));
}
if (getTermVectorsReader() != null) {
resources.add(Accountables.namedAccountable("term vectors", getTermVectorsReader()));
}
return Collections.unmodifiableList(resources);
}
@Override
public void checkIntegrity() throws IOException {
ensureOpen();
// stored fields
getFieldsReader().checkIntegrity();
// term vectors
TermVectorsReader termVectorsReader = getTermVectorsReader();
if (termVectorsReader != null) {
termVectorsReader.checkIntegrity();
}
// terms/postings
if (core.fields != null) {
core.fields.checkIntegrity();
}
// norms
if (core.normsProducer != null) {
core.normsProducer.checkIntegrity();
}
// docvalues
if (docValuesProducer != null) {
docValuesProducer.checkIntegrity();
}
}
}

View File

@ -0,0 +1,283 @@
package org.apache.lucene.index;
/*
* 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.
*/
import java.io.IOException;
import java.util.Iterator;
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.util.Bits;
/**
* Wraps arbitrary readers for merging. Note that this can cause slow
* and memory-intensive merges. Consider using {@link FilterCodecReader}
* instead.
*/
public final class SlowCodecReaderWrapper {
/** No instantiation */
private SlowCodecReaderWrapper() {}
/**
* Returns a {@code CodecReader} view of reader.
* <p>
* If {@code reader} is already a {@code CodecReader}, it is returned
* directly. Otherwise, a (slow) view is returned.
*/
public static CodecReader wrap(final LeafReader reader) throws IOException {
if (reader instanceof CodecReader) {
return (CodecReader)reader;
} else {
// simulate it slowly, over the leafReader api:
reader.checkIntegrity();
return new CodecReader() {
@Override
public TermVectorsReader getTermVectorsReader() {
reader.ensureOpen();
return readerToTermVectorsReader(reader);
}
@Override
public StoredFieldsReader getFieldsReader() {
reader.ensureOpen();
return readerToStoredFieldsReader(reader);
}
@Override
public NormsProducer getNormsReader() {
reader.ensureOpen();
return readerToNormsProducer(reader);
}
@Override
public DocValuesProducer getDocValuesReader() {
reader.ensureOpen();
return readerToDocValuesProducer(reader);
}
@Override
public FieldsProducer getPostingsReader() {
reader.ensureOpen();
try {
return readerToFieldsProducer(reader);
} catch (IOException bogus) {
throw new AssertionError(bogus);
}
}
@Override
public FieldInfos getFieldInfos() {
return reader.getFieldInfos();
}
@Override
public Bits getLiveDocs() {
return reader.getLiveDocs();
}
@Override
public int numDocs() {
return reader.numDocs();
}
@Override
public int maxDoc() {
return reader.maxDoc();
}
@Override
public void addCoreClosedListener(CoreClosedListener listener) {
reader.addCoreClosedListener(listener);
}
@Override
public void removeCoreClosedListener(CoreClosedListener listener) {
reader.removeCoreClosedListener(listener);
}
};
}
}
private static 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
}
@Override
public void close() {
}
@Override
public long ramBytesUsed() {
return 0;
}
};
}
private static 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
}
@Override
public void close() {
}
@Override
public long ramBytesUsed() {
return 0;
}
};
}
private static 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
}
@Override
public void close() {
}
@Override
public long ramBytesUsed() {
return 0;
}
};
}
private static 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
}
@Override
public void close() {
}
@Override
public long ramBytesUsed() {
return 0;
}
};
}
private static 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
}
@Override
public void close() {
}
@Override
public long ramBytesUsed() {
return 0;
}
};
}
}

View File

@ -165,9 +165,9 @@ public class TrackingIndexWriter {
return indexingGen.get();
}
/** Calls {@link IndexWriter#addIndexes(LeafReader...)}
/** Calls {@link IndexWriter#addIndexes(CodecReader...)}
* and returns the generation that reflects this change. */
public long addIndexes(LeafReader... readers) throws IOException {
public long addIndexes(CodecReader... readers) throws IOException {
writer.addIndexes(readers);
// Return gen as of when indexing finished:
return indexingGen.get();

View File

@ -1217,7 +1217,7 @@ public class TestAddIndexes extends LuceneTestCase {
public void testAddEmpty() throws Exception {
Directory d1 = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), d1);
w.addIndexes(new LeafReader[0]);
w.addIndexes(new CodecReader[0]);
w.close();
DirectoryReader dr = DirectoryReader.open(d1);
for (LeafReaderContext ctx : dr.leaves()) {
@ -1239,7 +1239,7 @@ public class TestAddIndexes extends LuceneTestCase {
w.close();
w = new RandomIndexWriter(random(), dest);
w.addIndexes(allDeletedReader);
w.addIndexes(SlowCodecReaderWrapper.wrap(allDeletedReader));
w.close();
DirectoryReader dr = DirectoryReader.open(src);
for (LeafReaderContext ctx : dr.leaves()) {

View File

@ -528,15 +528,20 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
}
@Override
public List<LeafReader> getMergeReaders() throws IOException {
public List<CodecReader> getMergeReaders() throws IOException {
if (parallelReaders == null) {
parallelReaders = new ArrayList<>();
for (LeafReader reader : super.getMergeReaders()) {
parallelReaders.add(getCurrentReader(reader, schemaGen));
for (CodecReader reader : super.getMergeReaders()) {
parallelReaders.add(getCurrentReader((SegmentReader)reader, schemaGen));
}
}
return parallelReaders;
// TODO: fix ParallelLeafReader, if this is a good use case
List<CodecReader> mergeReaders = new ArrayList<>();
for (LeafReader reader : parallelReaders) {
mergeReaders.add(SlowCodecReaderWrapper.wrap(reader));
}
return mergeReaders;
}
@Override

View File

@ -222,7 +222,7 @@ public class TestDoc extends LuceneTestCase {
TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(si1.info.dir);
final SegmentInfo si = new SegmentInfo(si1.info.dir, Version.LATEST, merged, -1, false, codec, null, StringHelper.randomId(), new HashMap<>());
SegmentMerger merger = new SegmentMerger(Arrays.<LeafReader>asList(r1, r2),
SegmentMerger merger = new SegmentMerger(Arrays.<CodecReader>asList(r1, r2),
si, InfoStream.getDefault(), trackingDir,
new FieldInfos.FieldNumbers(), context);

View File

@ -72,7 +72,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
Directory d3 = newDirectory();
w = new RandomIndexWriter(random(), d3);
w.addIndexes(SlowCompositeReaderWrapper.wrap(r1), SlowCompositeReaderWrapper.wrap(r2));
w.addIndexes(SlowCodecReaderWrapper.wrap(SlowCompositeReaderWrapper.wrap(r1)), SlowCodecReaderWrapper.wrap(SlowCompositeReaderWrapper.wrap(r2)));
r1.close();
d1.close();
r2.close();

View File

@ -139,7 +139,7 @@ public class TestFilterLeafReader extends LuceneTestCase {
writer = new IndexWriter(target, newIndexWriterConfig(new MockAnalyzer(random())));
try (LeafReader reader = new TestReader(DirectoryReader.open(directory))) {
writer.addIndexes(reader);
writer.addIndexes(SlowCodecReaderWrapper.wrap(reader));
}
writer.close();
IndexReader reader = DirectoryReader.open(target);

View File

@ -55,11 +55,11 @@ public class TestParallelReaderEmptyIndex extends LuceneTestCase {
SlowCompositeReaderWrapper.wrap(DirectoryReader.open(rd2)));
// When unpatched, Lucene crashes here with a NoSuchElementException (caused by ParallelTermEnum)
iwOut.addIndexes(apr);
iwOut.addIndexes(SlowCodecReaderWrapper.wrap(apr));
iwOut.forceMerge(1);
// 2nd try with a readerless parallel reader
iwOut.addIndexes(new ParallelLeafReader());
iwOut.addIndexes(SlowCodecReaderWrapper.wrap(new ParallelLeafReader()));
iwOut.forceMerge(1);
ParallelCompositeReader cpr = new ParallelCompositeReader(
@ -67,11 +67,11 @@ public class TestParallelReaderEmptyIndex extends LuceneTestCase {
DirectoryReader.open(rd2));
// When unpatched, Lucene crashes here with a NoSuchElementException (caused by ParallelTermEnum)
List<LeafReader> leaves = new ArrayList<>();
List<CodecReader> leaves = new ArrayList<>();
for (LeafReaderContext leaf : cpr.leaves()) {
leaves.add(leaf.reader());
leaves.add(SlowCodecReaderWrapper.wrap(leaf.reader()));
}
iwOut.addIndexes(leaves.toArray(new LeafReader[0]));
iwOut.addIndexes(leaves.toArray(new CodecReader[0]));
iwOut.forceMerge(1);
iwOut.close();
@ -142,7 +142,7 @@ public class TestParallelReaderEmptyIndex extends LuceneTestCase {
SlowCompositeReaderWrapper.wrap(reader2 = DirectoryReader.open(rd2)));
// When unpatched, Lucene crashes here with an ArrayIndexOutOfBoundsException (caused by TermVectorsWriter)
iwOut.addIndexes(pr);
iwOut.addIndexes(SlowCodecReaderWrapper.wrap(pr));
// ParallelReader closes any IndexReader you added to it:
pr.close();

View File

@ -84,7 +84,7 @@ public class TestSegmentMerger extends LuceneTestCase {
final Codec codec = Codec.getDefault();
final SegmentInfo si = new SegmentInfo(mergedDir, Version.LATEST, mergedSegment, -1, false, codec, null, StringHelper.randomId(), new HashMap<>());
SegmentMerger merger = new SegmentMerger(Arrays.<LeafReader>asList(reader1, reader2),
SegmentMerger merger = new SegmentMerger(Arrays.<CodecReader>asList(reader1, reader2),
si, InfoStream.getDefault(), mergedDir,
new FieldInfos.FieldNumbers(),
newIOContext(random(), new IOContext(new MergeInfo(-1, -1, false, -1))));

View File

@ -10,7 +10,9 @@ import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.CodecReader;
import org.apache.lucene.index.MultiReader;
import org.apache.lucene.index.SlowCodecReaderWrapper;
import org.apache.lucene.store.Directory;
/*
@ -52,9 +54,9 @@ public abstract class TaxonomyMergeUtils {
try {
List<LeafReaderContext> leaves = reader.leaves();
int numReaders = leaves.size();
LeafReader wrappedLeaves[] = new LeafReader[numReaders];
CodecReader wrappedLeaves[] = new CodecReader[numReaders];
for (int i = 0; i < numReaders; i++) {
wrappedLeaves[i] = new OrdinalMappingLeafReader(leaves.get(i).reader(), ordinalMap, srcConfig);
wrappedLeaves[i] = SlowCodecReaderWrapper.wrap(new OrdinalMappingLeafReader(leaves.get(i).reader(), ordinalMap, srcConfig));
}
destIndexWriter.addIndexes(wrappedLeaves);

View File

@ -38,7 +38,7 @@ class MergeReaderWrapper extends LeafReader {
MergeReaderWrapper(SegmentReader in) throws IOException {
this.in = in;
FieldsProducer fields = in.fields();
FieldsProducer fields = in.getPostingsReader();
if (fields != null) {
fields = fields.getMergeInstance();
}

View File

@ -33,7 +33,7 @@ import org.apache.lucene.util.Version;
/**
* This tool splits input index into multiple equal parts. The method employed
* here uses {@link IndexWriter#addIndexes(LeafReader[])} where the input data
* here uses {@link IndexWriter#addIndexes(CodecReader[])} where the input data
* comes from the input index with artificially applied deletes to the document
* id-s that fall outside the selected partition.
* <p>Note 1: Deletes are only applied to a buffered list of deleted docs and
@ -59,7 +59,7 @@ public class MultiPassIndexSplitter {
* assigned in a deterministic round-robin fashion to one of the output splits.
* @throws IOException If there is a low-level I/O error
*/
public void split(Version version, IndexReader in, Directory[] outputs, boolean seq) throws IOException {
public void split(IndexReader in, Directory[] outputs, boolean seq) throws IOException {
if (outputs == null || outputs.length < 2) {
throw new IOException("Invalid number of outputs.");
}
@ -102,7 +102,7 @@ public class MultiPassIndexSplitter {
System.err.println("Writing part " + (i + 1) + " ...");
// pass the subreaders directly, as our wrapper's numDocs/hasDeletetions are not up-to-date
final List<? extends FakeDeleteLeafIndexReader> sr = input.getSequentialSubReaders();
w.addIndexes(sr.toArray(new LeafReader[sr.size()])); // TODO: maybe take List<IR> here?
w.addIndexes(sr.toArray(new CodecReader[sr.size()])); // TODO: maybe take List<IR> here?
w.close();
}
System.err.println("Done.");
@ -170,7 +170,7 @@ public class MultiPassIndexSplitter {
} else {
input = new MultiReader(indexes.toArray(new IndexReader[indexes.size()]));
}
splitter.split(Version.LATEST, input, dirs, seq);
splitter.split(input, dirs, seq);
}
/**
@ -178,16 +178,16 @@ public class MultiPassIndexSplitter {
*/
private static final class FakeDeleteIndexReader extends BaseCompositeReader<FakeDeleteLeafIndexReader> {
public FakeDeleteIndexReader(IndexReader reader) {
public FakeDeleteIndexReader(IndexReader reader) throws IOException {
super(initSubReaders(reader));
}
private static FakeDeleteLeafIndexReader[] initSubReaders(IndexReader reader) {
private static FakeDeleteLeafIndexReader[] initSubReaders(IndexReader reader) throws IOException {
final List<LeafReaderContext> leaves = reader.leaves();
final FakeDeleteLeafIndexReader[] subs = new FakeDeleteLeafIndexReader[leaves.size()];
int i = 0;
for (final LeafReaderContext ctx : leaves) {
subs[i++] = new FakeDeleteLeafIndexReader(ctx.reader());
subs[i++] = new FakeDeleteLeafIndexReader(SlowCodecReaderWrapper.wrap(ctx.reader()));
}
return subs;
}
@ -210,10 +210,10 @@ public class MultiPassIndexSplitter {
// as we pass the subreaders directly to IW.addIndexes().
}
private static final class FakeDeleteLeafIndexReader extends FilterLeafReader {
private static final class FakeDeleteLeafIndexReader extends FilterCodecReader {
FixedBitSet liveDocs;
public FakeDeleteLeafIndexReader(LeafReader reader) {
public FakeDeleteLeafIndexReader(CodecReader reader) {
super(reader);
undeleteAll(); // initialize main bitset
}

View File

@ -98,12 +98,12 @@ public class PKIndexSplitter {
}
}
private void createIndex(IndexWriterConfig config, Directory target, IndexReader reader, Filter preserveFilter, boolean negateFilter) throws IOException {
private void createIndex(IndexWriterConfig config, Directory target, DirectoryReader reader, Filter preserveFilter, boolean negateFilter) throws IOException {
boolean success = false;
final IndexWriter w = new IndexWriter(target, config);
try {
final List<LeafReaderContext> leaves = reader.leaves();
final LeafReader[] subReaders = new LeafReader[leaves.size()];
final CodecReader[] subReaders = new CodecReader[leaves.size()];
int i = 0;
for (final LeafReaderContext ctx : leaves) {
subReaders[i++] = new DocumentFilteredLeafIndexReader(ctx, preserveFilter, negateFilter);
@ -119,12 +119,13 @@ public class PKIndexSplitter {
}
}
private static class DocumentFilteredLeafIndexReader extends FilterLeafReader {
private static class DocumentFilteredLeafIndexReader extends FilterCodecReader {
final Bits liveDocs;
final int numDocs;
public DocumentFilteredLeafIndexReader(LeafReaderContext context, Filter preserveFilter, boolean negateFilter) throws IOException {
super(context.reader());
// our cast is ok, since we open the Directory.
super((CodecReader) context.reader());
final int maxDoc = in.maxDoc();
final FixedBitSet bits = new FixedBitSet(maxDoc);
// ignore livedocs here, as we filter them later:

View File

@ -65,7 +65,7 @@ public final class SortingMergePolicy extends MergePolicy {
class SortingOneMerge extends OneMerge {
List<LeafReader> unsortedReaders;
List<CodecReader> unsortedReaders;
Sorter.DocMap docMap;
LeafReader sortedView;
final InfoStream infoStream;
@ -76,7 +76,7 @@ public final class SortingMergePolicy extends MergePolicy {
}
@Override
public List<LeafReader> getMergeReaders() throws IOException {
public List<CodecReader> getMergeReaders() throws IOException {
if (unsortedReaders == null) {
unsortedReaders = super.getMergeReaders();
if (infoStream.isEnabled("SMP")) {
@ -117,7 +117,7 @@ public final class SortingMergePolicy extends MergePolicy {
if (infoStream.isEnabled("SMP")) {
infoStream.message("SMP", "sorting readers by " + sort);
}
return Collections.singletonList(sortedView);
return Collections.singletonList(SlowCodecReaderWrapper.wrap(sortedView));
}
}
@ -128,7 +128,7 @@ public final class SortingMergePolicy extends MergePolicy {
super.setInfo(info);
}
private PackedLongValues getDeletes(List<LeafReader> readers) {
private PackedLongValues getDeletes(List<CodecReader> readers) {
PackedLongValues.Builder deletes = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
int deleteCount = 0;
for (LeafReader reader : readers) {

View File

@ -72,7 +72,7 @@ public class IndexSortingTest extends SorterTestBase {
Directory target = newDirectory();
IndexWriter writer = new IndexWriter(target, newIndexWriterConfig(null));
LeafReader reader = SortingLeafReader.wrap(unsortedReader, sorter);
writer.addIndexes(reader);
writer.addIndexes(SlowCodecReaderWrapper.wrap(reader));
writer.close();
// NOTE: also closes unsortedReader
reader.close();

View File

@ -66,7 +66,7 @@ public class TestMultiPassIndexSplitter extends LuceneTestCase {
newDirectory(),
newDirectory()
};
splitter.split(Version.LATEST, input, dirs, false);
splitter.split(input, dirs, false);
IndexReader ir;
ir = DirectoryReader.open(dirs[0]);
assertTrue(ir.numDocs() - NUM_DOCS / 3 <= 1); // rounding error
@ -111,7 +111,7 @@ public class TestMultiPassIndexSplitter extends LuceneTestCase {
newDirectory(),
newDirectory()
};
splitter.split(Version.LATEST, input, dirs, true);
splitter.split(input, dirs, true);
IndexReader ir;
ir = DirectoryReader.open(dirs[0]);
assertTrue(ir.numDocs() - NUM_DOCS / 3 <= 1);

View File

@ -138,7 +138,7 @@ public class MockRandomMergePolicy extends MergePolicy {
static class MockRandomOneMerge extends OneMerge {
final Random r;
ArrayList<LeafReader> readers;
ArrayList<CodecReader> readers;
MockRandomOneMerge(List<SegmentCommitInfo> segments, long seed) {
super(segments);
@ -146,21 +146,23 @@ public class MockRandomMergePolicy extends MergePolicy {
}
@Override
public List<LeafReader> getMergeReaders() throws IOException {
public List<CodecReader> getMergeReaders() throws IOException {
if (readers == null) {
readers = new ArrayList<LeafReader>(super.getMergeReaders());
readers = new ArrayList<CodecReader>(super.getMergeReaders());
for (int i = 0; i < readers.size(); i++) {
// wrap it (e.g. prevent bulk merge etc)
// TODO: cut this over to FilterCodecReader api, we can explicitly
// enable/disable bulk merge for portions of the index we want.
int thingToDo = r.nextInt(7);
if (thingToDo == 0) {
// simple no-op FilterReader
readers.set(i, new FilterLeafReader(readers.get(i)));
readers.set(i, SlowCodecReaderWrapper.wrap(new FilterLeafReader(readers.get(i))));
} else if (thingToDo == 1) {
// renumber fields
// NOTE: currently this only "blocks" bulk merges just by
// being a FilterReader. But it might find bugs elsewhere,
// and maybe the situation can be improved in the future.
readers.set(i, new MismatchedLeafReader(readers.get(i), r));
readers.set(i, SlowCodecReaderWrapper.wrap(new MismatchedLeafReader(readers.get(i), r)));
}
// otherwise, reader is unchanged
}

View File

@ -222,7 +222,7 @@ public class RandomIndexWriter implements Closeable {
w.addIndexes(dirs);
}
public void addIndexes(LeafReader... readers) throws IOException {
public void addIndexes(CodecReader... readers) throws IOException {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
w.addIndexes(readers);
}

View File

@ -82,8 +82,10 @@ import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.LogMergePolicy;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.MergeScheduler;
import org.apache.lucene.index.CodecReader;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.index.SlowCodecReaderWrapper;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.TieredMergePolicy;
@ -876,13 +878,13 @@ public final class TestUtil {
}
public static void addIndexesSlowly(IndexWriter writer, DirectoryReader... readers) throws IOException {
List<LeafReader> leaves = new ArrayList<>();
List<CodecReader> leaves = new ArrayList<>();
for (DirectoryReader reader : readers) {
for (LeafReaderContext context : reader.leaves()) {
leaves.add(context.reader());
leaves.add(SlowCodecReaderWrapper.wrap(context.reader()));
}
}
writer.addIndexes(leaves.toArray(new LeafReader[leaves.size()]));
writer.addIndexes(leaves.toArray(new CodecReader[leaves.size()]));
}
/** just tries to configure things to keep the open file

View File

@ -21,11 +21,11 @@
package org.apache.solr.update;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.CodecReader;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SlowCodecReaderWrapper;
import org.apache.lucene.index.Term;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.BooleanClause;
@ -469,15 +469,15 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
List<DirectoryReader> readers = cmd.readers;
if (readers != null && readers.size() > 0) {
List<LeafReader> leaves = new ArrayList<>();
List<CodecReader> mergeReaders = new ArrayList<>();
for (DirectoryReader reader : readers) {
for (LeafReaderContext leaf : reader.leaves()) {
leaves.add(leaf.reader());
mergeReaders.add(SlowCodecReaderWrapper.wrap(leaf.reader()));
}
}
RefCounted<IndexWriter> iw = solrCoreState.getIndexWriter(core);
try {
iw.get().addIndexes(leaves.toArray(new LeafReader[leaves.size()]));
iw.get().addIndexes(mergeReaders.toArray(new CodecReader[mergeReaders.size()]));
} finally {
iw.decref();
}

View File

@ -21,13 +21,14 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.index.FilterLeafReader;
import org.apache.lucene.index.CodecReader;
import org.apache.lucene.index.FilterCodecReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.SlowCodecReaderWrapper;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSetIterator;
@ -129,8 +130,8 @@ public class SolrIndexSplitter {
// This removes deletions but optimize might still be needed because sub-shards will have the same number of segments as the parent shard.
for (int segmentNumber = 0; segmentNumber<leaves.size(); segmentNumber++) {
log.info("SolrIndexSplitter: partition #" + partitionNumber + " partitionCount=" + numPieces + (ranges != null ? " range=" + ranges.get(partitionNumber) : "") + " segment #"+segmentNumber + " segmentCount=" + leaves.size());
LeafReader subReader = new LiveDocsReader( leaves.get(segmentNumber), segmentDocSets.get(segmentNumber)[partitionNumber] );
iw.addIndexes(subReader);
CodecReader subReader = SlowCodecReaderWrapper.wrap(leaves.get(segmentNumber).reader());
iw.addIndexes(new LiveDocsReader(subReader, segmentDocSets.get(segmentNumber)[partitionNumber]));
}
success = true;
} finally {
@ -232,12 +233,12 @@ public class SolrIndexSplitter {
// change livedocs on the reader to delete those docs we don't want
static class LiveDocsReader extends FilterLeafReader {
static class LiveDocsReader extends FilterCodecReader {
final FixedBitSet liveDocs;
final int numDocs;
public LiveDocsReader(LeafReaderContext context, FixedBitSet liveDocs) throws IOException {
super(context.reader());
public LiveDocsReader(CodecReader in, FixedBitSet liveDocs) throws IOException {
super(in);
this.liveDocs = liveDocs;
this.numDocs = liveDocs.cardinality();
}