Index: lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java (revision 1351132) +++ lucene/core/src/java/org/apache/lucene/index/BaseCompositeReader.java (working copy) @@ -18,6 +18,9 @@ */ 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 @@ * @lucene.internal */ public abstract class BaseCompositeReader 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 subReadersList; + /** * Constructs a {@code BaseCompositeReader} on the given subReaders. * @param subReaders the wrapped sub-readers. This array is returned by @@ -63,6 +70,7 @@ */ 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 @@ 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 getSequentialSubReaders() { + return subReadersList; } } Index: lucene/core/src/java/org/apache/lucene/index/CompositeReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/CompositeReader.java (revision 1351132) +++ lucene/core/src/java/org/apache/lucene/index/CompositeReader.java (working copy) @@ -17,6 +17,8 @@ * limitations under the License. */ +import java.util.List; + import org.apache.lucene.search.SearcherManager; // javadocs import org.apache.lucene.store.*; @@ -63,12 +65,12 @@ final StringBuilder buffer = new StringBuilder(); buffer.append(getClass().getSimpleName()); buffer.append('('); - final IndexReader[] subReaders = getSequentialSubReaders(); + final List 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(')'); @@ -85,7 +87,7 @@ * Doing so will corrupt the internal structure of this * {@code CompositeReader}. */ - public abstract IndexReader[] getSequentialSubReaders(); + public abstract List getSequentialSubReaders(); @Override public final CompositeReaderContext getTopReaderContext() { Index: lucene/core/src/java/org/apache/lucene/index/CompositeReaderContext.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/CompositeReaderContext.java (revision 1351132) +++ lucene/core/src/java/org/apache/lucene/index/CompositeReaderContext.java (working copy) @@ -17,8 +17,8 @@ * limitations under the License. */ -import java.io.IOException; -import org.apache.lucene.util.ReaderUtil; +import java.util.ArrayList; +import java.util.List; /** * {@link IndexReaderContext} for {@link CompositeReader} instance. @@ -26,7 +26,7 @@ */ public final class CompositeReaderContext extends IndexReaderContext { private final IndexReaderContext[] children; - private final AtomicReaderContext[] leaves; + private final List leaves; private final CompositeReader reader; static CompositeReaderContext create(CompositeReader reader) { @@ -45,22 +45,26 @@ /** * Creates a {@link CompositeReaderContext} for top-level readers with parent set to null */ - CompositeReaderContext(CompositeReader reader, IndexReaderContext[] children, AtomicReaderContext[] leaves) { + CompositeReaderContext(CompositeReader reader, IndexReaderContext[] children, List leaves) { this(null, reader, 0, 0, children, leaves); } private CompositeReaderContext(CompositeReaderContext parent, CompositeReader reader, int ordInParent, int docbaseInParent, IndexReaderContext[] children, - AtomicReaderContext[] leaves) { + List leaves) { super(parent, ordInParent, docbaseInParent); this.children = children; this.leaves = leaves; this.reader = reader; } + // nocommit: Make it return List, this is slow! + private static final AtomicReaderContext[] EMPTY = new AtomicReaderContext[0]; + @Override public AtomicReaderContext[] leaves() { - return leaves; + // nocommit: Make it return List, this is slow: + return leaves.toArray(EMPTY); } @@ -76,13 +80,11 @@ private static final class Builder { private final CompositeReader reader; - private final AtomicReaderContext[] leaves; - private int leafOrd = 0; + private final List leaves = new ArrayList(); private int leafDocBase = 0; public Builder(CompositeReader reader) { this.reader = reader; - leaves = new AtomicReaderContext[numLeaves(reader)]; } public CompositeReaderContext build() { @@ -92,14 +94,14 @@ 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 sequentialSubReaders = cr.getSequentialSubReaders(); + final IndexReaderContext[] children = new IndexReaderContext[sequentialSubReaders.size()]; final CompositeReaderContext newParent; if (parent == null) { newParent = new CompositeReaderContext(cr, children, leaves); @@ -107,31 +109,15 @@ 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[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]; - } - } } \ No newline at end of file Index: lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java (revision 1351132) +++ lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java (working copy) @@ -22,7 +22,6 @@ 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; Index: lucene/core/src/java/org/apache/lucene/index/MultiDocsAndPositionsEnum.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/MultiDocsAndPositionsEnum.java (revision 1351132) +++ lucene/core/src/java/org/apache/lucene/index/MultiDocsAndPositionsEnum.java (working copy) @@ -17,8 +17,8 @@ * 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 @@ // TODO: implement bulk read more efficiently than super public final static class EnumWithSlice { public DocsAndPositionsEnum docsAndPositionsEnum; - public ReaderUtil.Slice slice; + public ReaderSlice slice; } } Index: lucene/core/src/java/org/apache/lucene/index/MultiDocsEnum.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/MultiDocsEnum.java (revision 1351132) +++ lucene/core/src/java/org/apache/lucene/index/MultiDocsEnum.java (working copy) @@ -17,7 +17,8 @@ * 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 @@ // 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() { Index: lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java (revision 1351132) +++ lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java (working copy) @@ -29,7 +29,6 @@ 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; /** @@ -69,6 +68,8 @@ } private static class DocValuesPuller { + public DocValuesPuller() {} + public DocValues pull(AtomicReader reader, String field) throws IOException { return reader.docValues(field); } @@ -114,18 +115,18 @@ } - 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(); + assert reader instanceof CompositeReader; + final AtomicReaderContext[] subs = reader.getTopReaderContext().leaves(); if (subs.length == 0) { // no fields return null; } else if (subs.length == 1) { - return getDocValues(subs[0], field, puller); + return getDocValues(subs[0].reader(), field, puller); } else { final List slices = new ArrayList(); @@ -134,25 +135,23 @@ // 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; - } - final DocValues d = puller.pull(r, field); - if (d != null) { - TypePromoter incoming = TypePromoter.create(d.getType(), d.getValueSize()); - promotedType[0] = promotedType[0].promote(incoming); - } else if (puller.stopLoadingOnNull(r, field)){ - promotedType[0] = TypePromoter.getIdentityPromoter(); // set to identity to return null - stop = true; - } - slices.add(new DocValuesSlice(d, base, r.maxDoc())); + + boolean stop = false; + for (AtomicReaderContext ctx : subs) { + if (stop) { + break; } - }.run(); + 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); + } else if (puller.stopLoadingOnNull(r, field)){ + promotedType[0] = TypePromoter.getIdentityPromoter(); // set to identity to return null + stop = true; + } + slices.add(new DocValuesSlice(d, ctx.docBase, r.maxDoc())); + } // return null if no docvalues encountered anywhere if (promotedType[0] == TypePromoter.getIdentityPromoter()) { Index: lucene/core/src/java/org/apache/lucene/index/MultiFields.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/MultiFields.java (revision 1351132) +++ lucene/core/src/java/org/apache/lucene/index/MultiFields.java (working copy) @@ -28,8 +28,7 @@ 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. @@ -39,8 +38,9 @@ * MultiReader}). * *

