LUCENE-3866: CompositeReader.getSequentialSubReaders() now returns unmodifiable List<? extends IndexReader>. ReaderUtil.Gather was removed, as IndexReaderContext.leaves() is now the preferred way to access sub-readers

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1351590 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Uwe Schindler 2012-06-19 07:24:29 +00:00
parent f6bd05ca2f
commit ae212ff6f6
73 changed files with 608 additions and 680 deletions

View File

@ -278,6 +278,11 @@ Changes in backwards compatibility policy
* LUCENE-4055: You can't put foreign files into the index dir anymore.
* LUCENE-3866: CompositeReader.getSequentialSubReaders() now returns
unmodifiable List<? extends IndexReader>. ReaderUtil.Gather was
removed, as IndexReaderContext.leaves() is now the preferred way
to access sub-readers. (Uwe Schindler)
Changes in Runtime Behavior
* LUCENE-2846: omitNorms now behaves like omitTermFrequencyAndPositions, if you
@ -536,6 +541,10 @@ API Changes
which can be used to change the IndexWriter's live settings. IndexWriterConfig
is used only for initializing the IndexWriter. (Shai Erera)
* LUCENE-3866: IndexReaderContext.leaves() is now the preferred way to access
atomic sub-readers of any kind of IndexReader (for AtomicReaders it returns
itsself as only leaf with docBase=0). (Uwe Schindler)
New features
* LUCENE-2604: Added RegexpQuery support to QueryParser. Regular expressions

View File