NOTE: for multi readers, you'll get better - * performance by gathering the sub readers using {@link - * ReaderUtil#gatherSubReaders} and then operate per-reader, + * 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 @@ public final class MultiFields extends Fields { private final Fields[] subs; - private final ReaderUtil.Slice[] subSlices; + private final ReaderSlice[] subSlices; private final Map terms = new ConcurrentHashMap(); /** Returns a single {@link Fields} instance for this @@ -57,72 +57,54 @@ * has no postings. * *

NOTE: 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) { + public static Fields getFields(IndexReader reader) throws IOException { + final AtomicReaderContext[] subs = reader.getTopReaderContext().leaves(); + if (subs.length == 1) { + // already an atomic reader / reader with one leave + return subs[0].reader().fields(); + } else if (subs.length == 0) { // no fields return null; } else { final List fields = new ArrayList(); - final List slices = new ArrayList(); - - new ReaderUtil.Gather(r) { - @Override - protected void add(int base, AtomicReader r) throws IOException { - final Fields f = r.fields(); - if (f != null) { - fields.add(f); - slices.add(new ReaderUtil.Slice(base, r.maxDoc(), fields.size()-1)); - } + final List slices = new ArrayList(); + for (AtomicReaderContext ctx : subs) { + final AtomicReader r = ctx.reader(); + final Fields f = r.fields(); + if (f != null) { + fields.add(f); + 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)); + slices.toArray(ReaderSlice.EMPTY_ARRAY)); } } } - public static Bits getLiveDocs(IndexReader r) { - if (r.hasDeletions()) { - final List liveDocs = new ArrayList(); - final List starts = new ArrayList(); - - 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 AtomicReaderContext[] leaves = reader.getTopReaderContext().leaves(); + assert leaves.length > 0 : "A reader with deletions must have at least one leave"; + if (leaves.length == 1) { + return leaves[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[leaves.length]; + final int[] starts = new int[leaves.length + 1]; + for (int i = 0; i < leaves.length; i++) { + // record all liveDocs, even if they are null + liveDocs[i] = leaves[i].reader().getLiveDocs(); + starts[i] = leaves[i].docBase; } + starts[leaves.length] = reader.maxDoc(); + return new MultiBits(liveDocs, starts, true); } else { return null; } @@ -170,7 +152,7 @@ return null; } - public MultiFields(Fields[] subs, ReaderUtil.Slice[] subSlices) { + public MultiFields(Fields[] subs, ReaderSlice[] subSlices) { this.subs = subs; this.subSlices = subSlices; } @@ -179,7 +161,7 @@ public FieldsEnum iterator() throws IOException { final List fieldsEnums = new ArrayList(); - final List fieldsSlices = new ArrayList(); + final List fieldsSlices = new ArrayList(); for(int i=0;i subs2 = new ArrayList(); - final List slices2 = new ArrayList(); + final List slices2 = new ArrayList(); // Gather all sub-readers that share this field for(int i=0;i subReaders = new ArrayList(); - ReaderUtil.gatherSubReaders(subReaders, reader); final FieldInfos.Builder builder = new FieldInfos.Builder(); - for(AtomicReader subReader : subReaders) { - builder.add(subReader.getFieldInfos()); + for(AtomicReaderContext ctx : reader.getTopReaderContext().leaves()) { + builder.add(ctx.reader().getFieldInfos()); } return builder.finish(); } Index: lucene/core/src/java/org/apache/lucene/index/MultiFieldsEnum.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/MultiFieldsEnum.java (revision 1351132) +++ lucene/core/src/java/org/apache/lucene/index/MultiFieldsEnum.java (working copy) @@ -18,7 +18,7 @@ */ 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 @@ /** 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 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; Index: lucene/core/src/java/org/apache/lucene/index/MultiReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/MultiReader.java (revision 1351132) +++ lucene/core/src/java/org/apache/lucene/index/MultiReader.java (working copy) @@ -68,12 +68,12 @@ @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; Index: lucene/core/src/java/org/apache/lucene/index/MultiTerms.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/MultiTerms.java (revision 1351132) +++ lucene/core/src/java/org/apache/lucene/index/MultiTerms.java (working copy) @@ -23,7 +23,7 @@ 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 @@ public final class MultiTerms extends Terms { private final Terms[] subs; - private final ReaderUtil.Slice[] subSlices; + private final ReaderSlice[] subSlices; private final Comparator termComp; - public MultiTerms(Terms[] subs, ReaderUtil.Slice[] subSlices) throws IOException { + public MultiTerms(Terms[] subs, ReaderSlice[] subSlices) throws IOException { this.subs = subs; this.subSlices = subSlices; Index: lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java (revision 1351132) +++ lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java (working copy) @@ -22,7 +22,7 @@ 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 @@ 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 @@ } 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; Index: lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java (revision 1351132) +++ lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java (working copy) @@ -21,6 +21,7 @@ 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 @@ 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 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 @@ 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 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)"); } } Index: lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (revision 1351132) +++ lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (working copy) @@ -35,6 +35,7 @@ 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,10 @@ * @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); + final AtomicReaderContext[] leaves = reader.getTopReaderContext().leaves(); + for (final AtomicReaderContext ctx : leaves) { + final AtomicReader r = ctx.reader(); + mergeState.readers.add(new MergeState.IndexReaderAndLiveDocs(r, r.getLiveDocs(), r.numDeletedDocs())); } } @@ -311,7 +306,7 @@ private final void mergeTerms(SegmentWriteState segmentWriteState) throws CorruptIndexException, IOException { final List fields = new ArrayList(); - final List slices = new ArrayList(); + final List slices = new ArrayList(); int docBase = 0; @@ -320,7 +315,7 @@ 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 +326,7 @@ 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) { Index: lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java =================================================================== --- lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java (revision 1351132) +++ lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java (working copy) @@ -118,8 +118,8 @@ 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 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 @@ 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 @@ 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 @@ if (writer != null) { buffer.append(":nrt"); } - for(int i=0;i subs = ((CompositeReader)obj).getSequentialSubReaders(); + for (int j = 0; (null != subs) && (j < subs.size()); j++) { + all.add(subs.get(j).getCoreCacheKey()); } } Index: lucene/core/src/java/org/apache/lucene/util/MultiBits.java =================================================================== --- lucene/core/src/java/org/apache/lucene/util/MultiBits.java (revision 1351132) +++ lucene/core/src/java/org/apache/lucene/util/MultiBits.java (working copy) @@ -17,8 +17,6 @@ * limitations under the License. */ -import java.util.List; - /** * Concatenates multiple Bits together, on every lookup. * @@ -36,13 +34,10 @@ private final boolean defaultValue; - public MultiBits(List bits, List 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 enums = new IdentityHashMap(); - 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 enums = new IdentityHashMap(); + 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 @@ 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 enums = new IdentityHashMap(); MatchNoBits bits = new Bits.MatchNoBits(open.maxDoc()); @@ -125,8 +121,8 @@ DirectoryReader firstReader = DirectoryReader.open(dir); DirectoryReader secondReader = DirectoryReader.open(dir); - IndexReader[] sequentialSubReaders = firstReader.getSequentialSubReaders(); - IndexReader[] sequentialSubReaders2 = secondReader.getSequentialSubReaders(); + List sequentialSubReaders = firstReader.getSequentialSubReaders(); + List sequentialSubReaders2 = secondReader.getSequentialSubReaders(); for (IndexReader indexReader : sequentialSubReaders) { Terms terms = ((AtomicReader) indexReader).terms("body"); @@ -154,11 +150,11 @@ 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 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()); } Index: lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java (revision 1351132) +++ lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java (working copy) @@ -67,7 +67,7 @@ public void onCommit(List 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 @@ 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 @@ 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 @@ 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 @@ 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 @@ // 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 @@ // 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(); Index: lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java (revision 1351132) +++ lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java (working copy) @@ -550,7 +550,7 @@ 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 @@ 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 @@ assertNotNull(r2); r.close(); - IndexReader[] subs = r2.getSequentialSubReaders(); - for(int i=0;i subs = r2.getSequentialSubReaders(); + for(AtomicReader s : subs) { + assertEquals(36, s.getUniqueTermCount()); } r2.close(); writer.close(); @@ -842,7 +842,7 @@ // 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 @@ assertNotNull(r2); assertNull(DirectoryReader.openIfChanged(r2)); r.close(); - IndexReader[] subReaders = r2.getSequentialSubReaders(); - assertEquals(2, subReaders.length); - for(int i=0;i<2;i++) { + List 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 Index: lucene/core/src/test/org/apache/lucene/index/TestDirectoryReaderReopen.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestDirectoryReaderReopen.java (revision 1351132) +++ lucene/core/src/test/org/apache/lucene/index/TestDirectoryReaderReopen.java (working copy) @@ -469,9 +469,9 @@ 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 @@ } if (checkSubReaders && reader instanceof CompositeReader) { - IndexReader[] subReaders = ((CompositeReader) reader).getSequentialSubReaders(); - for (int i = 0; i < subReaders.length; i++) { - assertReaderClosed(subReaders[i], checkSubReaders, checkNormsClosed); + List subReaders = ((CompositeReader) reader).getSequentialSubReaders(); + for (IndexReader r : subReaders) { + assertReaderClosed(r, checkSubReaders, checkNormsClosed); } } } Index: lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java (revision 1351132) +++ lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java (working copy) @@ -77,7 +77,7 @@ 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 @@ 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 @@ 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); Index: lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java (revision 1351132) +++ lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java (working copy) @@ -282,7 +282,7 @@ // 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 @@ 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 @@ 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(); } Index: lucene/core/src/test/org/apache/lucene/index/TestIndexWriterForceMerge.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestIndexWriterForceMerge.java (revision 1351132) +++ lucene/core/src/test/org/apache/lucene/index/TestIndexWriterForceMerge.java (working copy) @@ -187,7 +187,7 @@ 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 @@ writer.close(); DirectoryReader reader = DirectoryReader.open(dir); - assertTrue(reader.getSequentialSubReaders().length > 1); + assertTrue(reader.getSequentialSubReaders().size() > 1); reader.close(); SegmentInfos infos = new SegmentInfos(); Index: lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java (revision 1351132) +++ lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java (working copy) @@ -982,7 +982,7 @@ 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."); Index: lucene/core/src/test/org/apache/lucene/index/TestIndexWriterUnicode.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestIndexWriterUnicode.java (revision 1351132) +++ lucene/core/src/test/org/apache/lucene/index/TestIndexWriterUnicode.java (working copy) @@ -30,7 +30,6 @@ 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 @@ 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 Index: lucene/core/src/test/org/apache/lucene/index/TestParallelCompositeReader.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestParallelCompositeReader.java (revision 1351132) +++ lucene/core/src/test/org/apache/lucene/index/TestParallelCompositeReader.java (working copy) @@ -339,13 +339,13 @@ 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); Index: lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java (revision 1351132) +++ lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java (working copy) @@ -281,7 +281,7 @@ } private static void printDocs(DirectoryReader r) throws Throwable { - IndexReader[] subs = r.getSequentialSubReaders(); + List subs = r.getSequentialSubReaders(); for(IndexReader sub : subs) { // TODO: improve this Bits liveDocs = ((AtomicReader)sub).getLiveDocs(); Index: lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java (revision 1351132) +++ lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java (working copy) @@ -742,7 +742,7 @@ 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); Index: lucene/core/src/test/org/apache/lucene/index/TestThreadedForceMerge.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestThreadedForceMerge.java (revision 1351132) +++ lucene/core/src/test/org/apache/lucene/index/TestThreadedForceMerge.java (working copy) @@ -130,7 +130,7 @@ 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(); } Index: lucene/core/src/test/org/apache/lucene/index/TestTypePromotion.java =================================================================== --- lucene/core/src/test/org/apache/lucene/index/TestTypePromotion.java (revision 1351132) +++ lucene/core/src/test/org/apache/lucene/index/TestTypePromotion.java (working copy) @@ -120,7 +120,7 @@ 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); @@ -372,7 +372,7 @@ 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"); Index: lucene/core/src/test/org/apache/lucene/search/TestMatchAllDocsQuery.java =================================================================== --- lucene/core/src/test/org/apache/lucene/search/TestMatchAllDocsQuery.java (revision 1351132) +++ lucene/core/src/test/org/apache/lucene/search/TestMatchAllDocsQuery.java (working copy) @@ -54,6 +54,8 @@ IndexSearcher is = newSearcher(ir); ScoreDoc[] hits; + + System.out.println(ir.getTopReaderContext().leaves().length); hits = is.search(new MatchAllDocsQuery(), null, 1000).scoreDocs; assertEquals(3, hits.length); Index: lucene/core/src/test/org/apache/lucene/search/TestShardSearching.java =================================================================== --- lucene/core/src/test/org/apache/lucene/search/TestShardSearching.java (revision 1351132) +++ lucene/core/src/test/org/apache/lucene/search/TestShardSearching.java (working copy) @@ -311,13 +311,13 @@ final int numNodes = shardSearcher.nodeVersions.length; int[] base = new int[numNodes]; - final IndexReader[] subs = ((CompositeReader) mockSearcher.getIndexReader()).getSequentialSubReaders(); - assertEquals(numNodes, subs.length); + final List subs = ((CompositeReader) mockSearcher.getIndexReader()).getSequentialSubReaders(); + assertEquals(numNodes, subs.size()); int docCount = 0; for(int nodeID=0;nodeID subReaders = new ArrayList(); - ReaderUtil.gatherSubReaders(subReaders, s.getIndexReader()); - subSearchers = new ShardSearcher[subReaders.size()]; final IndexReaderContext ctx = s.getTopReaderContext(); + final AtomicReaderContext[] leaves = ctx.leaves(); + subSearchers = new ShardSearcher[leaves.length]; if (ctx instanceof AtomicReaderContext) { assert subSearchers.length == 1; subSearchers[0] = new ShardSearcher((AtomicReaderContext) ctx, ctx); @@ -644,10 +643,8 @@ } docStarts = new int[subSearchers.length]; - int docBase = 0; for(int subIDX=0;subIDX sr = input.getSequentialSubReaders(); + w.addIndexes(sr.toArray(new IndexReader[sr.size()])); // TODO: maybe take List here? w.close(); } System.err.println("Done."); @@ -177,34 +178,36 @@ /** * This class emulates deletions on the underlying index. */ - private static final class FakeDeleteIndexReader extends MultiReader { + private static final class FakeDeleteIndexReader extends BaseCompositeReader { public FakeDeleteIndexReader(IndexReader reader) throws IOException { super(initSubReaders(reader)); } - private static AtomicReader[] initSubReaders(IndexReader reader) throws IOException { - final ArrayList subs = new ArrayList(); - 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 AtomicReaderContext[] leaves = reader.getTopReaderContext().leaves(); + final FakeDeleteAtomicIndexReader[] subs = new FakeDeleteAtomicIndexReader[leaves.length]; + 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(). } Index: lucene/misc/src/java/org/apache/lucene/misc/HighFreqTerms.java =================================================================== --- lucene/misc/src/java/org/apache/lucene/misc/HighFreqTerms.java (revision 1351132) +++ lucene/misc/src/java/org/apache/lucene/misc/HighFreqTerms.java (working copy) @@ -18,6 +18,7 @@ */ 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 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; } } Index: lucene/misc/src/test/org/apache/lucene/index/TestBalancedSegmentMergePolicy.java =================================================================== --- lucene/misc/src/test/org/apache/lucene/index/TestBalancedSegmentMergePolicy.java (revision 1351132) +++ lucene/misc/src/test/org/apache/lucene/index/TestBalancedSegmentMergePolicy.java (working copy) @@ -67,7 +67,7 @@ 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(); } Index: lucene/misc/src/test/org/apache/lucene/index/TestIndexSplitter.java =================================================================== --- lucene/misc/src/test/org/apache/lucene/index/TestIndexSplitter.java (revision 1351132) +++ lucene/misc/src/test/org/apache/lucene/index/TestIndexSplitter.java (working copy) @@ -59,7 +59,7 @@ } 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 @@ // 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(); } Index: lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java =================================================================== --- lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java (revision 1351132) +++ lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java (working copy) @@ -28,6 +28,7 @@ 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 @@ 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(); Index: lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java =================================================================== --- lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (revision 1351132) +++ lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (working copy) @@ -435,11 +435,12 @@ * 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 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; } /** Index: solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java =================================================================== --- solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java (revision 1351132) +++ solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java (working copy) @@ -544,7 +544,7 @@ 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().length); indexInfo.add("current", reader.isCurrent() ); indexInfo.add("hasDeletions", reader.hasDeletions() ); indexInfo.add("directory", dir ); Index: solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java =================================================================== --- solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java (revision 1351132) +++ solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java (working copy) @@ -679,7 +679,7 @@ 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()); Index: solr/core/src/test/org/apache/solr/search/TestSort.java =================================================================== --- solr/core/src/test/org/apache/solr/search/TestSort.java (revision 1351132) +++ solr/core/src/test/org/apache/solr/search/TestSort.java (working copy) @@ -198,7 +198,7 @@ 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