@ -1,5 +1,8 @@
package org.apache.lucene.index;
import java.util.Collections;
import java.util.List;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -28,7 +31,7 @@ public final class AtomicReaderContext extends IndexReaderContext {
public final int docBase;
private final AtomicReader reader;
private final AtomicReaderContext[] leaves;
private final List<AtomicReaderContext> leaves;
/**
* Creates a new {@link AtomicReaderContext}
@ -39,7 +42,7 @@ public final class AtomicReaderContext extends IndexReaderContext {
this.ord = leafOrd;
this.docBase = leafDocBase;
this.reader = reader;
this.leaves = isTopLevel ? new AtomicReaderContext[] { this } : null;
this.leaves = isTopLevel ? Collections.singletonList(this) : null;
}
AtomicReaderContext(AtomicReader atomicReader) {
@ -47,12 +50,15 @@ public final class AtomicReaderContext extends IndexReaderContext {
}
@Override
public AtomicReaderContext[] leaves() {
public List<AtomicReaderContext> leaves() {
if (!isTopLevel)
throw new UnsupportedOperationException("This is not a top-level context.");
assert leaves != null;
return leaves;
}
@Override
public IndexReaderContext[] children() {
public List<IndexReaderContext> children() {
return null;
}

View File

@ -18,6 +18,9 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.ReaderUtil;
@ -47,12 +50,16 @@ import org.apache.lucene.util.ReaderUtil;
* @lucene.internal
*/
public abstract class BaseCompositeReader<R extends IndexReader> extends CompositeReader {
protected final R[] subReaders;
protected final int[] starts; // 1st docno for each reader
private final R[] subReaders;
private final int[] starts; // 1st docno for each reader
private final int maxDoc;
private final int numDocs;
private final boolean hasDeletions;
/** List view solely for {@link #getSequentialSubReaders()},
* for effectiveness the array is used internally. */
private final List<R> subReadersList;
/**
* Constructs a {@code BaseCompositeReader} on the given subReaders.
* @param subReaders the wrapped sub-readers. This array is returned by
@ -63,6 +70,7 @@ public abstract class BaseCompositeReader<R extends IndexReader> extends Composi
*/
protected BaseCompositeReader(R[] subReaders) throws IOException {
this.subReaders = subReaders;
this.subReadersList = Collections.unmodifiableList(Arrays.asList(subReaders));
starts = new int[subReaders.length + 1]; // build starts array
int maxDoc = 0, numDocs = 0;
boolean hasDeletions = false;
@ -135,8 +143,16 @@ public abstract class BaseCompositeReader<R extends IndexReader> extends Composi
return ReaderUtil.subIndex(docID, this.starts);
}
/** Helper method for subclasses to get the docBase of the given sub-reader index. */
protected final int readerBase(int readerIndex) {
if (readerIndex < 0 || readerIndex >= subReaders.length) {
throw new IllegalArgumentException("readerIndex must be >= 0 and < getSequentialSubReaders().size()");
}
return this.starts[readerIndex];
}
@Override
public final R[] getSequentialSubReaders() {
return subReaders;
public final List<? extends R> getSequentialSubReaders() {
return subReadersList;
}
}

View File

@ -17,6 +17,8 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import java.util.List;
import org.apache.lucene.search.SearcherManager; // javadocs
import org.apache.lucene.store.*;
@ -63,12 +65,12 @@ public abstract class CompositeReader extends IndexReader {
final StringBuilder buffer = new StringBuilder();
buffer.append(getClass().getSimpleName());
buffer.append('(');
final IndexReader[] subReaders = getSequentialSubReaders();
final List<? extends IndexReader> subReaders = getSequentialSubReaders();
assert subReaders != null;
if (subReaders.length > 0) {
buffer.append(subReaders[0]);
for (int i = 1; i < subReaders.length; ++i) {
buffer.append(" ").append(subReaders[i]);
if (!subReaders.isEmpty()) {
buffer.append(subReaders.get(0));
for (int i = 1, c = subReaders.size(); i < c; ++i) {
buffer.append(" ").append(subReaders.get(i));
}
}
buffer.append(')');
@ -81,11 +83,8 @@ public abstract class CompositeReader extends IndexReader {
* If this method returns an empty array, that means this
* reader is a null reader (for example a MultiReader
* that has no sub readers).
* <p><b>Warning:</b> Don't modify the returned array!
* Doing so will corrupt the internal structure of this
* {@code CompositeReader}.
*/
public abstract IndexReader[] getSequentialSubReaders();
public abstract List<? extends IndexReader> getSequentialSubReaders();
@Override
public final CompositeReaderContext getTopReaderContext() {

View File

@ -17,16 +17,18 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.util.ReaderUtil;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
/**
* {@link IndexReaderContext} for {@link CompositeReader} instance.
* @lucene.experimental
*/
public final class CompositeReaderContext extends IndexReaderContext {
private final IndexReaderContext[] children;
private final AtomicReaderContext[] leaves;
private final List<IndexReaderContext> children;
private final List<AtomicReaderContext> leaves;
private final CompositeReader reader;
static CompositeReaderContext create(CompositeReader reader) {
@ -38,34 +40,37 @@ public final class CompositeReaderContext extends IndexReaderContext {
* not top-level readers in the current context
*/
CompositeReaderContext(CompositeReaderContext parent, CompositeReader reader,
int ordInParent, int docbaseInParent, IndexReaderContext[] children) {
int ordInParent, int docbaseInParent, List<IndexReaderContext> children) {
this(parent, reader, ordInParent, docbaseInParent, children, null);
}
/**
* Creates a {@link CompositeReaderContext} for top-level readers with parent set to <code>null</code>
*/
CompositeReaderContext(CompositeReader reader, IndexReaderContext[] children, AtomicReaderContext[] leaves) {
CompositeReaderContext(CompositeReader reader, List<IndexReaderContext> children, List<AtomicReaderContext> leaves) {
this(null, reader, 0, 0, children, leaves);
}
private CompositeReaderContext(CompositeReaderContext parent, CompositeReader reader,
int ordInParent, int docbaseInParent, IndexReaderContext[] children,
AtomicReaderContext[] leaves) {
int ordInParent, int docbaseInParent, List<IndexReaderContext> children,
List<AtomicReaderContext> leaves) {
super(parent, ordInParent, docbaseInParent);
this.children = children;
this.leaves = leaves;
this.children = Collections.unmodifiableList(children);
this.leaves = leaves == null ? null : Collections.unmodifiableList(leaves);
this.reader = reader;
}
@Override
public AtomicReaderContext[] leaves() {
public List<AtomicReaderContext> leaves() {
if (!isTopLevel)
throw new UnsupportedOperationException("This is not a top-level context.");
assert leaves != null;
return leaves;
}
@Override
public IndexReaderContext[] children() {
public List<IndexReaderContext> children() {
return children;
}
@ -76,13 +81,11 @@ public final class CompositeReaderContext extends IndexReaderContext {
private static final class Builder {
private final CompositeReader reader;
private final AtomicReaderContext[] leaves;
private int leafOrd = 0;
private final List<AtomicReaderContext> leaves = new ArrayList<AtomicReaderContext>();
private int leafDocBase = 0;
public Builder(CompositeReader reader) {
this.reader = reader;
leaves = new AtomicReaderContext[numLeaves(reader)];
}
public CompositeReaderContext build() {
@ -92,14 +95,14 @@ public final class CompositeReaderContext extends IndexReaderContext {
private IndexReaderContext build(CompositeReaderContext parent, IndexReader reader, int ord, int docBase) {
if (reader instanceof AtomicReader) {
final AtomicReader ar = (AtomicReader) reader;
final AtomicReaderContext atomic = new AtomicReaderContext(parent, ar, ord, docBase, leafOrd, leafDocBase);
leaves[leafOrd++] = atomic;
final AtomicReaderContext atomic = new AtomicReaderContext(parent, ar, ord, docBase, leaves.size(), leafDocBase);
leaves.add(atomic);
leafDocBase += reader.maxDoc();
return atomic;
} else {
final CompositeReader cr = (CompositeReader) reader;
final IndexReader[] sequentialSubReaders = cr.getSequentialSubReaders();
final IndexReaderContext[] children = new IndexReaderContext[sequentialSubReaders.length];
final List<? extends IndexReader> sequentialSubReaders = cr.getSequentialSubReaders();
final List<IndexReaderContext> children = Arrays.asList(new IndexReaderContext[sequentialSubReaders.size()]);
final CompositeReaderContext newParent;
if (parent == null) {
newParent = new CompositeReaderContext(cr, children, leaves);
@ -107,31 +110,15 @@ public final class CompositeReaderContext extends IndexReaderContext {
newParent = new CompositeReaderContext(parent, cr, ord, docBase, children);
}
int newDocBase = 0;
for (int i = 0; i < sequentialSubReaders.length; i++) {
children[i] = build(newParent, sequentialSubReaders[i], i, newDocBase);
newDocBase += sequentialSubReaders[i].maxDoc();
for (int i = 0, c = sequentialSubReaders.size(); i < c; i++) {
final IndexReader r = sequentialSubReaders.get(i);
children.set(i, build(newParent, r, i, newDocBase));
newDocBase += r.maxDoc();
}
assert newDocBase == cr.maxDoc();
return newParent;
}
}
private int numLeaves(IndexReader reader) {
final int[] numLeaves = new int[1];
try {
new ReaderUtil.Gather(reader) {
@Override
protected void add(int base, AtomicReader r) {
numLeaves[0]++;
}
}.run();
} catch (IOException ioe) {
// won't happen
throw new RuntimeException(ioe);
}
return numLeaves[0];
}
}
}

View File

@ -22,7 +22,6 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import org.apache.lucene.search.SearcherManager; // javadocs
import org.apache.lucene.store.Directory;

View File

@ -1,5 +1,7 @@
package org.apache.lucene.index;
import java.util.List;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -45,23 +47,19 @@ public abstract class IndexReaderContext {
public abstract IndexReader reader();
/**
* Returns the context's leaves if this context is a top-level context
* otherwise <code>null</code>. For convenience, if this is an
* {@link AtomicReaderContext} this returns itsself as the only leaf.
* Returns the context's leaves if this context is a top-level context.
* For convenience, if this is an {@link AtomicReaderContext} this
* returns itself as the only leaf.
* <p>Note: this is convenience method since leaves can always be obtained by
* walking the context tree.
* <p><b>Warning:</b> Don't modify the returned array!
* Doing so will corrupt the internal structure of this
* {@code IndexReaderContext}.
* walking the context tree using {@link #children()}.
* @throws UnsupportedOperationExceception if this is not a top-level context.
* @see #children()
*/
public abstract AtomicReaderContext[] leaves();
public abstract List<AtomicReaderContext> leaves();
/**
* Returns the context's children iff this context is a composite context
* otherwise <code>null</code>.
* <p><b>Warning:</b> Don't modify the returned array!
* Doing so will corrupt the internal structure of this
* {@code IndexReaderContext}.
*/
public abstract IndexReaderContext[] children();
public abstract List<IndexReaderContext> children();
}

View File

@ -29,13 +29,18 @@ import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.ReaderUtil;
import org.apache.lucene.util.ReaderUtil.Gather;
import org.apache.lucene.util.packed.PackedInts.Reader;
/**
* A wrapper for CompositeIndexReader providing access to per segment
* {@link DocValues}
*
* <p><b>NOTE</b>: for multi readers, you'll get better
* performance by gathering the sub readers using
* {@link IndexReader#getTopReaderContext()} to get the
* atomic leaves and then operate per-AtomicReader,
* instead of using this class.
*
* @lucene.experimental
* @lucene.internal
*/
@ -69,6 +74,8 @@ public class MultiDocValues extends DocValues {
}
private static class DocValuesPuller {
public DocValuesPuller() {}
public DocValues pull(AtomicReader reader, String field) throws IOException {
return reader.docValues(field);
}
@ -94,8 +101,9 @@ public class MultiDocValues extends DocValues {
* their values on the fly.
*
* <p>
* <b>NOTE</b>: this is a slow way to access DocValues. It's better to get the
* sub-readers (using {@link Gather}) and iterate through them yourself.
* <b>NOTE</b>: this is a slow way to access DocValues.
* It's better to get the sub-readers and iterate through them
* yourself.
*/
public static DocValues getDocValues(IndexReader r, final String field) throws IOException {
return getDocValues(r, field, DEFAULT_PULLER);
@ -106,80 +114,74 @@ public class MultiDocValues extends DocValues {
* their values on the fly.
*
* <p>
* <b>NOTE</b>: this is a slow way to access DocValues. It's better to get the
* sub-readers (using {@link Gather}) and iterate through them yourself.
* <b>NOTE</b>: this is a slow way to access DocValues.
* It's better to get the sub-readers and iterate through them
* yourself.
*/
public static DocValues getNormDocValues(IndexReader r, final String field) throws IOException {
return getDocValues(r, field, NORMS_PULLER);
}
private static DocValues getDocValues(IndexReader r, final String field, final DocValuesPuller puller) throws IOException {
if (r instanceof AtomicReader) {
private static DocValues getDocValues(IndexReader reader, final String field, final DocValuesPuller puller) throws IOException {
if (reader instanceof AtomicReader) {
// already an atomic reader
return puller.pull((AtomicReader) r, field);
return puller.pull((AtomicReader) reader, field);
}
assert r instanceof CompositeReader;
final IndexReader[] subs = ((CompositeReader) r).getSequentialSubReaders();
if (subs.length == 0) {
// no fields
return null;
} else if (subs.length == 1) {
return getDocValues(subs[0], field, puller);
} else {
final List<DocValuesSlice> slices = new ArrayList<DocValuesSlice>();
final TypePromoter promotedType[] = new TypePromoter[1];
promotedType[0] = TypePromoter.getIdentityPromoter();
// gather all docvalues fields, accumulating a promoted type across
// potentially incompatible types
new ReaderUtil.Gather(r) {
boolean stop = false;
@Override
protected void add(int base, AtomicReader r) throws IOException {
if (stop) {
return;
}
assert reader instanceof CompositeReader;
final List<AtomicReaderContext> leaves = reader.getTopReaderContext().leaves();
switch (leaves.size()) {
case 0:
// no fields
return null;
case 1:
// already an atomic reader / reader with one leave
return getDocValues(leaves.get(0).reader(), field, puller);
default:
final List<DocValuesSlice> slices = new ArrayList<DocValuesSlice>();
TypePromoter promotedType = TypePromoter.getIdentityPromoter();
// gather all docvalues fields, accumulating a promoted type across
// potentially incompatible types
for (final AtomicReaderContext ctx : leaves) {
final AtomicReader r = ctx.reader();
final DocValues d = puller.pull(r, field);
if (d != null) {
TypePromoter incoming = TypePromoter.create(d.getType(), d.getValueSize());
promotedType[0] = promotedType[0].promote(incoming);
promotedType = promotedType.promote(incoming);
} else if (puller.stopLoadingOnNull(r, field)){
promotedType[0] = TypePromoter.getIdentityPromoter(); // set to identity to return null
stop = true;
return null;
}
slices.add(new DocValuesSlice(d, base, r.maxDoc()));
slices.add(new DocValuesSlice(d, ctx.docBase, r.maxDoc()));
}
}.run();
// return null if no docvalues encountered anywhere
if (promotedType[0] == TypePromoter.getIdentityPromoter()) {
return null;
}
// populate starts and fill gaps with empty docvalues
int starts[] = new int[slices.size()];
for (int i = 0; i < slices.size(); i++) {
DocValuesSlice slice = slices.get(i);
starts[i] = slice.start;
if (slice.docValues == null) {
Type promoted = promotedType[0].type();
switch(promoted) {
case BYTES_FIXED_DEREF:
case BYTES_FIXED_STRAIGHT:
case BYTES_FIXED_SORTED:
assert promotedType[0].getValueSize() >= 0;
slice.docValues = new EmptyFixedDocValues(slice.length, promoted, promotedType[0].getValueSize());
break;
default:
slice.docValues = new EmptyDocValues(slice.length, promoted);
// return null if no docvalues encountered anywhere
if (promotedType == TypePromoter.getIdentityPromoter()) {
return null;
}
// populate starts and fill gaps with empty docvalues
int starts[] = new int[slices.size()];
for (int i = 0; i < slices.size(); i++) {
DocValuesSlice slice = slices.get(i);
starts[i] = slice.start;
if (slice.docValues == null) {
Type promoted = promotedType.type();
switch(promoted) {
case BYTES_FIXED_DEREF:
case BYTES_FIXED_STRAIGHT:
case BYTES_FIXED_SORTED:
assert promotedType.getValueSize() >= 0;
slice.docValues = new EmptyFixedDocValues(slice.length, promoted, promotedType.getValueSize());
break;
default:
slice.docValues = new EmptyDocValues(slice.length, promoted);
}
}
}
}
return new MultiDocValues(slices.toArray(new DocValuesSlice[slices.size()]), starts, promotedType[0]);
return new MultiDocValues(slices.toArray(new DocValuesSlice[slices.size()]), starts, promotedType);
}
}

View File

@ -17,8 +17,8 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import org.apache.lucene.util.ReaderUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.ReaderSlice;
import java.io.IOException;
@ -148,7 +148,7 @@ public final class MultiDocsAndPositionsEnum extends DocsAndPositionsEnum {
// TODO: implement bulk read more efficiently than super
public final static class EnumWithSlice {
public DocsAndPositionsEnum docsAndPositionsEnum;
public ReaderUtil.Slice slice;
public ReaderSlice slice;
}
}

View File

@ -17,7 +17,8 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import org.apache.lucene.util.ReaderUtil;
import org.apache.lucene.util.ReaderSlice;
import java.io.IOException;
import java.util.Arrays;
@ -123,7 +124,7 @@ public final class MultiDocsEnum extends DocsEnum {
// TODO: implement bulk read more efficiently than super
public final static class EnumWithSlice {
public DocsEnum docsEnum;
public ReaderUtil.Slice slice;
public ReaderSlice slice;
@Override
public String toString() {

View File

@ -28,8 +28,7 @@ import java.util.concurrent.ConcurrentHashMap;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.MultiBits;
import org.apache.lucene.util.ReaderUtil.Gather; // for javadocs
import org.apache.lucene.util.ReaderUtil;
import org.apache.lucene.util.ReaderSlice;
/**
* Exposes flex API, merged from flex API of sub-segments.
@ -38,9 +37,10 @@ import org.apache.lucene.util.ReaderUtil;
* sub-readers (eg {@link DirectoryReader} or {@link
* MultiReader}).
*
* <p><b>NOTE</b>: for multi readers, you'll get better
* performance by gathering the sub readers using {@link
* ReaderUtil#gatherSubReaders} and then operate per-reader,
* <p><b>NOTE</b>: for composite readers, you'll get better
* performance by gathering the sub readers using
* {@link IndexReader#getTopReaderContext()} to get the
* atomic leaves and then operate per-AtomicReader,
* instead of using this class.
*
* @lucene.experimental
@ -48,7 +48,7 @@ import org.apache.lucene.util.ReaderUtil;
public final class MultiFields extends Fields {
private final Fields[] subs;
private final ReaderUtil.Slice[] subSlices;
private final ReaderSlice[] subSlices;
private final Map<String,Terms> terms = new ConcurrentHashMap<String,Terms>();
/** Returns a single {@link Fields} instance for this
@ -57,72 +57,57 @@ public final class MultiFields extends Fields {
* has no postings.
*
* <p><b>NOTE</b>: this is a slow way to access postings.
* It's better to get the sub-readers (using {@link
* Gather}) and iterate through them
* It's better to get the sub-readers and iterate through them
* yourself. */
public static Fields getFields(IndexReader r) throws IOException {
if (r instanceof AtomicReader) {
// already an atomic reader
return ((AtomicReader) r).fields();
}
assert r instanceof CompositeReader;
final IndexReader[] subs = ((CompositeReader) r).getSequentialSubReaders();
if (subs.length == 0) {
// no fields
return null;
} else {
final List<Fields> fields = new ArrayList<Fields>();
final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
new ReaderUtil.Gather(r) {
@Override
protected void add(int base, AtomicReader r) throws IOException {
public static Fields getFields(IndexReader reader) throws IOException {
final List<AtomicReaderContext> leaves = reader.getTopReaderContext().leaves();
switch (leaves.size()) {
case 0:
// no fields
return null;
case 1:
// already an atomic reader / reader with one leave
return leaves.get(0).reader().fields();
default:
final List<Fields> fields = new ArrayList<Fields>();
final List<ReaderSlice> slices = new ArrayList<ReaderSlice>();
for (final AtomicReaderContext ctx : leaves) {
final AtomicReader r = ctx.reader();
final Fields f = r.fields();
if (f != null) {
fields.add(f);
slices.add(new ReaderUtil.Slice(base, r.maxDoc(), fields.size()-1));
slices.add(new ReaderSlice(ctx.docBase, r.maxDoc(), fields.size()-1));
}
}
}.run();
if (fields.isEmpty()) {
return null;
} else if (fields.size() == 1) {
return fields.get(0);
} else {
return new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY));
}
if (fields.isEmpty()) {
return null;
} else if (fields.size() == 1) {
return fields.get(0);
} else {
return new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
slices.toArray(ReaderSlice.EMPTY_ARRAY));
}
}
}
public static Bits getLiveDocs(IndexReader r) {
if (r.hasDeletions()) {
final List<Bits> liveDocs = new ArrayList<Bits>();
final List<Integer> starts = new ArrayList<Integer>();
try {
final int maxDoc = new ReaderUtil.Gather(r) {
@Override
protected void add(int base, AtomicReader r) throws IOException {
// record all liveDocs, even if they are null
liveDocs.add(r.getLiveDocs());
starts.add(base);
}
}.run();
starts.add(maxDoc);
} catch (IOException ioe) {
// should not happen
throw new RuntimeException(ioe);
public static Bits getLiveDocs(IndexReader reader) {
if (reader.hasDeletions()) {
final List<AtomicReaderContext> leaves = reader.getTopReaderContext().leaves();
final int size = leaves.size();
assert size > 0 : "A reader with deletions must have at least one leave";
if (size == 1) {
return leaves.get(0).reader().getLiveDocs();
}
assert liveDocs.size() > 0;
if (liveDocs.size() == 1) {
// Only one actual sub reader -- optimize this case
return liveDocs.get(0);
} else {
return new MultiBits(liveDocs, starts, true);
final Bits[] liveDocs = new Bits[size];
final int[] starts = new int[size + 1];
for (int i = 0; i < size; i++) {
// record all liveDocs, even if they are null
final AtomicReaderContext ctx = leaves.get(i);
liveDocs[i] = ctx.reader().getLiveDocs();
starts[i] = ctx.docBase;
}
starts[size] = reader.maxDoc();
return new MultiBits(liveDocs, starts, true);
} else {
return null;
}
@ -170,7 +155,7 @@ public final class MultiFields extends Fields {
return null;
}
public MultiFields(Fields[] subs, ReaderUtil.Slice[] subSlices) {
public MultiFields(Fields[] subs, ReaderSlice[] subSlices) {
this.subs = subs;
this.subSlices = subSlices;
}
@ -179,7 +164,7 @@ public final class MultiFields extends Fields {
public FieldsEnum iterator() throws IOException {
final List<FieldsEnum> fieldsEnums = new ArrayList<FieldsEnum>();
final List<ReaderUtil.Slice> fieldsSlices = new ArrayList<ReaderUtil.Slice>();
final List<ReaderSlice> fieldsSlices = new ArrayList<ReaderSlice>();
for(int i=0;i<subs.length;i++) {
fieldsEnums.add(subs[i].iterator());
fieldsSlices.add(subSlices[i]);
@ -189,13 +174,12 @@ public final class MultiFields extends Fields {
} else {
return new MultiFieldsEnum(this,
fieldsEnums.toArray(FieldsEnum.EMPTY_ARRAY),
fieldsSlices.toArray(ReaderUtil.Slice.EMPTY_ARRAY));
fieldsSlices.toArray(ReaderSlice.EMPTY_ARRAY));
}
}
@Override
public Terms terms(String field) throws IOException {
Terms result = terms.get(field);
if (result != null)
return result;
@ -204,7 +188,7 @@ public final class MultiFields extends Fields {
// Lazy init: first time this field is requested, we
// create & add to terms:
final List<Terms> subs2 = new ArrayList<Terms>();
final List<ReaderUtil.Slice> slices2 = new ArrayList<ReaderUtil.Slice>();
final List<ReaderSlice> slices2 = new ArrayList<ReaderSlice>();
// Gather all sub-readers that share this field
for(int i=0;i<subs.length;i++) {
@ -220,7 +204,7 @@ public final class MultiFields extends Fields {
// is unbounded.
} else {
result = new MultiTerms(subs2.toArray(Terms.EMPTY_ARRAY),
slices2.toArray(ReaderUtil.Slice.EMPTY_ARRAY));
slices2.toArray(ReaderSlice.EMPTY_ARRAY));
terms.put(field, result);
}
@ -252,18 +236,16 @@ public final class MultiFields extends Fields {
* will be unavailable.
*/
public static FieldInfos getMergedFieldInfos(IndexReader reader) {
final List<AtomicReader> subReaders = new ArrayList<AtomicReader>();
ReaderUtil.gatherSubReaders(subReaders, reader);
final FieldInfos.Builder builder = new FieldInfos.Builder();
for(AtomicReader subReader : subReaders) {
builder.add(subReader.getFieldInfos());
for(final AtomicReaderContext ctx : reader.getTopReaderContext().leaves()) {
builder.add(ctx.reader().getFieldInfos());
}
return builder.finish();
}
public static Collection<String> getIndexedFields(IndexReader reader) {
final Collection<String> fields = new HashSet<String>();
for(FieldInfo fieldInfo : getMergedFieldInfos(reader)) {
for(final FieldInfo fieldInfo : getMergedFieldInfos(reader)) {
if (fieldInfo.isIndexed()) {
fields.add(fieldInfo.name);
}

View File

@ -18,7 +18,7 @@ package org.apache.lucene.index;
*/
import org.apache.lucene.util.PriorityQueue;
import org.apache.lucene.util.ReaderUtil;
import org.apache.lucene.util.ReaderSlice;
import java.io.IOException;
import java.util.List;
@ -47,7 +47,7 @@ public final class MultiFieldsEnum extends FieldsEnum {
/** The subs array must be newly initialized FieldsEnum
* (ie, {@link FieldsEnum#next} has not been called. */
public MultiFieldsEnum(MultiFields fields, FieldsEnum[] subs, ReaderUtil.Slice[] subSlices) throws IOException {
public MultiFieldsEnum(MultiFields fields, FieldsEnum[] subs, ReaderSlice[] subSlices) throws IOException {
this.fields = fields;
queue = new FieldMergeQueue(subs.length);
top = new FieldsEnumWithSlice[subs.length];
@ -107,11 +107,11 @@ public final class MultiFieldsEnum extends FieldsEnum {
public final static class FieldsEnumWithSlice {
public static final FieldsEnumWithSlice[] EMPTY_ARRAY = new FieldsEnumWithSlice[0];
final FieldsEnum fields;
final ReaderUtil.Slice slice;
final ReaderSlice slice;
final int index;
String current;
public FieldsEnumWithSlice(FieldsEnum fields, ReaderUtil.Slice slice, int index) throws IOException {
public FieldsEnumWithSlice(FieldsEnum fields, ReaderSlice slice, int index) throws IOException {
this.slice = slice;
this.index = index;
assert slice.length >= 0: "length=" + slice.length;

View File

@ -68,12 +68,12 @@ public class MultiReader extends BaseCompositeReader<IndexReader> {
@Override
protected synchronized void doClose() throws IOException {
IOException ioe = null;
for (int i = 0; i < subReaders.length; i++) {
for (final IndexReader r : getSequentialSubReaders()) {
try {
if (closeSubReaders) {
subReaders[i].close();
r.close();
} else {
subReaders[i].decRef();
r.decRef();
}
} catch (IOException e) {
if (ioe == null) ioe = e;

View File

@ -23,7 +23,7 @@ import java.util.Comparator;
import java.util.List;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.ReaderUtil;
import org.apache.lucene.util.ReaderSlice;
import org.apache.lucene.util.automaton.CompiledAutomaton;
@ -36,10 +36,10 @@ import org.apache.lucene.util.automaton.CompiledAutomaton;
public final class MultiTerms extends Terms {
private final Terms[] subs;
private final ReaderUtil.Slice[] subSlices;
private final ReaderSlice[] subSlices;
private final Comparator<BytesRef> termComp;
public MultiTerms(Terms[] subs, ReaderUtil.Slice[] subSlices) throws IOException {
public MultiTerms(Terms[] subs, ReaderSlice[] subSlices) throws IOException {
this.subs = subs;
this.subSlices = subSlices;

View File

@ -22,7 +22,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BitsSlice;
import org.apache.lucene.util.MultiBits;
import org.apache.lucene.util.ReaderUtil;
import org.apache.lucene.util.ReaderSlice;
import java.io.IOException;
import java.util.Arrays;
@ -71,7 +71,7 @@ public final class MultiTermsEnum extends TermsEnum {
return top;
}
public MultiTermsEnum(ReaderUtil.Slice[] slices) {
public MultiTermsEnum(ReaderSlice[] slices) {
queue = new TermMergeQueue(slices.length);
top = new TermsEnumWithSlice[slices.length];
subs = new TermsEnumWithSlice[slices.length];
@ -494,12 +494,12 @@ public final class MultiTermsEnum extends TermsEnum {
}
private final static class TermsEnumWithSlice {
private final ReaderUtil.Slice subSlice;
private final ReaderSlice subSlice;
private TermsEnum terms;
public BytesRef current;
final int index;
public TermsEnumWithSlice(int index, ReaderUtil.Slice subSlice) {
public TermsEnumWithSlice(int index, ReaderSlice subSlice) {
this.subSlice = subSlice;
this.index = index;
assert subSlice.length >= 0: "length=" + subSlice.length;

View File

@ -21,6 +21,7 @@ import java.io.IOException;
import java.util.Collections;
import java.util.IdentityHashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
/** An {@link CompositeReader} which reads multiple, parallel indexes. Each index added
@ -85,44 +86,45 @@ public final class ParallelCompositeReader extends BaseCompositeReader<IndexRead
throw new IllegalArgumentException("There must be at least one main reader if storedFieldsReaders are used.");
return new IndexReader[0];
} else {
final IndexReader[] firstSubReaders = readers[0].getSequentialSubReaders();
final List<? extends IndexReader> firstSubReaders = readers[0].getSequentialSubReaders();
// check compatibility:
final int maxDoc = readers[0].maxDoc();
final int[] childMaxDoc = new int[firstSubReaders.length];
final boolean[] childAtomic = new boolean[firstSubReaders.length];
for (int i = 0; i < firstSubReaders.length; i++) {
childMaxDoc[i] = firstSubReaders[i].maxDoc();
childAtomic[i] = firstSubReaders[i] instanceof AtomicReader;
final int maxDoc = readers[0].maxDoc(), noSubs = firstSubReaders.size();
final int[] childMaxDoc = new int[noSubs];
final boolean[] childAtomic = new boolean[noSubs];
for (int i = 0; i < noSubs; i++) {
final IndexReader r = firstSubReaders.get(i);
childMaxDoc[i] = r.maxDoc();
childAtomic[i] = r instanceof AtomicReader;
}
validate(readers, maxDoc, childMaxDoc, childAtomic);
validate(storedFieldsReaders, maxDoc, childMaxDoc, childAtomic);
// hierarchically build the same subreader structure as the first CompositeReader with Parallel*Readers:
final IndexReader[] subReaders = new IndexReader[firstSubReaders.length];
final IndexReader[] subReaders = new IndexReader[noSubs];
for (int i = 0; i < subReaders.length; i++) {
if (firstSubReaders[i] instanceof AtomicReader) {
if (firstSubReaders.get(i) instanceof AtomicReader) {
final AtomicReader[] atomicSubs = new AtomicReader[readers.length];
for (int j = 0; j < readers.length; j++) {
atomicSubs[j] = (AtomicReader) readers[j].getSequentialSubReaders()[i];
atomicSubs[j] = (AtomicReader) readers[j].getSequentialSubReaders().get(i);
}
final AtomicReader[] storedSubs = new AtomicReader[storedFieldsReaders.length];
for (int j = 0; j < storedFieldsReaders.length; j++) {
storedSubs[j] = (AtomicReader) storedFieldsReaders[j].getSequentialSubReaders()[i];
storedSubs[j] = (AtomicReader) storedFieldsReaders[j].getSequentialSubReaders().get(i);
}
// we simply enable closing of subReaders, to prevent incRefs on subReaders
// -> for synthetic subReaders, close() is never
// called by our doClose()
subReaders[i] = new ParallelAtomicReader(true, atomicSubs, storedSubs);
} else {
assert firstSubReaders[i] instanceof CompositeReader;
assert firstSubReaders.get(i) instanceof CompositeReader;
final CompositeReader[] compositeSubs = new CompositeReader[readers.length];
for (int j = 0; j < readers.length; j++) {
compositeSubs[j] = (CompositeReader) readers[j].getSequentialSubReaders()[i];
compositeSubs[j] = (CompositeReader) readers[j].getSequentialSubReaders().get(i);
}
final CompositeReader[] storedSubs = new CompositeReader[storedFieldsReaders.length];
for (int j = 0; j < storedFieldsReaders.length; j++) {
storedSubs[j] = (CompositeReader) storedFieldsReaders[j].getSequentialSubReaders()[i];
storedSubs[j] = (CompositeReader) storedFieldsReaders[j].getSequentialSubReaders().get(i);
}
// we simply enable closing of subReaders, to prevent incRefs on subReaders
// -> for synthetic subReaders, close() is never called by our doClose()
@ -136,18 +138,20 @@ public final class ParallelCompositeReader extends BaseCompositeReader<IndexRead
private static void validate(CompositeReader[] readers, int maxDoc, int[] childMaxDoc, boolean[] childAtomic) {
for (int i = 0; i < readers.length; i++) {
final CompositeReader reader = readers[i];
final IndexReader[] subs = reader.getSequentialSubReaders();
final List<? extends IndexReader> subs = reader.getSequentialSubReaders();
if (reader.maxDoc() != maxDoc) {
throw new IllegalArgumentException("All readers must have same maxDoc: "+maxDoc+"!="+reader.maxDoc());
}
if (subs.length != childMaxDoc.length) {
final int noSubs = subs.size();
if (noSubs != childMaxDoc.length) {
throw new IllegalArgumentException("All readers must have same number of subReaders");
}
for (int subIDX = 0; subIDX < subs.length; subIDX++) {
if (subs[subIDX].maxDoc() != childMaxDoc[subIDX]) {
for (int subIDX = 0; subIDX < noSubs; subIDX++) {
final IndexReader r = subs.get(subIDX);
if (r.maxDoc() != childMaxDoc[subIDX]) {
throw new IllegalArgumentException("All readers must have same corresponding subReader maxDoc");
}
if (!(childAtomic[subIDX] ? (subs[subIDX] instanceof AtomicReader) : (subs[subIDX] instanceof CompositeReader))) {
if (!(childAtomic[subIDX] ? (r instanceof AtomicReader) : (r instanceof CompositeReader))) {
throw new IllegalArgumentException("All readers must have same corresponding subReader types (atomic or composite)");
}
}

View File

@ -35,6 +35,7 @@ import org.apache.lucene.util.Bits;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.ReaderUtil;
import org.apache.lucene.util.ReaderSlice;
/**
* The SegmentMerger class combines two or more Segments, represented by an IndexReader ({@link #add},
@ -76,16 +77,9 @@ final class SegmentMerger {
* @param reader
*/
final void add(IndexReader reader) {
try {
new ReaderUtil.Gather(reader) {
@Override
protected void add(int base, AtomicReader r) {
mergeState.readers.add(new MergeState.IndexReaderAndLiveDocs(r, r.getLiveDocs(), r.numDeletedDocs()));
}
}.run();
} catch (IOException ioe) {
// won't happen
throw new RuntimeException(ioe);
for (final AtomicReaderContext ctx : reader.getTopReaderContext().leaves()) {
final AtomicReader r = ctx.reader();
mergeState.readers.add(new MergeState.IndexReaderAndLiveDocs(r, r.getLiveDocs(), r.numDeletedDocs()));
}
}
@ -311,7 +305,7 @@ final class SegmentMerger {
private final void mergeTerms(SegmentWriteState segmentWriteState) throws CorruptIndexException, IOException {
final List<Fields> fields = new ArrayList<Fields>();
final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
final List<ReaderSlice> slices = new ArrayList<ReaderSlice>();
int docBase = 0;
@ -320,7 +314,7 @@ final class SegmentMerger {
final Fields f = r.reader.fields();
final int maxDoc = r.reader.maxDoc();
if (f != null) {
slices.add(new ReaderUtil.Slice(docBase, maxDoc, readerIndex));
slices.add(new ReaderSlice(docBase, maxDoc, readerIndex));
fields.add(f);
}
docBase += maxDoc;
@ -331,7 +325,7 @@ final class SegmentMerger {
try {
consumer.merge(mergeState,
new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY)));
slices.toArray(ReaderSlice.EMPTY_ARRAY)));
success = true;
} finally {
if (success) {

View File

@ -22,7 +22,6 @@ import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ReaderUtil; // javadoc
import org.apache.lucene.index.DirectoryReader; // javadoc
import org.apache.lucene.index.MultiReader; // javadoc
@ -32,17 +31,16 @@ import org.apache.lucene.index.MultiReader; // javadoc
* MultiReader} or {@link DirectoryReader}) to emulate an
* atomic reader. This requires implementing the postings
* APIs on-the-fly, using the static methods in {@link
* MultiFields}, {@link MultiDocValues},
* by stepping through the sub-readers to merge fields/terms,
* appending docs, etc.
* MultiFields}, {@link MultiDocValues}, by stepping through
* the sub-readers to merge fields/terms, appending docs, etc.
*
* <p><b>NOTE</b>: this class almost always results in a
* performance hit. If this is important to your use case,
* it's better to get the sequential sub readers (see {@link
* ReaderUtil#gatherSubReaders}, instead, and iterate through them
* yourself.</p>
* you'll get better performance by gathering the sub readers using
* {@link IndexReader#getTopReaderContext()} to get the
* atomic leaves and then operate per-AtomicReader,
* instead of using this class.
*/
public final class SlowCompositeReaderWrapper extends AtomicReader {
private final CompositeReader in;

View File

@ -118,8 +118,8 @@ final class StandardDirectoryReader extends DirectoryReader {
writer, segmentInfos, writer.getConfig().getReaderTermsIndexDivisor(), applyAllDeletes);
}
/** This constructor is only used for {@link #doOpenIfChanged()} */
private static DirectoryReader open(Directory directory, IndexWriter writer, SegmentInfos infos, AtomicReader[] oldReaders,
/** This constructor is only used for {@link #doOpenIfChanged(SegmentInfos, IndexWriter)} */
private static DirectoryReader open(Directory directory, IndexWriter writer, SegmentInfos infos, List<? extends AtomicReader> oldReaders,
int termInfosIndexDivisor) throws IOException {
// we put the old SegmentReaders in a map, that allows us
// to lookup a reader using its segment name
@ -127,8 +127,9 @@ final class StandardDirectoryReader extends DirectoryReader {
if (oldReaders != null) {
// create a Map SegmentName->SegmentReader
for (int i = 0; i < oldReaders.length; i++) {
segmentReaders.put(((SegmentReader) oldReaders[i]).getSegmentName(), Integer.valueOf(i));
for (int i = 0, c = oldReaders.size(); i < c; i++) {
final SegmentReader sr = (SegmentReader) oldReaders.get(i);
segmentReaders.put(sr.getSegmentName(), Integer.valueOf(i));
}
}
@ -146,7 +147,7 @@ final class StandardDirectoryReader extends DirectoryReader {
newReaders[i] = null;
} else {
// there is an old reader for this segment - we'll try to reopen it
newReaders[i] = (SegmentReader) oldReaders[oldReaderIndex.intValue()];
newReaders[i] = (SegmentReader) oldReaders.get(oldReaderIndex.intValue());
}
boolean success = false;
@ -216,9 +217,9 @@ final class StandardDirectoryReader extends DirectoryReader {
if (writer != null) {
buffer.append(":nrt");
}
for(int i=0;i<subReaders.length;i++) {
for (final AtomicReader r : getSequentialSubReaders()) {
buffer.append(' ');
buffer.append(subReaders[i]);
buffer.append(r);
}
buffer.append(')');
return buffer.toString();
@ -298,7 +299,7 @@ final class StandardDirectoryReader extends DirectoryReader {
}
synchronized DirectoryReader doOpenIfChanged(SegmentInfos infos, IndexWriter writer) throws CorruptIndexException, IOException {
return StandardDirectoryReader.open(directory, writer, infos, subReaders, termInfosIndexDivisor);
return StandardDirectoryReader.open(directory, writer, infos, getSequentialSubReaders(), termInfosIndexDivisor);
}
@Override
@ -329,10 +330,10 @@ final class StandardDirectoryReader extends DirectoryReader {
@Override
protected synchronized void doClose() throws IOException {
IOException ioe = null;
for (int i = 0; i < subReaders.length; i++) {
for (final AtomicReader r : getSequentialSubReaders()) {
// try to close each reader, even if an exception is thrown
try {
subReaders[i].decRef();
r.decRef();
} catch (IOException e) {
if (ioe == null) ioe = e;
}

View File

@ -18,7 +18,9 @@ package org.apache.lucene.search;
*/
import java.io.IOException;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
import java.util.NoSuchElementException;
import java.util.Set;
import java.util.concurrent.Callable;
@ -80,7 +82,7 @@ public class IndexSearcher {
// NOTE: these members might change in incompatible ways
// in the next release
protected final IndexReaderContext readerContext;
protected final AtomicReaderContext[] leafContexts;
protected final List<AtomicReaderContext> leafContexts;
// used with executor - each slice holds a set of leafs executed within one thread
protected final LeafSlice[] leafSlices;
@ -165,10 +167,10 @@ public class IndexSearcher {
* Each {@link LeafSlice} is executed in a single thread. By default there
* will be one {@link LeafSlice} per leaf ({@link AtomicReaderContext}).
*/
protected LeafSlice[] slices(AtomicReaderContext...leaves) {
LeafSlice[] slices = new LeafSlice[leaves.length];
protected LeafSlice[] slices(List<AtomicReaderContext> leaves) {
LeafSlice[] slices = new LeafSlice[leaves.size()];
for (int i = 0; i < slices.length; i++) {
slices[i] = new LeafSlice(leaves[i]);
slices[i] = new LeafSlice(leaves.get(i));
}
return slices;
}
@ -440,7 +442,7 @@ public class IndexSearcher {
* {@link IndexSearcher#search(Query,Filter,int)} instead.
* @throws BooleanQuery.TooManyClauses
*/
protected TopDocs search(AtomicReaderContext[] leaves, Weight weight, ScoreDoc after, int nDocs) throws IOException {
protected TopDocs search(List<AtomicReaderContext> leaves, Weight weight, ScoreDoc after, int nDocs) throws IOException {
// single thread
int limit = reader.maxDoc();
if (limit == 0) {
@ -477,7 +479,7 @@ public class IndexSearcher {
* <p>NOTE: this does not compute scores by default. If you
* need scores, create a {@link TopFieldCollector}
* instance by calling {@link TopFieldCollector#create} and
* then pass that to {@link #search(AtomicReaderContext[], Weight,
* then pass that to {@link #search(List, Weight,
* Collector)}.</p>
*/
protected TopFieldDocs search(Weight weight, FieldDoc after, int nDocs,
@ -525,7 +527,7 @@ public class IndexSearcher {
* whether or not the fields in the returned {@link FieldDoc} instances should
* be set by specifying fillFields.
*/
protected TopFieldDocs search(AtomicReaderContext[] leaves, Weight weight, FieldDoc after, int nDocs,
protected TopFieldDocs search(List<AtomicReaderContext> leaves, Weight weight, FieldDoc after, int nDocs,
Sort sort, boolean fillFields, boolean doDocScores, boolean doMaxScore) throws IOException {
// single thread
int limit = reader.maxDoc();
@ -559,15 +561,15 @@ public class IndexSearcher {
* to receive hits
* @throws BooleanQuery.TooManyClauses
*/
protected void search(AtomicReaderContext[] leaves, Weight weight, Collector collector)
protected void search(List<AtomicReaderContext> leaves, Weight weight, Collector collector)
throws IOException {
// TODO: should we make this
// threaded...? the Collector could be sync'd?
// always use single thread:
for (int i = 0; i < leaves.length; i++) { // search each subreader
collector.setNextReader(leaves[i]);
Scorer scorer = weight.scorer(leaves[i], !collector.acceptsDocsOutOfOrder(), true, leaves[i].reader().getLiveDocs());
for (AtomicReaderContext ctx : leaves) { // search each subreader
collector.setNextReader(ctx);
Scorer scorer = weight.scorer(ctx, !collector.acceptsDocsOutOfOrder(), true, ctx.reader().getLiveDocs());
if (scorer != null) {
scorer.score(collector);
}
@ -611,9 +613,10 @@ public class IndexSearcher {
*/
protected Explanation explain(Weight weight, int doc) throws IOException {
int n = ReaderUtil.subIndex(doc, leafContexts);
int deBasedDoc = doc - leafContexts[n].docBase;
final AtomicReaderContext ctx = leafContexts.get(n);
int deBasedDoc = doc - ctx.docBase;
return weight.explain(leafContexts[n], deBasedDoc);
return weight.explain(ctx, deBasedDoc);
}
/**
@ -669,7 +672,7 @@ public class IndexSearcher {
}
public TopDocs call() throws IOException {
final TopDocs docs = searcher.search(slice.leaves, weight, after, nDocs);
final TopDocs docs = searcher.search(Arrays.asList(slice.leaves), weight, after, nDocs);
final ScoreDoc[] scoreDocs = docs.scoreDocs;
//it would be so nice if we had a thread-safe insert
lock.lock();
@ -757,11 +760,13 @@ public class IndexSearcher {
public TopFieldDocs call() throws IOException {
assert slice.leaves.length == 1;
final TopFieldDocs docs = searcher.search(slice.leaves, weight, after, nDocs, sort, true, doDocScores, doMaxScore);
final TopFieldDocs docs = searcher.search(Arrays.asList(slice.leaves),
weight, after, nDocs, sort, true, doDocScores, doMaxScore);
lock.lock();
try {
final int base = slice.leaves[0].docBase;
hq.setNextReader(slice.leaves[0]);
final AtomicReaderContext ctx = slice.leaves[0];
final int base = ctx.docBase;
hq.setNextReader(ctx);
hq.setScorer(fakeScorer);
for(ScoreDoc scoreDoc : docs.scoreDocs) {
fakeScorer.doc = scoreDoc.doc - base;
@ -837,7 +842,7 @@ public class IndexSearcher {
public static class LeafSlice {
final AtomicReaderContext[] leaves;
public LeafSlice(AtomicReaderContext...leaves) {
public LeafSlice(AtomicReaderContext... leaves) {
this.leaves = leaves;
}
}

View File

@ -48,8 +48,7 @@ abstract class TermCollectingRewrite<Q extends Query> extends MultiTermQuery.Rew
final void collectTerms(IndexReader reader, MultiTermQuery query, TermCollector collector) throws IOException {
IndexReaderContext topReaderContext = reader.getTopReaderContext();
Comparator<BytesRef> lastTermComp = null;
final AtomicReaderContext[] leaves = topReaderContext.leaves();
for (AtomicReaderContext context : leaves) {
for (AtomicReaderContext context : topReaderContext.leaves()) {
final Fields fields = context.reader().fields();
if (fields == null) {
// reader has no fields

View File

@ -184,8 +184,7 @@ public class PayloadSpanUtil {
for (Term term : terms) {
termContexts.put(term, TermContext.build(context, term, true));
}
final AtomicReaderContext[] leaves = context.leaves();
for (AtomicReaderContext atomicReaderContext : leaves) {
for (AtomicReaderContext atomicReaderContext : context.leaves()) {
final Spans spans = query.getSpans(atomicReaderContext, atomicReaderContext.reader().getLiveDocs(), termContexts);
while (spans.next() == true) {
if (spans.isPayloadAvailable()) {

View File

@ -25,7 +25,7 @@ public final class BitsSlice implements Bits {
private final int length;
// start is inclusive; end is exclusive (length = end-start)
public BitsSlice(Bits parent, ReaderUtil.Slice slice) {
public BitsSlice(Bits parent, ReaderSlice slice) {
this.parent = parent;
this.start = slice.start;
this.length = slice.length;

View File

@ -280,9 +280,9 @@ public final class FieldCacheSanityChecker {
for (int i = 0; i < all.size(); i++) {
Object obj = all.get(i);
if (obj instanceof CompositeReader) {
IndexReader[] subs = ((CompositeReader)obj).getSequentialSubReaders();
for (int j = 0; (null != subs) && (j < subs.length); j++) {
all.add(subs[j].getCoreCacheKey());
List<? extends IndexReader> subs = ((CompositeReader)obj).getSequentialSubReaders();
for (int j = 0; (null != subs) && (j < subs.size()); j++) {
all.add(subs.get(j).getCoreCacheKey());
}
}

View File

@ -17,8 +17,6 @@ package org.apache.lucene.util;
* limitations under the License.
*/
import java.util.List;
/**
* Concatenates multiple Bits together, on every lookup.
*
@ -36,13 +34,10 @@ public final class MultiBits implements Bits {
private final boolean defaultValue;
public MultiBits(List<Bits> bits, List<Integer> starts, boolean defaultValue) {
assert starts.size() == 1+bits.size();
this.subs = bits.toArray(Bits.EMPTY_ARRAY);
this.starts = new int[starts.size()];
for(int i=0;i<this.starts.length;i++) {
this.starts[i] = starts.get(i);
}
public MultiBits(Bits[] subs, int[] starts, boolean defaultValue) {
assert starts.length == 1+subs.length;
this.subs = subs;
this.starts = starts;
this.defaultValue = defaultValue;
}
@ -84,7 +79,7 @@ public final class MultiBits implements Bits {
/**
* Represents a sub-Bits from
* {@link MultiBits#getMatchingSub(org.apache.lucene.util.ReaderUtil.Slice) getMatchingSub()}.
* {@link MultiBits#getMatchingSub(org.apache.lucene.util.ReaderSlice) getMatchingSub()}.
*/
public final static class SubResult {
public boolean matches;
@ -99,7 +94,7 @@ public final class MultiBits implements Bits {
* {@link SubResult#matches} instead to ensure the sub was
* actually found.
*/
public SubResult getMatchingSub(ReaderUtil.Slice slice) {
public SubResult getMatchingSub(ReaderSlice slice) {
int reader = ReaderUtil.subIndex(slice.start, starts);
assert reader != -1;
assert reader < subs.length: "slice=" + slice + " starts[-1]=" + starts[starts.length-1];

View File

@ -0,0 +1,39 @@
package org.apache.lucene.util;
/*
* 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.
*/
/**
* Subreader slice from a parent composite reader.
*/
public final class ReaderSlice {
public static final ReaderSlice[] EMPTY_ARRAY = new ReaderSlice[0];
public final int start;
public final int length;
public final int readerIndex;
public ReaderSlice(int start, int length, int readerIndex) {
this.start = start;
this.length = length;
this.readerIndex = readerIndex;
}
@Override
public String toString() {
return "slice start=" + start + " length=" + length + " readerIndex=" + readerIndex;
}
}

View File

@ -18,11 +18,8 @@ package org.apache.lucene.util;
*/
import java.util.List;
import java.io.IOException;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.CompositeReader;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReaderContext;
@ -35,92 +32,6 @@ public final class ReaderUtil {
private ReaderUtil() {} // no instance
/**
* Subreader slice from a parent composite reader.
*/
public static class Slice {
public static final Slice[] EMPTY_ARRAY = new Slice[0];
public final int start;
public final int length;
public final int readerIndex;
public Slice(int start, int length, int readerIndex) {
this.start = start;
this.length = length;
this.readerIndex = readerIndex;
}
@Override
public String toString() {
return "slice start=" + start + " length=" + length + " readerIndex=" + readerIndex;
}
}
/**
* Gathers sub-readers from reader into a List. See
* {@link Gather} for are more general way to gather
* whatever you need to, per reader.
*
* @lucene.experimental
*
* @param allSubReaders
* @param reader
*/
public static void gatherSubReaders(final List<AtomicReader> allSubReaders, IndexReader reader) {
try {
new Gather(reader) {
@Override
protected void add(int base, AtomicReader r) {
allSubReaders.add(r);
}
}.run();
} catch (IOException ioe) {
// won't happen
throw new RuntimeException(ioe);
}
}
/** Recursively visits all sub-readers of a reader. You
* should subclass this and override the add method to
* gather what you need.
*
* @lucene.experimental */
public static abstract class Gather {
private final IndexReader topReader;
public Gather(IndexReader r) {
topReader = r;
}
public int run() throws IOException {
return run(0, topReader);
}
public int run(int docBase) throws IOException {
return run(docBase, topReader);
}
private int run(final int base, final IndexReader reader) throws IOException {
if (reader instanceof AtomicReader) {
// atomic reader
add(base, (AtomicReader) reader);
return base + reader.maxDoc();
} else {
assert reader instanceof CompositeReader : "must be a composite reader";
int newBase = base;
IndexReader[] subReaders = ((CompositeReader) reader).getSequentialSubReaders();
for (int i = 0; i < subReaders.length; i++) {
newBase = run(newBase, subReaders[i]);
}
assert newBase == base + reader.maxDoc();
return newBase;
}
}
protected abstract void add(int base, AtomicReader r) throws IOException;
}
/**
* Walks up the reader tree and return the given context's top level reader
* context, or in other words the reader tree's root context.
@ -162,20 +73,20 @@ public final class ReaderUtil {
* Returns index of the searcher/reader for document <code>n</code> in the
* array used to construct this searcher/reader.
*/
public static int subIndex(int n, AtomicReaderContext[] leaves) { // find
public static int subIndex(int n, List<AtomicReaderContext> leaves) { // find
// searcher/reader for doc n:
int size = leaves.length;
int size = leaves.size();
int lo = 0; // search starts array
int hi = size - 1; // for first element less than n, return its index
while (hi >= lo) {
int mid = (lo + hi) >>> 1;
int midValue = leaves[mid].docBase;
int midValue = leaves.get(mid).docBase;
if (n < midValue)
hi = mid - 1;
else if (n > midValue)
lo = mid + 1;
else { // found a match
while (mid + 1 < size && leaves[mid + 1].docBase == midValue) {
while (mid + 1 < size && leaves.get(mid + 1).docBase == midValue) {
mid++; // scan to last match
}
return mid;

View File

@ -57,7 +57,7 @@ public final class TermContext {
if (context.leaves() == null) {
len = 1;
} else {
len = context.leaves().length;
len = context.leaves().size();
}
states = new TermState[len];
}
@ -85,11 +85,10 @@ public final class TermContext {
final String field = term.field();
final BytesRef bytes = term.bytes();
final TermContext perReaderTermState = new TermContext(context);
final AtomicReaderContext[] leaves = context.leaves();
//if (DEBUG) System.out.println("prts.build term=" + term);
for (int i = 0; i < leaves.length; i++) {
for (final AtomicReaderContext ctx : context.leaves()) {
//if (DEBUG) System.out.println(" r=" + leaves[i].reader);
final Fields fields = leaves[i].reader().fields();
final Fields fields = ctx.reader().fields();
if (fields != null) {
final Terms terms = fields.terms(field);
if (terms != null) {
@ -97,7 +96,7 @@ public final class TermContext {
if (termsEnum.seekExact(bytes, cache)) {
final TermState termState = termsEnum.termState();
//if (DEBUG) System.out.println(" found");
perReaderTermState.register(termState, leaves[i].ord, termsEnum.docFreq(), termsEnum.totalTermFreq());
perReaderTermState.register(termState, ctx.ord, termsEnum.docFreq(), termsEnum.totalTermFreq());
}
}
}

View File

@ -17,6 +17,7 @@ package org.apache.lucene.codecs.lucene40;
*/
import java.io.IOException;
import java.util.IdentityHashMap;
import java.util.List;
import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
@ -35,7 +36,6 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LineFileDocs;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.ReaderUtil;
import org.apache.lucene.util._TestUtil;
public class TestReuseDocsEnum extends LuceneTestCase {
@ -50,21 +50,18 @@ public class TestReuseDocsEnum extends LuceneTestCase {
writer.commit();
DirectoryReader open = DirectoryReader.open(dir);
new ReaderUtil.Gather(open) {
@Override
protected void add(int base, AtomicReader r) throws IOException {
Terms terms = r.terms("body");
TermsEnum iterator = terms.iterator(null);
IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<DocsEnum, Boolean>();
MatchNoBits bits = new Bits.MatchNoBits(r.maxDoc());
while ((iterator.next()) != null) {
DocsEnum docs = iterator.docs(random().nextBoolean() ? bits : new Bits.MatchNoBits(r.maxDoc()), null, random().nextBoolean());
enums.put(docs, true);
}
assertEquals(terms.size(), enums.size());
for (AtomicReader indexReader : open.getSequentialSubReaders()) {
Terms terms = indexReader.terms("body");
TermsEnum iterator = terms.iterator(null);
IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<DocsEnum, Boolean>();
MatchNoBits bits = new Bits.MatchNoBits(indexReader.maxDoc());
while ((iterator.next()) != null) {
DocsEnum docs = iterator.docs(random().nextBoolean() ? bits : new Bits.MatchNoBits(indexReader.maxDoc()), null, random().nextBoolean());
enums.put(docs, true);
}
}.run();
assertEquals(terms.size(), enums.size());
}
IOUtils.close(writer, open, dir);
}
@ -79,9 +76,8 @@ public class TestReuseDocsEnum extends LuceneTestCase {
writer.commit();
DirectoryReader open = DirectoryReader.open(dir);
IndexReader[] sequentialSubReaders = open.getSequentialSubReaders();
for (IndexReader indexReader : sequentialSubReaders) {
Terms terms = ((AtomicReader) indexReader).terms("body");
for (AtomicReader indexReader : open.getSequentialSubReaders()) {
Terms terms = indexReader.terms("body");
TermsEnum iterator = terms.iterator(null);
IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<DocsEnum, Boolean>();
MatchNoBits bits = new Bits.MatchNoBits(open.maxDoc());
@ -125,8 +121,8 @@ public class TestReuseDocsEnum extends LuceneTestCase {
DirectoryReader firstReader = DirectoryReader.open(dir);
DirectoryReader secondReader = DirectoryReader.open(dir);
IndexReader[] sequentialSubReaders = firstReader.getSequentialSubReaders();
IndexReader[] sequentialSubReaders2 = secondReader.getSequentialSubReaders();
List<? extends AtomicReader> sequentialSubReaders = firstReader.getSequentialSubReaders();
List<? extends AtomicReader> sequentialSubReaders2 = secondReader.getSequentialSubReaders();
for (IndexReader indexReader : sequentialSubReaders) {
Terms terms = ((AtomicReader) indexReader).terms("body");
@ -154,11 +150,11 @@ public class TestReuseDocsEnum extends LuceneTestCase {
IOUtils.close(writer, firstReader, secondReader, dir);
}
public DocsEnum randomDocsEnum(String field, BytesRef term, IndexReader[] readers, Bits bits) throws IOException {
public DocsEnum randomDocsEnum(String field, BytesRef term, List<? extends AtomicReader> readers, Bits bits) throws IOException {
if (random().nextInt(10) == 0) {
return null;
}
AtomicReader indexReader = (AtomicReader) readers[random().nextInt(readers.length)];
AtomicReader indexReader = (AtomicReader) readers.get(random().nextInt(readers.size()));
return indexReader.termDocsEnum(bits, field, term, random().nextBoolean());
}

View File

@ -145,25 +145,24 @@ public class TestCustomNorms extends LuceneTestCase {
writer.close();
assertEquals(numAdded, reader.numDocs());
IndexReaderContext topReaderContext = reader.getTopReaderContext();
AtomicReaderContext[] leaves = topReaderContext.leaves();
for (int j = 0; j < leaves.length; j++) {
AtomicReader atomicReader = leaves[j].reader();
Source source = random().nextBoolean() ? atomicReader.normValues("foo").getSource() : atomicReader.normValues("foo").getDirectSource();
Bits liveDocs = atomicReader.getLiveDocs();
Type t = source.getType();
for (int i = 0; i < atomicReader.maxDoc(); i++) {
assertEquals(0, source.getFloat(i), 0.000f);
}
source = random().nextBoolean() ? atomicReader.normValues("bar").getSource() : atomicReader.normValues("bar").getDirectSource();
for (int i = 0; i < atomicReader.maxDoc(); i++) {
if (liveDocs == null || liveDocs.get(i)) {
assertEquals("type: " + t, 1, source.getFloat(i), 0.000f);
} else {
assertEquals("type: " + t, 0, source.getFloat(i), 0.000f);
for (final AtomicReaderContext ctx : topReaderContext.leaves()) {
AtomicReader atomicReader = ctx.reader();
Source source = random().nextBoolean() ? atomicReader.normValues("foo").getSource() : atomicReader.normValues("foo").getDirectSource();
Bits liveDocs = atomicReader.getLiveDocs();
Type t = source.getType();
for (int i = 0; i < atomicReader.maxDoc(); i++) {
assertEquals(0, source.getFloat(i), 0.000f);
}
source = random().nextBoolean() ? atomicReader.normValues("bar").getSource() : atomicReader.normValues("bar").getDirectSource();
for (int i = 0; i < atomicReader.maxDoc(); i++) {
if (liveDocs == null || liveDocs.get(i)) {
assertEquals("type: " + t, 1, source.getFloat(i), 0.000f);
} else {
assertEquals("type: " + t, 0, source.getFloat(i), 0.000f);
}
}
}
}
reader.close();
dir.close();

View File

@ -67,7 +67,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
public void onCommit(List<? extends IndexCommit> commits) throws IOException {
IndexCommit lastCommit = commits.get(commits.size()-1);
DirectoryReader r = DirectoryReader.open(dir);
assertEquals("lastCommit.segmentCount()=" + lastCommit.getSegmentCount() + " vs IndexReader.segmentCount=" + r.getSequentialSubReaders().length, r.getSequentialSubReaders().length, lastCommit.getSegmentCount());
assertEquals("lastCommit.segmentCount()=" + lastCommit.getSegmentCount() + " vs IndexReader.segmentCount=" + r.getSequentialSubReaders().size(), r.getSequentialSubReaders().size(), lastCommit.getSegmentCount());
r.close();
verifyCommitOrder(commits);
numOnCommit++;
@ -325,7 +325,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
final boolean needsMerging;
{
DirectoryReader r = DirectoryReader.open(dir);
needsMerging = r.getSequentialSubReaders().length != 1;
needsMerging = r.getSequentialSubReaders().size() != 1;
r.close();
}
if (needsMerging) {
@ -442,7 +442,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
DirectoryReader r = DirectoryReader.open(dir);
// Still merged, still 11 docs
assertEquals(1, r.getSequentialSubReaders().length);
assertEquals(1, r.getSequentialSubReaders().size());
assertEquals(11, r.numDocs());
r.close();
@ -458,7 +458,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
r = DirectoryReader.open(dir);
// Not fully merged because we rolled it back, and now only
// 10 docs
assertTrue(r.getSequentialSubReaders().length > 1);
assertTrue(r.getSequentialSubReaders().size() > 1);
assertEquals(10, r.numDocs());
r.close();
@ -468,7 +468,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
writer.close();
r = DirectoryReader.open(dir);
assertEquals(1, r.getSequentialSubReaders().length);
assertEquals(1, r.getSequentialSubReaders().size());
assertEquals(10, r.numDocs());
r.close();
@ -480,7 +480,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
// Reader still sees fully merged index, because writer
// opened on the prior commit has not yet committed:
r = DirectoryReader.open(dir);
assertEquals(1, r.getSequentialSubReaders().length);
assertEquals(1, r.getSequentialSubReaders().size());
assertEquals(10, r.numDocs());
r.close();
@ -488,7 +488,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
// Now reader sees not-fully-merged index:
r = DirectoryReader.open(dir);
assertTrue(r.getSequentialSubReaders().length > 1);
assertTrue(r.getSequentialSubReaders().size() > 1);
assertEquals(10, r.numDocs());
r.close();

View File

@ -550,7 +550,7 @@ public void testFilesOpenClose() throws IOException {
assertEquals("IndexReaders have different values for numDocs.", index1.numDocs(), index2.numDocs());
assertEquals("IndexReaders have different values for maxDoc.", index1.maxDoc(), index2.maxDoc());
assertEquals("Only one IndexReader has deletions.", index1.hasDeletions(), index2.hasDeletions());
assertEquals("Single segment test differs.", index1.getSequentialSubReaders().length == 1, index2.getSequentialSubReaders().length == 1);
assertEquals("Single segment test differs.", index1.getSequentialSubReaders().size() == 1, index2.getSequentialSubReaders().size() == 1);
// check field names
FieldInfos fieldInfos1 = MultiFields.getMergedFieldInfos(index1);
@ -785,7 +785,7 @@ public void testFilesOpenClose() throws IOException {
DirectoryReader r2 = DirectoryReader.openIfChanged(r);
assertNotNull(r2);
r.close();
AtomicReader sub0 = r2.getSequentialSubReaders()[0];
AtomicReader sub0 = r2.getSequentialSubReaders().get(0);
final int[] ints2 = FieldCache.DEFAULT.getInts(sub0, "number", false);
r2.close();
assertTrue(ints == ints2);
@ -814,9 +814,9 @@ public void testFilesOpenClose() throws IOException {
assertNotNull(r2);
r.close();
IndexReader[] subs = r2.getSequentialSubReaders();
for(int i=0;i<subs.length;i++) {
assertEquals(36, ((AtomicReader) subs[i]).getUniqueTermCount());
List<? extends AtomicReader> subs = r2.getSequentialSubReaders();
for(AtomicReader s : subs) {
assertEquals(36, s.getUniqueTermCount());
}
r2.close();
writer.close();
@ -842,7 +842,7 @@ public void testFilesOpenClose() throws IOException {
// expected
}
assertEquals(-1, ((SegmentReader) r.getSequentialSubReaders()[0]).getTermInfosIndexDivisor());
assertEquals(-1, ((SegmentReader) r.getSequentialSubReaders().get(0)).getTermInfosIndexDivisor());
writer = new IndexWriter(
dir,
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).
@ -857,11 +857,11 @@ public void testFilesOpenClose() throws IOException {
assertNotNull(r2);
assertNull(DirectoryReader.openIfChanged(r2));
r.close();
IndexReader[] subReaders = r2.getSequentialSubReaders();
assertEquals(2, subReaders.length);
for(int i=0;i<2;i++) {
List<? extends AtomicReader> subReaders = r2.getSequentialSubReaders();
assertEquals(2, subReaders.size());
for(AtomicReader s : subReaders) {
try {
subReaders[i].docFreq(new Term("field", "f"));
s.docFreq(new Term("field", "f"));
fail("did not hit expected exception");
} catch (IllegalStateException ise) {
// expected

View File

@ -469,9 +469,9 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
DirectoryReader r = DirectoryReader.open(dir);
if (multiSegment) {
assertTrue(r.getSequentialSubReaders().length > 1);
assertTrue(r.getSequentialSubReaders().size() > 1);
} else {
assertTrue(r.getSequentialSubReaders().length == 1);
assertTrue(r.getSequentialSubReaders().size() == 1);
}
r.close();
}
@ -542,9 +542,9 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
}
if (checkSubReaders && reader instanceof CompositeReader) {
IndexReader[] subReaders = ((CompositeReader) reader).getSequentialSubReaders();
for (int i = 0; i < subReaders.length; i++) {
assertReaderClosed(subReaders[i], checkSubReaders, checkNormsClosed);
List<? extends IndexReader> subReaders = ((CompositeReader) reader).getSequentialSubReaders();
for (IndexReader r : subReaders) {
assertReaderClosed(r, checkSubReaders, checkNormsClosed);
}
}
}

View File

@ -77,7 +77,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
writer.close(true);
DirectoryReader reader = DirectoryReader.open(dir, 1);
assertEquals(1, reader.getSequentialSubReaders().length);
assertEquals(1, reader.getSequentialSubReaders().size());
IndexSearcher searcher = new IndexSearcher(reader);
@ -750,7 +750,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
w.forceMerge(1);
DirectoryReader r = w.getReader();
w.close();
assertEquals(17, r.getSequentialSubReaders()[0].docValues("field").load().getInt(0));
assertEquals(17, getOnlySegmentReader(r).docValues("field").load().getInt(0));
r.close();
d.close();
}
@ -994,8 +994,9 @@ public class TestDocValuesIndexing extends LuceneTestCase {
w.addDocument(doc);
bytes[0] = 1;
w.addDocument(doc);
w.forceMerge(1);
DirectoryReader r = w.getReader();
Source s = r.getSequentialSubReaders()[0].docValues("field").getSource();
Source s = getOnlySegmentReader(r).docValues("field").getSource();
BytesRef bytes1 = s.getBytes(0, new BytesRef());
assertEquals(bytes.length, bytes1.length);

View File

@ -64,8 +64,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
for (int i = 0; i < num; i++) {
BytesRef bytes = new BytesRef("1");
IndexReaderContext topReaderContext = reader.getTopReaderContext();
AtomicReaderContext[] leaves = topReaderContext.leaves();
for (AtomicReaderContext atomicReaderContext : leaves) {
for (AtomicReaderContext atomicReaderContext : topReaderContext.leaves()) {
DocsAndPositionsEnum docsAndPosEnum = getDocsAndPositions(
atomicReaderContext.reader(), bytes, null);
assertNotNull(docsAndPosEnum);
@ -140,8 +139,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
for (int i = 0; i < num; i++) {
BytesRef bytes = new BytesRef("" + term);
IndexReaderContext topReaderContext = reader.getTopReaderContext();
AtomicReaderContext[] leaves = topReaderContext.leaves();
for (AtomicReaderContext atomicReaderContext : leaves) {
for (AtomicReaderContext atomicReaderContext : topReaderContext.leaves()) {
DocsAndPositionsEnum docsAndPosEnum = getDocsAndPositions(
atomicReaderContext.reader(), bytes, null);
assertNotNull(docsAndPosEnum);
@ -216,8 +214,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
for (int i = 0; i < num; i++) {
BytesRef bytes = new BytesRef("" + term);
IndexReaderContext topReaderContext = reader.getTopReaderContext();
AtomicReaderContext[] leaves = topReaderContext.leaves();
for (AtomicReaderContext context : leaves) {
for (AtomicReaderContext context : topReaderContext.leaves()) {
int maxDoc = context.reader().maxDoc();
DocsEnum docsEnum = _TestUtil.docs(random(), context.reader(), fieldName, bytes, null, null, true);
if (findNext(freqInDoc, context.docBase, context.docBase + maxDoc) == Integer.MAX_VALUE) {
@ -295,8 +292,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
BytesRef bytes = new BytesRef("even");
IndexReaderContext topReaderContext = reader.getTopReaderContext();
AtomicReaderContext[] leaves = topReaderContext.leaves();
for (AtomicReaderContext atomicReaderContext : leaves) {
for (AtomicReaderContext atomicReaderContext : topReaderContext.leaves()) {
DocsAndPositionsEnum docsAndPosEnum = getDocsAndPositions(
atomicReaderContext.reader(), bytes, null);
assertNotNull(docsAndPosEnum);

View File

@ -282,7 +282,7 @@ public class TestIndexWriterCommit extends LuceneTestCase {
// Reader should see index as multi-seg at this
// point:
assertTrue("Reader incorrectly sees one segment", reader.getSequentialSubReaders().length > 1);
assertTrue("Reader incorrectly sees one segment", reader.getSequentialSubReaders().size() > 1);
reader.close();
// Abort the writer:
@ -293,7 +293,7 @@ public class TestIndexWriterCommit extends LuceneTestCase {
reader = DirectoryReader.open(dir);
// Reader should still see index as multi-segment
assertTrue("Reader incorrectly sees one segment", reader.getSequentialSubReaders().length > 1);
assertTrue("Reader incorrectly sees one segment", reader.getSequentialSubReaders().size() > 1);
reader.close();
if (VERBOSE) {
@ -312,7 +312,7 @@ public class TestIndexWriterCommit extends LuceneTestCase {
reader = DirectoryReader.open(dir);
// Reader should see index as one segment
assertEquals("Reader incorrectly sees more than one segment", 1, reader.getSequentialSubReaders().length);
assertEquals("Reader incorrectly sees more than one segment", 1, reader.getSequentialSubReaders().size());
reader.close();
dir.close();
}

View File

@ -187,7 +187,7 @@ public class TestIndexWriterForceMerge extends LuceneTestCase {
if (0 == pass) {
writer.close();
DirectoryReader reader = DirectoryReader.open(dir);
assertEquals(1, reader.getSequentialSubReaders().length);
assertEquals(1, reader.getSequentialSubReaders().size());
reader.close();
} else {
// Get another segment to flush so we can verify it is
@ -197,7 +197,7 @@ public class TestIndexWriterForceMerge extends LuceneTestCase {
writer.close();
DirectoryReader reader = DirectoryReader.open(dir);
assertTrue(reader.getSequentialSubReaders().length > 1);
assertTrue(reader.getSequentialSubReaders().size() > 1);
reader.close();
SegmentInfos infos = new SegmentInfos();

View File

@ -982,7 +982,7 @@ public class TestIndexWriterReader extends LuceneTestCase {
Document doc = new Document();
doc.add(new TextField("f", "val", Field.Store.NO));
w.addDocument(doc);
IndexReader r = DirectoryReader.open(w, true).getSequentialSubReaders()[0];
SegmentReader r = getOnlySegmentReader(DirectoryReader.open(w, true));
try {
_TestUtil.docs(random(), r, "f", new BytesRef("val"), null, null, false);
fail("should have failed to seek since terms index was not loaded.");

View File

@ -30,7 +30,6 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.ReaderUtil;
import org.apache.lucene.util.UnicodeUtil;
public class TestIndexWriterUnicode extends LuceneTestCase {
@ -316,12 +315,9 @@ public class TestIndexWriterUnicode extends LuceneTestCase {
IndexReader r = writer.getReader();
// Test each sub-segment
new ReaderUtil.Gather(r) {
@Override
protected void add(int base, AtomicReader r) throws IOException {
checkTermsOrder(r, allTerms, false);
}
}.run();
for (AtomicReaderContext ctx : r.getTopReaderContext().leaves()) {
checkTermsOrder(ctx.reader(), allTerms, false);
}
checkTermsOrder(r, allTerms, true);
// Test multi segment

View File

@ -339,13 +339,13 @@ public class TestParallelCompositeReader extends LuceneTestCase {
if (compositeComposite) {
rd1 = new MultiReader(DirectoryReader.open(dir1), DirectoryReader.open(dir1));
rd2 = new MultiReader(DirectoryReader.open(dir2), DirectoryReader.open(dir2));
assertEquals(2, rd1.getSequentialSubReaders().length);
assertEquals(2, rd2.getSequentialSubReaders().length);
assertEquals(2, rd1.getSequentialSubReaders().size());
assertEquals(2, rd2.getSequentialSubReaders().size());
} else {
rd1 = DirectoryReader.open(dir1);
rd2 = DirectoryReader.open(dir2);
assertEquals(3, rd1.getSequentialSubReaders().length);
assertEquals(3, rd2.getSequentialSubReaders().length);
assertEquals(3, rd1.getSequentialSubReaders().size());
assertEquals(3, rd2.getSequentialSubReaders().size());
}
ParallelCompositeReader pr = new ParallelCompositeReader(rd1, rd2);
return newSearcher(pr);

View File

@ -281,7 +281,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
}
private static void printDocs(DirectoryReader r) throws Throwable {
IndexReader[] subs = r.getSequentialSubReaders();
List<? extends AtomicReader> subs = r.getSequentialSubReaders();
for(IndexReader sub : subs) {
// TODO: improve this
Bits liveDocs = ((AtomicReader)sub).getLiveDocs();

View File

@ -742,7 +742,7 @@ public class TestTermsEnum extends LuceneTestCase {
w.forceMerge(1);
DirectoryReader r = w.getReader();
w.close();
AtomicReader sub = r.getSequentialSubReaders()[0];
AtomicReader sub = getOnlySegmentReader(r);
Terms terms = sub.fields().terms("field");
Automaton automaton = new RegExp(".*", RegExp.NONE).toAutomaton();
CompiledAutomaton ca = new CompiledAutomaton(automaton, false, false);

View File

@ -130,7 +130,7 @@ public class TestThreadedForceMerge extends LuceneTestCase {
OpenMode.APPEND).setMaxBufferedDocs(2));
DirectoryReader reader = DirectoryReader.open(directory);
assertEquals("reader=" + reader, 1, reader.getSequentialSubReaders().length);
assertEquals("reader=" + reader, 1, reader.getSequentialSubReaders().size());
assertEquals(expectedDocCount, reader.numDocs());
reader.close();
}

View File

@ -19,6 +19,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.EnumSet;
import java.util.List;
import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
@ -120,11 +121,11 @@ public class TestTypePromotion extends LuceneTestCase {
private void assertValues(TestType type, Directory dir, long[] values, Type[] sourceType)
throws CorruptIndexException, IOException {
DirectoryReader reader = DirectoryReader.open(dir);
assertEquals(1, reader.getSequentialSubReaders().length);
assertEquals(1, reader.getSequentialSubReaders().size());
IndexReaderContext topReaderContext = reader.getTopReaderContext();
AtomicReaderContext[] children = topReaderContext.leaves();
assertEquals(1, children.length);
DocValues docValues = children[0].reader().docValues("promote");
List<AtomicReaderContext> leaves = topReaderContext.leaves();
assertEquals(1, leaves.size());
DocValues docValues = leaves.get(0).reader().docValues("promote");
Source directSource = docValues.getDirectSource();
for (int i = 0; i < values.length; i++) {
int id = Integer.parseInt(reader.document(i).get("id"));
@ -372,10 +373,10 @@ public class TestTypePromotion extends LuceneTestCase {
writer.forceMerge(1);
writer.close();
DirectoryReader reader = DirectoryReader.open(dir);
assertEquals(1, reader.getSequentialSubReaders().length);
assertEquals(1, reader.getSequentialSubReaders().size());
IndexReaderContext topReaderContext = reader.getTopReaderContext();
AtomicReaderContext[] children = topReaderContext.leaves();
DocValues docValues = children[0].reader().docValues("promote");
List<AtomicReaderContext> leaves = topReaderContext.leaves();
DocValues docValues = leaves.get(0).reader().docValues("promote");
assertNotNull(docValues);
assertValues(TestType.Byte, dir, values, sourceType);
assertEquals(Type.BYTES_VAR_STRAIGHT, docValues.getType());

View File

@ -226,7 +226,7 @@ public class TestBooleanQuery extends LuceneTestCase {
Weight weight = s.createNormalizedWeight(q);
Scorer scorer = weight.scorer(s.leafContexts[0],
Scorer scorer = weight.scorer(s.leafContexts.get(0),
true, false, null);
// First pass: just use .nextDoc() to gather all hits
@ -244,7 +244,7 @@ public class TestBooleanQuery extends LuceneTestCase {
for(int iter2=0;iter2<10;iter2++) {
weight = s.createNormalizedWeight(q);
scorer = weight.scorer(s.leafContexts[0],
scorer = weight.scorer(s.leafContexts.get(0),
true, false, null);
if (VERBOSE) {

View File

@ -54,7 +54,7 @@ public class TestMatchAllDocsQuery extends LuceneTestCase {
IndexSearcher is = newSearcher(ir);
ScoreDoc[] hits;
hits = is.search(new MatchAllDocsQuery(), null, 1000).scoreDocs;
assertEquals(3, hits.length);
assertEquals("one", is.doc(hits[0].doc).get("key"));

View File

@ -311,13 +311,13 @@ public class TestShardSearching extends ShardSearchingTestBase {
final int numNodes = shardSearcher.nodeVersions.length;
int[] base = new int[numNodes];
final IndexReader[] subs = ((CompositeReader) mockSearcher.getIndexReader()).getSequentialSubReaders();
assertEquals(numNodes, subs.length);
final List<? extends IndexReader> subs = ((CompositeReader) mockSearcher.getIndexReader()).getSequentialSubReaders();
assertEquals(numNodes, subs.size());
int docCount = 0;
for(int nodeID=0;nodeID<numNodes;nodeID++) {
base[nodeID] = docCount;
docCount += subs[nodeID].maxDoc();
docCount += subs.get(nodeID).maxDoc();
}
if (VERBOSE) {

View File

@ -19,6 +19,7 @@ package org.apache.lucene.search;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.document.Document;
@ -39,11 +40,11 @@ import org.apache.lucene.util._TestUtil;
public class TestTopDocsMerge extends LuceneTestCase {
private static class ShardSearcher extends IndexSearcher {
private final AtomicReaderContext[] ctx;
private final List<AtomicReaderContext> ctx;
public ShardSearcher(AtomicReaderContext ctx, IndexReaderContext parent) {
super(parent);
this.ctx = new AtomicReaderContext[] {ctx};
this.ctx = Collections.singletonList(ctx);
}
public void search(Weight weight, Collector collector) throws IOException {
@ -56,7 +57,7 @@ public class TestTopDocsMerge extends LuceneTestCase {
@Override
public String toString() {
return "ShardSearcher(" + ctx[0] + ")";
return "ShardSearcher(" + ctx.get(0) + ")";
}
}
@ -131,13 +132,15 @@ public class TestTopDocsMerge extends LuceneTestCase {
docStarts[0] = 0;
} else {
final CompositeReaderContext compCTX = (CompositeReaderContext) ctx;
subSearchers = new ShardSearcher[compCTX.leaves().length];
docStarts = new int[compCTX.leaves().length];
final int size = compCTX.leaves().size();
subSearchers = new ShardSearcher[size];
docStarts = new int[size];
int docBase = 0;
for(int searcherIDX=0;searcherIDX<subSearchers.length;searcherIDX++) {
subSearchers[searcherIDX] = new ShardSearcher(compCTX.leaves()[searcherIDX], compCTX);
for(int searcherIDX=0;searcherIDX<subSearchers.length;searcherIDX++) {
final AtomicReaderContext leave = compCTX.leaves().get(searcherIDX);
subSearchers[searcherIDX] = new ShardSearcher(leave, compCTX);
docStarts[searcherIDX] = docBase;
docBase += compCTX.leaves()[searcherIDX].reader().maxDoc();
docBase += leave.reader().maxDoc();
}
}

View File

@ -21,11 +21,11 @@ import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.TreeSet;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.DocIdSetIterator;
@ -42,14 +42,16 @@ import org.apache.lucene.util.TermContext;
public class MultiSpansWrapper extends Spans { // can't be package private due to payloads
private SpanQuery query;
private AtomicReaderContext[] leaves;
private List<AtomicReaderContext> leaves;
private int leafOrd = 0;
private Spans current;
private Map<Term,TermContext> termContexts;
private final int numLeaves;
private MultiSpansWrapper(AtomicReaderContext[] leaves, SpanQuery query, Map<Term,TermContext> termContexts) {
private MultiSpansWrapper(List<AtomicReaderContext> leaves, SpanQuery query, Map<Term,TermContext> termContexts) {
this.query = query;
this.leaves = leaves;
this.numLeaves = leaves.size();
this.termContexts = termContexts;
}
@ -61,27 +63,30 @@ public class MultiSpansWrapper extends Spans { // can't be package private due t
for (Term term : terms) {
termContexts.put(term, TermContext.build(topLevelReaderContext, term, true));
}
AtomicReaderContext[] leaves = topLevelReaderContext.leaves();
if(leaves.length == 1) {
return query.getSpans(leaves[0], leaves[0].reader().getLiveDocs(), termContexts);
final List<AtomicReaderContext> leaves = topLevelReaderContext.leaves();
if(leaves.size() == 1) {
final AtomicReaderContext ctx = leaves.get(0);
return query.getSpans(ctx, ctx.reader().getLiveDocs(), termContexts);
}
return new MultiSpansWrapper(leaves, query, termContexts);
}
@Override
public boolean next() throws IOException {
if (leafOrd >= leaves.length) {
if (leafOrd >= numLeaves) {
return false;
}
if (current == null) {
current = query.getSpans(leaves[leafOrd], leaves[leafOrd].reader().getLiveDocs(), termContexts);
final AtomicReaderContext ctx = leaves.get(leafOrd);
current = query.getSpans(ctx, ctx.reader().getLiveDocs(), termContexts);
}
while(true) {
if (current.next()) {
return true;
}
if (++leafOrd < leaves.length) {
current = query.getSpans(leaves[leafOrd], leaves[leafOrd].reader().getLiveDocs(), termContexts);
if (++leafOrd < numLeaves) {
final AtomicReaderContext ctx = leaves.get(leafOrd);
current = query.getSpans(ctx, ctx.reader().getLiveDocs(), termContexts);
} else {
current = null;
break;
@ -92,27 +97,30 @@ public class MultiSpansWrapper extends Spans { // can't be package private due t
@Override
public boolean skipTo(int target) throws IOException {
if (leafOrd >= leaves.length) {
if (leafOrd >= numLeaves) {
return false;
}
int subIndex = ReaderUtil.subIndex(target, leaves);
assert subIndex >= leafOrd;
if (subIndex != leafOrd) {
current = query.getSpans(leaves[subIndex], leaves[subIndex].reader().getLiveDocs(), termContexts);
final AtomicReaderContext ctx = leaves.get(subIndex);
current = query.getSpans(ctx, ctx.reader().getLiveDocs(), termContexts);
leafOrd = subIndex;
} else if (current == null) {
current = query.getSpans(leaves[leafOrd], leaves[leafOrd].reader().getLiveDocs(), termContexts);
final AtomicReaderContext ctx = leaves.get(leafOrd);
current = query.getSpans(ctx, ctx.reader().getLiveDocs(), termContexts);
}
while (true) {
if (current.skipTo(target - leaves[leafOrd].docBase)) {
if (current.skipTo(target - leaves.get(leafOrd).docBase)) {
return true;
}
if (++leafOrd < leaves.length) {
current = query.getSpans(leaves[leafOrd], leaves[leafOrd].reader().getLiveDocs(), termContexts);
if (++leafOrd < numLeaves) {
final AtomicReaderContext ctx = leaves.get(leafOrd);
current = query.getSpans(ctx, ctx.reader().getLiveDocs(), termContexts);
} else {
current = null;
break;
current = null;
break;
}
}
@ -124,7 +132,7 @@ public class MultiSpansWrapper extends Spans { // can't be package private due t
if (current == null) {
return DocIdSetIterator.NO_MORE_DOCS;
}
return current.doc() + leaves[leafOrd].docBase;
return current.doc() + leaves.get(leafOrd).docBase;
}
@Override

View File

@ -166,8 +166,8 @@ public class TestNearSpansOrdered extends LuceneTestCase {
SpanNearQuery q = makeQuery();
Weight w = searcher.createNormalizedWeight(q);
IndexReaderContext topReaderContext = searcher.getTopReaderContext();
AtomicReaderContext[] leaves = topReaderContext.leaves();
Scorer s = w.scorer(leaves[0], true, false, leaves[0].reader().getLiveDocs());
AtomicReaderContext leave = topReaderContext.leaves().get(0);
Scorer s = w.scorer(leave, true, false, leave.reader().getLiveDocs());
assertEquals(1, s.advance(1));
}

View File

@ -18,6 +18,7 @@ package org.apache.lucene.search.spans;
*/
import java.io.IOException;
import java.util.List;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
@ -404,10 +405,10 @@ public class TestSpans extends LuceneTestCase {
boolean ordered = true;
int slop = 1;
IndexReaderContext topReaderContext = searcher.getTopReaderContext();
AtomicReaderContext[] leaves = topReaderContext.leaves();
List<AtomicReaderContext> leaves = topReaderContext.leaves();
int subIndex = ReaderUtil.subIndex(11, leaves);
for (int i = 0; i < leaves.length; i++) {
for (int i = 0, c = leaves.size(); i < c; i++) {
final AtomicReaderContext ctx = leaves.get(i);
final Similarity sim = new DefaultSimilarity() {
@Override
@ -427,13 +428,13 @@ public class TestSpans extends LuceneTestCase {
slop,
ordered);
spanScorer = searcher.createNormalizedWeight(snq).scorer(leaves[i], true, false, leaves[i].reader().getLiveDocs());
spanScorer = searcher.createNormalizedWeight(snq).scorer(ctx, true, false, ctx.reader().getLiveDocs());
} finally {
searcher.setSimilarity(oldSim);
}
if (i == subIndex) {
assertTrue("first doc", spanScorer.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals("first doc number", spanScorer.docID() + leaves[i].docBase, 11);
assertEquals("first doc number", spanScorer.docID() + ctx.docBase, 11);
float score = spanScorer.score();
assertTrue("first doc score should be zero, " + score, score == 0.0f);
} else {

View File

@ -629,25 +629,16 @@ public class TestGrouping extends LuceneTestCase {
public final int[] docStarts;
public ShardState(IndexSearcher s) {
List<AtomicReader> subReaders = new ArrayList<AtomicReader>();
ReaderUtil.gatherSubReaders(subReaders, s.getIndexReader());
subSearchers = new ShardSearcher[subReaders.size()];
final IndexReaderContext ctx = s.getTopReaderContext();
if (ctx instanceof AtomicReaderContext) {
assert subSearchers.length == 1;
subSearchers[0] = new ShardSearcher((AtomicReaderContext) ctx, ctx);
} else {
final CompositeReaderContext compCTX = (CompositeReaderContext) ctx;
for(int searcherIDX=0;searcherIDX<subSearchers.length;searcherIDX++) {
subSearchers[searcherIDX] = new ShardSearcher(compCTX.leaves()[searcherIDX], compCTX);
}
final List<AtomicReaderContext> leaves = ctx.leaves();
subSearchers = new ShardSearcher[leaves.size()];
for(int searcherIDX=0;searcherIDX<subSearchers.length;searcherIDX++) {
subSearchers[searcherIDX] = new ShardSearcher(leaves.get(searcherIDX), ctx);
}
docStarts = new int[subSearchers.length];
int docBase = 0;
for(int subIDX=0;subIDX<docStarts.length;subIDX++) {
docStarts[subIDX] = docBase;
docBase += subReaders.get(subIDX).maxDoc();
docStarts[subIDX] = leaves.get(subIDX).docBase;
//System.out.println("docStarts[" + subIDX + "]=" + docStarts[subIDX]);
}
}
@ -1315,24 +1306,20 @@ public class TestGrouping extends LuceneTestCase {
}
private static class ShardSearcher extends IndexSearcher {
private final AtomicReaderContext[] ctx;
private final List<AtomicReaderContext> ctx;
public ShardSearcher(AtomicReaderContext ctx, IndexReaderContext parent) {
super(parent);
this.ctx = new AtomicReaderContext[] {ctx};
this.ctx = Collections.singletonList(ctx);
}
public void search(Weight weight, Collector collector) throws IOException {
search(ctx, weight, collector);
}
public TopDocs search(Weight weight, int topN) throws IOException {
return search(ctx, weight, null, topN);
}
@Override
public String toString() {
return "ShardSearcher(" + ctx[0].reader() + ")";
return "ShardSearcher(" + ctx.get(0).reader() + ")";
}
}

View File

@ -247,9 +247,9 @@ public class TestBlockJoin extends LuceneTestCase {
}
private Document getParentDoc(IndexReader reader, Filter parents, int childDocID) throws IOException {
final AtomicReaderContext[] leaves = reader.getTopReaderContext().leaves();
final List<AtomicReaderContext> leaves = reader.getTopReaderContext().leaves();
final int subIndex = ReaderUtil.subIndex(childDocID, leaves);
final AtomicReaderContext leaf = leaves[subIndex];
final AtomicReaderContext leaf = leaves.get(subIndex);
final FixedBitSet bits = (FixedBitSet) parents.getDocIdSet(leaf, null);
return leaf.reader().document(bits.nextSetBit(childDocID - leaf.docBase));
}
@ -961,7 +961,7 @@ public class TestBlockJoin extends LuceneTestCase {
ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ScoreMode.Avg);
Weight weight = s.createNormalizedWeight(q);
DocIdSetIterator disi = weight.scorer(s.getIndexReader().getTopReaderContext().leaves()[0], true, true, null);
DocIdSetIterator disi = weight.scorer(s.getIndexReader().getTopReaderContext().leaves().get(0), true, true, null);
assertEquals(1, disi.advance(1));
r.close();
dir.close();
@ -995,7 +995,7 @@ public class TestBlockJoin extends LuceneTestCase {
ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ScoreMode.Avg);
Weight weight = s.createNormalizedWeight(q);
DocIdSetIterator disi = weight.scorer(s.getIndexReader().getTopReaderContext().leaves()[0], true, true, null);
DocIdSetIterator disi = weight.scorer(s.getIndexReader().getTopReaderContext().leaves().get(0), true, true, null);
assertEquals(2, disi.advance(0));
r.close();
dir.close();

View File

@ -20,13 +20,13 @@ package org.apache.lucene.index;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ReaderUtil;
import org.apache.lucene.util.Version;
/**
@ -101,7 +101,8 @@ public class MultiPassIndexSplitter {
.setOpenMode(OpenMode.CREATE));
System.err.println("Writing part " + (i + 1) + " ...");
// pass the subreaders directly, as our wrapper's numDocs/hasDeletetions are not up-to-date
w.addIndexes(input.getSequentialSubReaders());
final List<? extends FakeDeleteAtomicIndexReader> sr = input.getSequentialSubReaders();
w.addIndexes(sr.toArray(new IndexReader[sr.size()])); // TODO: maybe take List<IR> here?
w.close();
}
System.err.println("Done.");
@ -177,34 +178,36 @@ public class MultiPassIndexSplitter {
/**
* This class emulates deletions on the underlying index.
*/
private static final class FakeDeleteIndexReader extends MultiReader {
private static final class FakeDeleteIndexReader extends BaseCompositeReader<FakeDeleteAtomicIndexReader> {
public FakeDeleteIndexReader(IndexReader reader) throws IOException {
super(initSubReaders(reader));
}
private static AtomicReader[] initSubReaders(IndexReader reader) throws IOException {
final ArrayList<AtomicReader> subs = new ArrayList<AtomicReader>();
new ReaderUtil.Gather(reader) {
@Override
protected void add(int base, AtomicReader r) {
subs.add(new FakeDeleteAtomicIndexReader(r));
}
}.run();
return subs.toArray(new AtomicReader[subs.size()]);
private static FakeDeleteAtomicIndexReader[] initSubReaders(IndexReader reader) throws IOException {
final List<AtomicReaderContext> leaves = reader.getTopReaderContext().leaves();
final FakeDeleteAtomicIndexReader[] subs = new FakeDeleteAtomicIndexReader[leaves.size()];
int i = 0;
for (final AtomicReaderContext ctx : leaves) {
subs[i++] = new FakeDeleteAtomicIndexReader(ctx.reader());
}
return subs;
}
public void deleteDocument(int docID) {
final int i = readerIndex(docID);
((FakeDeleteAtomicIndexReader) subReaders[i]).deleteDocument(docID - starts[i]);
getSequentialSubReaders().get(i).deleteDocument(docID - readerBase(i));
}
public void undeleteAll() {
for (IndexReader r : subReaders) {
((FakeDeleteAtomicIndexReader) r).undeleteAll();
for (FakeDeleteAtomicIndexReader r : getSequentialSubReaders()) {
r.undeleteAll();
}
}
@Override
protected void doClose() throws IOException {}
// no need to override numDocs/hasDeletions,
// as we pass the subreaders directly to IW.addIndexes().
}

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.List;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.search.DocIdSet;
@ -101,10 +102,11 @@ public class PKIndexSplitter {
boolean success = false;
final IndexWriter w = new IndexWriter(target, config);
try {
final AtomicReaderContext[] leaves = reader.getTopReaderContext().leaves();
final IndexReader[] subReaders = new IndexReader[leaves.length];
for (int i = 0; i < leaves.length; i++) {
subReaders[i] = new DocumentFilteredAtomicIndexReader(leaves[i], preserveFilter, negateFilter);
final List<AtomicReaderContext> leaves = reader.getTopReaderContext().leaves();
final IndexReader[] subReaders = new IndexReader[leaves.size()];
int i = 0;
for (final AtomicReaderContext ctx : leaves) {
subReaders[i++] = new DocumentFilteredAtomicIndexReader(ctx, preserveFilter, negateFilter);
}
w.addIndexes(subReaders);
success = true;

View File

@ -18,6 +18,7 @@ package org.apache.lucene.misc;
*/
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.MultiFields;
@ -184,33 +185,29 @@ public class HighFreqTerms {
}
public static long getTotalTermFreq(IndexReader reader, final String field, final BytesRef termText) throws Exception {
final long totalTF[] = new long[1];
new ReaderUtil.Gather(reader) {
@Override
protected void add(int base, AtomicReader r) throws IOException {
Bits liveDocs = r.getLiveDocs();
if (liveDocs == null) {
// TODO: we could do this up front, during the scan
// (next()), instead of after-the-fact here w/ seek,
// if the codec supports it and there are no del
// docs...
final long totTF = r.totalTermFreq(field, termText);
if (totTF != -1) {
totalTF[0] += totTF;
return;
}
}
DocsEnum de = r.termDocsEnum(liveDocs, field, termText, true);
if (de != null) {
while (de.nextDoc() != DocIdSetIterator.NO_MORE_DOCS)
totalTF[0] += de.freq();
}
long totalTF = 0L;
for (final AtomicReaderContext ctx : reader.getTopReaderContext().leaves()) {
AtomicReader r = ctx.reader();
Bits liveDocs = r.getLiveDocs();
if (liveDocs == null) {
// TODO: we could do this up front, during the scan
// (next()), instead of after-the-fact here w/ seek,
// if the codec supports it and there are no del
// docs...
final long totTF = r.totalTermFreq(field, termText);
if (totTF != -1) {
totalTF += totTF;
continue;
} // otherwise we fall-through
}
}.run();
DocsEnum de = r.termDocsEnum(liveDocs, field, termText, true);
if (de != null) {
while (de.nextDoc() != DocIdSetIterator.NO_MORE_DOCS)
totalTF += de.freq();
}
}
return totalTF[0];
return totalTF;
}
}

View File

@ -67,7 +67,7 @@ public class TestBalancedSegmentMergePolicy extends LuceneTestCase {
int numSegments = _TestUtil.nextInt(random(), 1, 4);
iw.forceMerge(numSegments);
DirectoryReader ir = iw.getReader();
assertTrue(ir.getSequentialSubReaders().length <= numSegments);
assertTrue(ir.getSequentialSubReaders().size() <= numSegments);
ir.close();
}

View File

@ -59,7 +59,7 @@ public class TestIndexSplitter extends LuceneTestCase {
}
iw.commit();
DirectoryReader iwReader = iw.getReader();
assertEquals(3, iwReader.getSequentialSubReaders().length);
assertEquals(3, iwReader.getSequentialSubReaders().size());
iwReader.close();
iw.close();
// we should have 2 segments now
@ -87,7 +87,7 @@ public class TestIndexSplitter extends LuceneTestCase {
// now remove the copied segment from src
IndexSplitter.main(new String[] {dir.getAbsolutePath(), "-d", splitSegName});
r = DirectoryReader.open(fsDir);
assertEquals(2, r.getSequentialSubReaders().length);
assertEquals(2, r.getSequentialSubReaders().size());
r.close();
fsDir.close();
}

View File

@ -25,6 +25,7 @@ import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.util.ReaderUtil;
import java.io.IOException;
import java.util.List;
import java.util.Map;
/**
@ -60,7 +61,7 @@ public class ScaleFloatFunction extends ValueSource {
}
private ScaleInfo createScaleInfo(Map context, AtomicReaderContext readerContext) throws IOException {
final AtomicReaderContext[] leaves = ReaderUtil.getTopLevelContext(readerContext).leaves();
final List<AtomicReaderContext> leaves = ReaderUtil.getTopLevelContext(readerContext).leaves();
float minVal = Float.POSITIVE_INFINITY;
float maxVal = Float.NEGATIVE_INFINITY;

View File

@ -44,6 +44,7 @@ import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.util.List;
public class TestParser extends LuceneTestCase {
@ -194,8 +195,8 @@ public class TestParser extends LuceneTestCase {
}
public void testDuplicateFilterQueryXML() throws ParserException, IOException {
AtomicReaderContext leaves[] = searcher.getTopReaderContext().leaves();
Assume.assumeTrue(leaves == null || leaves.length == 1);
List<AtomicReaderContext> leaves = searcher.getTopReaderContext().leaves();
Assume.assumeTrue(leaves.size() == 1);
Query q = parse("DuplicateFilterQuery.xml");
int h = searcher.search(q, null, 1000).totalHits;
assertEquals("DuplicateFilterQuery should produce 1 result ", 1, h);

View File

@ -28,6 +28,7 @@ import org.apache.lucene.document.FieldType;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
@ -497,14 +498,11 @@ public class SpellChecker implements java.io.Closeable {
final IndexReader reader = searcher.getIndexReader();
if (reader.maxDoc() > 0) {
new ReaderUtil.Gather(reader) {
@Override
protected void add(int base, AtomicReader r) throws IOException {
Terms terms = r.terms(F_WORD);
if (terms != null)
termsEnums.add(terms.iterator(null));
}
}.run();
for (final AtomicReaderContext ctx : reader.getTopReaderContext().leaves()) {
Terms terms = ctx.reader().terms(F_WORD);
if (terms != null)
termsEnums.add(terms.iterator(null));
}
}
boolean isEmpty = termsEnums.isEmpty();

View File

@ -18,6 +18,7 @@ package org.apache.lucene.search;
*/
import java.io.IOException;
import java.util.List;
import java.util.Random;
import junit.framework.Assert;
@ -233,7 +234,7 @@ public class QueryUtils {
*/
public static void checkSkipTo(final Query q, final IndexSearcher s) throws IOException {
//System.out.println("Checking "+q);
final AtomicReaderContext[] readerContextArray = s.getTopReaderContext().leaves();
final List<AtomicReaderContext> readerContextArray = s.getTopReaderContext().leaves();
if (s.createNormalizedWeight(q).scoresDocsOutOfOrder()) return; // in this case order of skipTo() might differ from that of next().
final int skip_op = 0;
@ -278,7 +279,7 @@ public class QueryUtils {
try {
if (scorer == null) {
Weight w = s.createNormalizedWeight(q);
AtomicReaderContext context = readerContextArray[leafPtr];
AtomicReaderContext context = readerContextArray.get(leafPtr);
scorer = w.scorer(context, true, false, context.reader().getLiveDocs());
}
@ -334,7 +335,7 @@ public class QueryUtils {
leafPtr++;
}
lastReader[0] = context.reader();
assert readerContextArray[leafPtr].reader() == context.reader();
assert readerContextArray.get(leafPtr).reader() == context.reader();
this.scorer = null;
lastDoc[0] = -1;
}
@ -368,7 +369,7 @@ public class QueryUtils {
final float maxDiff = 1e-3f;
final int lastDoc[] = {-1};
final AtomicReader lastReader[] = {null};
final AtomicReaderContext[] context = s.getTopReaderContext().leaves();
final List<AtomicReaderContext> context = s.getTopReaderContext().leaves();
s.search(q,new Collector() {
private Scorer scorer;
private int leafPtr;
@ -384,7 +385,7 @@ public class QueryUtils {
long startMS = System.currentTimeMillis();
for (int i=lastDoc[0]+1; i<=doc; i++) {
Weight w = s.createNormalizedWeight(q);
Scorer scorer = w.scorer(context[leafPtr], true, false, liveDocs);
Scorer scorer = w.scorer(context.get(leafPtr), true, false, liveDocs);
Assert.assertTrue("query collected "+doc+" but skipTo("+i+") says no more docs!",scorer.advance(i) != DocIdSetIterator.NO_MORE_DOCS);
Assert.assertEquals("query collected "+doc+" but skipTo("+i+") got to "+scorer.docID(),doc,scorer.docID());
float skipToScore = scorer.score();

View File

@ -435,11 +435,12 @@ public abstract class LuceneTestCase extends Assert {
* do tests on that segment's reader. This is an utility method to help them.
*/
public static SegmentReader getOnlySegmentReader(DirectoryReader reader) {
IndexReader[] subReaders = reader.getSequentialSubReaders();
if (subReaders.length != 1)
throw new IllegalArgumentException(reader + " has " + subReaders.length + " segments instead of exactly one");
assertTrue(subReaders[0] instanceof SegmentReader);
return (SegmentReader) subReaders[0];
List<? extends IndexReader> subReaders = reader.getSequentialSubReaders();
if (subReaders.size() != 1)
throw new IllegalArgumentException(reader + " has " + subReaders.size() + " segments instead of exactly one");
final IndexReader r = subReaders.get(0);
assertTrue(r instanceof SegmentReader);
return (SegmentReader) r;
}
/**

View File

@ -544,7 +544,7 @@ public class LukeRequestHandler extends RequestHandlerBase
indexInfo.add("maxDoc", reader.maxDoc());
indexInfo.add("version", reader.getVersion()); // TODO? Is this different then: IndexReader.getCurrentVersion( dir )?
indexInfo.add("segmentCount", reader.getSequentialSubReaders().length);
indexInfo.add("segmentCount", reader.getTopReaderContext().leaves().size());
indexInfo.add("current", reader.isCurrent() );
indexInfo.add("hasDeletions", reader.hasDeletions() );
indexInfo.add("directory", dir );

View File

@ -437,11 +437,11 @@ public class QueryComponent extends SearchComponent
NamedList<Object[]> sortVals = new NamedList<Object[]>(); // order is important for the sort fields
Field field = new StringField("dummy", "", Field.Store.NO); // a dummy Field
IndexReaderContext topReaderContext = searcher.getTopReaderContext();
AtomicReaderContext[] leaves = topReaderContext.leaves();
List<AtomicReaderContext> leaves = topReaderContext.leaves();
AtomicReaderContext currentLeaf = null;
if (leaves.length==1) {
if (leaves.size()==1) {
// if there is a single segment, use that subReader and avoid looking up each time
currentLeaf = leaves[0];
currentLeaf = leaves.get(0);
leaves=null;
}
@ -478,7 +478,7 @@ public class QueryComponent extends SearchComponent
if (leaves != null) {
idx = ReaderUtil.subIndex(doc, leaves);
currentLeaf = leaves[idx];
currentLeaf = leaves.get(idx);
if (idx != lastIdx) {
// we switched segments. invalidate comparator.
comparator = null;

View File

@ -679,7 +679,7 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
if (buildOnCommit) {
buildSpellIndex(newSearcher);
} else if (buildOnOptimize) {
if (newSearcher.getIndexReader().getSequentialSubReaders().length == 1) {
if (newSearcher.getIndexReader().getSequentialSubReaders().size() == 1) {
buildSpellIndex(newSearcher);
} else {
LOG.info("Index is not optimized therefore skipping building spell check index for: " + checker.getDictionaryName());

View File

@ -82,7 +82,7 @@ class PerSegmentSingleValuedFaceting {
// reuse the translation logic to go from top level set to per-segment set
baseSet = docs.getTopFilter();
final AtomicReaderContext[] leaves = searcher.getTopReaderContext().leaves();
final List<AtomicReaderContext> leaves = searcher.getTopReaderContext().leaves();
// The list of pending tasks that aren't immediately submitted
// TODO: Is there a completion service, or a delegating executor that can
// limit the number of concurrent tasks submitted to a bigger executor?
@ -90,8 +90,8 @@ class PerSegmentSingleValuedFaceting {
int threads = nThreads <= 0 ? Integer.MAX_VALUE : nThreads;
for (int i=0; i<leaves.length; i++) {
final SegFacet segFacet = new SegFacet(leaves[i]);
for (final AtomicReaderContext leave : leaves) {
final SegFacet segFacet = new SegFacet(leave);
Callable<SegFacet> task = new Callable<SegFacet>() {
public SegFacet call() throws Exception {
@ -111,7 +111,7 @@ class PerSegmentSingleValuedFaceting {
// now merge the per-segment results
PriorityQueue<SegFacet> queue = new PriorityQueue<SegFacet>(leaves.length) {
PriorityQueue<SegFacet> queue = new PriorityQueue<SegFacet>(leaves.size()) {
@Override
protected boolean lessThan(SegFacet a, SegFacet b) {
return a.tempBR.compareTo(b.tempBR) < 0;
@ -121,7 +121,7 @@ class PerSegmentSingleValuedFaceting {
boolean hasMissingCount=false;
int missingCount=0;
for (int i=0; i<leaves.length; i++) {
for (int i=0, c=leaves.size(); i<c; i++) {
SegFacet seg = null;
try {

View File

@ -17,6 +17,7 @@
package org.apache.solr.response.transform;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import org.apache.lucene.index.AtomicReaderContext;
@ -26,7 +27,6 @@ import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.util.ReaderUtil;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrException;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.search.QParser;
import org.apache.solr.search.SolrIndexSearcher;
@ -64,7 +64,7 @@ public class ValueSourceAugmenter extends DocTransformer
try {
IndexReader reader = qparser.getReq().getSearcher().getIndexReader();
readerContexts = reader.getTopReaderContext().leaves();
docValuesArr = new FunctionValues[readerContexts.length];
docValuesArr = new FunctionValues[readerContexts.size()];
searcher = qparser.getReq().getSearcher();
fcontext = ValueSource.newContext(searcher);
@ -77,7 +77,7 @@ public class ValueSourceAugmenter extends DocTransformer
Map fcontext;
SolrIndexSearcher searcher;
AtomicReaderContext[] readerContexts;
List<AtomicReaderContext> readerContexts;
FunctionValues docValuesArr[];
@ -89,7 +89,7 @@ public class ValueSourceAugmenter extends DocTransformer
// TODO: calculate this stuff just once across diff functions
int idx = ReaderUtil.subIndex(docid, readerContexts);
AtomicReaderContext rcontext = readerContexts[idx];
AtomicReaderContext rcontext = readerContexts.get(idx);
FunctionValues values = docValuesArr[idx];
if (values == null) {
docValuesArr[idx] = values = valueSource.getValues(fcontext, rcontext);

View File

@ -602,11 +602,9 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
*/
public long lookupId(BytesRef idBytes) throws IOException {
String field = schema.getUniqueKeyField().getName();
final AtomicReaderContext[] leaves = leafContexts;
for (int i=0; i<leaves.length; i++) {
final AtomicReaderContext leaf = leaves[i];
for (int i=0, c=leafContexts.size(); i<c; i++) {
final AtomicReaderContext leaf = leafContexts.get(i);
final AtomicReader reader = leaf.reader();
final Fields fields = reader.fields();
@ -756,11 +754,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
collector = pf.postFilter;
}
final AtomicReaderContext[] leaves = leafContexts;
for (int i=0; i<leaves.length; i++) {
final AtomicReaderContext leaf = leaves[i];
for (final AtomicReaderContext leaf : leafContexts) {
final AtomicReader reader = leaf.reader();
final Bits liveDocs = reader.getLiveDocs(); // TODO: the filter may already only have liveDocs...
DocIdSet idSet = null;
@ -989,10 +983,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
if (filter==null) {
if (query instanceof TermQuery) {
Term t = ((TermQuery)query).getTerm();
final AtomicReaderContext[] leaves = leafContexts;
for (int i=0; i<leaves.length; i++) {
final AtomicReaderContext leaf = leaves[i];
for (final AtomicReaderContext leaf : leafContexts) {
final AtomicReader reader = leaf.reader();
collector.setNextReader(leaf);
Fields fields = reader.fields();
@ -1799,7 +1790,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
while (iter.hasNext()) {
int doc = iter.nextDoc();
while (doc>=end) {
AtomicReaderContext leaf = leafContexts[readerIndex++];
AtomicReaderContext leaf = leafContexts.get(readerIndex++);
base = leaf.docBase;
end = base + leaf.reader().maxDoc();
topCollector.setNextReader(leaf);

View File

@ -159,7 +159,7 @@ public class VersionInfo {
ValueSource vs = versionField.getType().getValueSource(versionField, null);
Map context = ValueSource.newContext(searcher);
vs.createWeight(context, searcher);
FunctionValues fv = vs.getValues(context, searcher.getTopReaderContext().leaves()[(int)(lookup>>32)]);
FunctionValues fv = vs.getValues(context, searcher.getTopReaderContext().leaves().get((int)(lookup>>32)));
long ver = fv.longVal((int)lookup);
return ver;

View File

@ -19,6 +19,7 @@ package org.apache.solr.search;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Random;
import org.apache.lucene.index.FieldInfo;
@ -471,7 +472,7 @@ public class TestDocSet extends LuceneTestCase {
DocIdSet da;
DocIdSet db;
AtomicReaderContext[] leaves = topLevelContext.leaves();
List<AtomicReaderContext> leaves = topLevelContext.leaves();
// first test in-sequence sub readers
for (AtomicReaderContext readerContext : leaves) {
@ -480,10 +481,10 @@ public class TestDocSet extends LuceneTestCase {
doTestIteratorEqual(da, db);
}
int nReaders = leaves.length;
int nReaders = leaves.size();
// now test out-of-sequence sub readers
for (int i=0; i<nReaders; i++) {
AtomicReaderContext readerContext = leaves[rand.nextInt(nReaders)];
AtomicReaderContext readerContext = leaves.get(rand.nextInt(nReaders));
da = fa.getDocIdSet(readerContext, null);
db = fb.getDocIdSet(readerContext, null);
doTestIteratorEqual(da, db);

View File

@ -26,6 +26,7 @@ import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.schema.SchemaField;
import org.junit.BeforeClass;
import java.util.List;
import java.util.Map;
import java.io.IOException;
@ -50,9 +51,9 @@ public class TestIndexSearcher extends SolrTestCaseJ4 {
Map context = ValueSource.newContext(sqr.getSearcher());
vs.createWeight(context, sqr.getSearcher());
IndexReaderContext topReaderContext = sqr.getSearcher().getTopReaderContext();
AtomicReaderContext[] leaves = topReaderContext.leaves();
List<AtomicReaderContext> leaves = topReaderContext.leaves();
int idx = ReaderUtil.subIndex(doc, leaves);
AtomicReaderContext leaf = leaves[idx];
AtomicReaderContext leaf = leaves.get(idx);
FunctionValues vals = vs.getValues(context, leaf);
return vals.strVal(doc-leaf.docBase);
}
@ -78,7 +79,7 @@ public class TestIndexSearcher extends SolrTestCaseJ4 {
// make sure the readers share the first segment
// Didn't work w/ older versions of lucene2.9 going from segment -> multi
assertEquals(rCtx1.leaves()[0].reader(), rCtx2.leaves()[0].reader());
assertEquals(rCtx1.leaves().get(0).reader(), rCtx2.leaves().get(0).reader());
assertU(adoc("id","5", "v_f","3.14159"));
assertU(adoc("id","6", "v_f","8983", "v_s1","string6"));
@ -88,8 +89,8 @@ public class TestIndexSearcher extends SolrTestCaseJ4 {
IndexReaderContext rCtx3 = sr3.getSearcher().getTopReaderContext();
// make sure the readers share segments
// assertEquals(r1.getLeafReaders()[0], r3.getLeafReaders()[0]);
assertEquals(rCtx2.leaves()[0].reader(), rCtx3.leaves()[0].reader());
assertEquals(rCtx2.leaves()[1].reader(), rCtx3.leaves()[1].reader());
assertEquals(rCtx2.leaves().get(0).reader(), rCtx3.leaves().get(0).reader());
assertEquals(rCtx2.leaves().get(1).reader(), rCtx3.leaves().get(1).reader());
sr1.close();
sr2.close();
@ -123,8 +124,8 @@ public class TestIndexSearcher extends SolrTestCaseJ4 {
assertU(commit());
SolrQueryRequest sr6 = req("q","foo");
IndexReaderContext rCtx6 = sr6.getSearcher().getTopReaderContext();
assertEquals(1, rCtx6.leaves()[0].reader().numDocs()); // only a single doc left in the first segment
assertTrue( !rCtx5.leaves()[0].reader().equals(rCtx6.leaves()[0].reader()) ); // readers now different
assertEquals(1, rCtx6.leaves().get(0).reader().numDocs()); // only a single doc left in the first segment
assertTrue( !rCtx5.leaves().get(0).reader().equals(rCtx6.leaves().get(0).reader()) ); // readers now different
sr5.close();
sr6.close();

View File

@ -198,7 +198,7 @@ public class TestSort extends SolrTestCaseJ4 {
DirectoryReader reader = DirectoryReader.open(dir);
IndexSearcher searcher = new IndexSearcher(reader);
// System.out.println("segments="+searcher.getIndexReader().getSequentialSubReaders().length);
assertTrue(reader.getSequentialSubReaders().length > 1);
assertTrue(reader.getSequentialSubReaders().size() > 1);
for (int i=0; i<qiter; i++) {
Filter filt = new Filter() {