LUCENE-3892: merge trunk

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/pforcodec_3892@1373358 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2012-08-15 11:58:34 +00:00
parent 922efb15b8
commit 7b6955d42a
58 changed files with 243 additions and 693 deletions

View File

@ -123,18 +123,7 @@ public final class MappingMultiDocsAndPositionsEnum extends DocsAndPositionsEnum
@Override @Override
public BytesRef getPayload() throws IOException { public BytesRef getPayload() throws IOException {
BytesRef payload = current.getPayload(); return current.getPayload();
if (mergeState.currentPayloadProcessor[upto] != null && payload != null) {
// to not violate the D&P api, we must give the processor a private copy
// TODO: reuse a BytesRef if there is a PPP
payload = BytesRef.deepCopyOf(payload);
mergeState.currentPayloadProcessor[upto].processPayload(payload);
if (payload.length == 0) {
// don't let PayloadProcessors corrumpt the index
return null;
}
}
return payload;
} }
} }

View File

@ -27,8 +27,6 @@ import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields; import org.apache.lucene.index.Fields;
import org.apache.lucene.index.MergeState; import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.PayloadProcessorProvider.PayloadProcessor;
import org.apache.lucene.index.PayloadProcessorProvider.ReaderPayloadProcessor;
import org.apache.lucene.index.Terms; import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.DocIdSetIterator;
@ -170,12 +168,7 @@ public abstract class TermVectorsWriter implements Closeable {
final AtomicReader reader = mergeState.readers.get(i); final AtomicReader reader = mergeState.readers.get(i);
final int maxDoc = reader.maxDoc(); final int maxDoc = reader.maxDoc();
final Bits liveDocs = reader.getLiveDocs(); final Bits liveDocs = reader.getLiveDocs();
// set PayloadProcessor
if (mergeState.payloadProcessorProvider != null) {
mergeState.currentReaderPayloadProcessor = mergeState.readerPayloadProcessor[i];
} else {
mergeState.currentReaderPayloadProcessor = null;
}
for (int docID = 0; docID < maxDoc; docID++) { for (int docID = 0; docID < maxDoc; docID++) {
if (liveDocs != null && !liveDocs.get(docID)) { if (liveDocs != null && !liveDocs.get(docID)) {
// skip deleted docs // skip deleted docs
@ -215,9 +208,6 @@ public abstract class TermVectorsWriter implements Closeable {
TermsEnum termsEnum = null; TermsEnum termsEnum = null;
DocsAndPositionsEnum docsAndPositionsEnum = null; DocsAndPositionsEnum docsAndPositionsEnum = null;
final ReaderPayloadProcessor readerPayloadProcessor = mergeState.currentReaderPayloadProcessor;
PayloadProcessor payloadProcessor = null;
for(String fieldName : vectors) { for(String fieldName : vectors) {
final FieldInfo fieldInfo = mergeState.fieldInfos.fieldInfo(fieldName); final FieldInfo fieldInfo = mergeState.fieldInfos.fieldInfo(fieldName);
@ -251,10 +241,6 @@ public abstract class TermVectorsWriter implements Closeable {
startTerm(termsEnum.term(), freq); startTerm(termsEnum.term(), freq);
if (hasPayloads && readerPayloadProcessor != null) {
payloadProcessor = readerPayloadProcessor.getProcessor(fieldName, termsEnum.term());
}
if (hasPositions || hasOffsets) { if (hasPositions || hasOffsets) {
docsAndPositionsEnum = termsEnum.docsAndPositions(null, docsAndPositionsEnum); docsAndPositionsEnum = termsEnum.docsAndPositions(null, docsAndPositionsEnum);
assert docsAndPositionsEnum != null; assert docsAndPositionsEnum != null;
@ -268,17 +254,7 @@ public abstract class TermVectorsWriter implements Closeable {
final int startOffset = docsAndPositionsEnum.startOffset(); final int startOffset = docsAndPositionsEnum.startOffset();
final int endOffset = docsAndPositionsEnum.endOffset(); final int endOffset = docsAndPositionsEnum.endOffset();
BytesRef payload = docsAndPositionsEnum.getPayload(); final BytesRef payload = docsAndPositionsEnum.getPayload();
if (payloadProcessor != null && payload != null) {
// to not violate the D&P api, we must give the processor a private copy
payload = BytesRef.deepCopyOf(payload);
payloadProcessor.processPayload(payload);
if (payload.length == 0) {
// don't let PayloadProcessors corrumpt the index
payload = null;
}
}
assert !hasPositions || pos >= 0; assert !hasPositions || pos >= 0;
addPosition(pos, startOffset, endOffset, payload); addPosition(pos, startOffset, endOffset, payload);

View File

@ -154,14 +154,7 @@ public abstract class TermsConsumer {
postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(null, postingsEnumIn, DocsAndPositionsEnum.FLAG_PAYLOADS); postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(null, postingsEnumIn, DocsAndPositionsEnum.FLAG_PAYLOADS);
assert postingsEnumIn != null; assert postingsEnumIn != null;
postingsEnum.reset(postingsEnumIn); postingsEnum.reset(postingsEnumIn);
// set PayloadProcessor
if (mergeState.payloadProcessorProvider != null) {
for (int i = 0; i < mergeState.readers.size(); i++) {
if (mergeState.readerPayloadProcessor[i] != null) {
mergeState.currentPayloadProcessor[i] = mergeState.readerPayloadProcessor[i].getProcessor(mergeState.fieldInfo.name, term);
}
}
}
final PostingsConsumer postingsConsumer = startTerm(term); final PostingsConsumer postingsConsumer = startTerm(term);
final TermStats stats = postingsConsumer.merge(mergeState, postingsEnum, visitedDocs); final TermStats stats = postingsConsumer.merge(mergeState, postingsEnum, visitedDocs);
if (stats.docFreq > 0) { if (stats.docFreq > 0) {
@ -188,14 +181,7 @@ public abstract class TermsConsumer {
postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(null, postingsEnumIn); postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(null, postingsEnumIn);
assert postingsEnumIn != null; assert postingsEnumIn != null;
postingsEnum.reset(postingsEnumIn); postingsEnum.reset(postingsEnumIn);
// set PayloadProcessor
if (mergeState.payloadProcessorProvider != null) {
for (int i = 0; i < mergeState.readers.size(); i++) {
if (mergeState.readerPayloadProcessor[i] != null) {
mergeState.currentPayloadProcessor[i] = mergeState.readerPayloadProcessor[i].getProcessor(mergeState.fieldInfo.name, term);
}
}
}
final PostingsConsumer postingsConsumer = startTerm(term); final PostingsConsumer postingsConsumer = startTerm(term);
final TermStats stats = postingsConsumer.merge(mergeState, postingsEnum, visitedDocs); final TermStats stats = postingsConsumer.merge(mergeState, postingsEnum, visitedDocs);
if (stats.docFreq > 0) { if (stats.docFreq > 0) {

View File

@ -318,7 +318,7 @@ final class BlockPostingsWriter extends PostingsWriterBase {
// current block // current block
if (lastBlockDocID != -1 && docBufferUpto == 1) { if (lastBlockDocID != -1 && docBufferUpto == 1) {
// nocomit move to startDoc? ie we can write skip // TODO: can we move this to startDoc? ie we can write skip
// data as soon as the next doc starts... // data as soon as the next doc starts...
if (DEBUG) { if (DEBUG) {
System.out.println(" bufferSkip at writeBlock: lastDocID=" + lastBlockDocID + " docCount=" + (docCount-1)); System.out.println(" bufferSkip at writeBlock: lastDocID=" + lastBlockDocID + " docCount=" + (docCount-1));

View File

@ -315,12 +315,7 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
int numDocs = 0; int numDocs = 0;
for (int i = 0; i < mergeState.readers.size(); i++) { for (int i = 0; i < mergeState.readers.size(); i++) {
final AtomicReader reader = mergeState.readers.get(i); final AtomicReader reader = mergeState.readers.get(i);
// set PayloadProcessor
if (mergeState.payloadProcessorProvider != null) {
mergeState.currentReaderPayloadProcessor = mergeState.readerPayloadProcessor[i];
} else {
mergeState.currentReaderPayloadProcessor = null;
}
final SegmentReader matchingSegmentReader = mergeState.matchingSegmentReaders[idx++]; final SegmentReader matchingSegmentReader = mergeState.matchingSegmentReaders[idx++];
Lucene40TermVectorsReader matchingVectorsReader = null; Lucene40TermVectorsReader matchingVectorsReader = null;
if (matchingSegmentReader != null) { if (matchingSegmentReader != null) {
@ -353,8 +348,8 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
final int maxDoc = reader.maxDoc(); final int maxDoc = reader.maxDoc();
final Bits liveDocs = reader.getLiveDocs(); final Bits liveDocs = reader.getLiveDocs();
int totalNumDocs = 0; int totalNumDocs = 0;
if (matchingVectorsReader != null && mergeState.currentReaderPayloadProcessor == null) { if (matchingVectorsReader != null) {
// We can bulk-copy because the fieldInfos are "congruent" and there is no payload processor // We can bulk-copy because the fieldInfos are "congruent"
for (int docNum = 0; docNum < maxDoc;) { for (int docNum = 0; docNum < maxDoc;) {
if (!liveDocs.get(docNum)) { if (!liveDocs.get(docNum)) {
// skip deleted docs // skip deleted docs
@ -404,8 +399,8 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
int rawDocLengths2[]) int rawDocLengths2[])
throws IOException { throws IOException {
final int maxDoc = reader.maxDoc(); final int maxDoc = reader.maxDoc();
if (matchingVectorsReader != null && mergeState.currentReaderPayloadProcessor == null) { if (matchingVectorsReader != null) {
// We can bulk-copy because the fieldInfos are "congruent" and there is no payload processor // We can bulk-copy because the fieldInfos are "congruent"
int docCount = 0; int docCount = 0;
while (docCount < maxDoc) { while (docCount < maxDoc) {
int len = Math.min(MAX_RAW_MERGE_DOCS, maxDoc - docCount); int len = Math.min(MAX_RAW_MERGE_DOCS, maxDoc - docCount);

View File

@ -54,7 +54,7 @@ public abstract class AtomicReader extends IndexReader {
} }
@Override @Override
public final AtomicReaderContext getTopReaderContext() { public final AtomicReaderContext getContext() {
ensureOpen(); ensureOpen();
return readerContext; return readerContext;
} }

View File

@ -21,8 +21,7 @@ import java.util.Collections;
import java.util.List; import java.util.List;
/** /**
* {@link IndexReaderContext} for {@link AtomicReader} instances * {@link IndexReaderContext} for {@link AtomicReader} instances.
* @lucene.experimental
*/ */
public final class AtomicReaderContext extends IndexReaderContext { public final class AtomicReaderContext extends IndexReaderContext {
/** The readers ord in the top-level's leaves array */ /** The readers ord in the top-level's leaves array */

View File

@ -439,7 +439,7 @@ class BufferedDeletesStream {
// Delete by query // Delete by query
private static long applyQueryDeletes(Iterable<QueryAndLimit> queriesIter, ReadersAndLiveDocs rld, final SegmentReader reader) throws IOException { private static long applyQueryDeletes(Iterable<QueryAndLimit> queriesIter, ReadersAndLiveDocs rld, final SegmentReader reader) throws IOException {
long delCount = 0; long delCount = 0;
final AtomicReaderContext readerContext = reader.getTopReaderContext(); final AtomicReaderContext readerContext = reader.getContext();
boolean any = false; boolean any = false;
for (QueryAndLimit ent : queriesIter) { for (QueryAndLimit ent : queriesIter) {
Query query = ent.query; Query query = ent.query;

View File

@ -24,7 +24,6 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
import org.apache.lucene.util.MergedIterator;
import org.apache.lucene.index.BufferedDeletesStream.QueryAndLimit; import org.apache.lucene.index.BufferedDeletesStream.QueryAndLimit;
class CoalescedDeletes { class CoalescedDeletes {

View File

@ -78,16 +78,18 @@ public abstract class CompositeReader extends IndexReader {
} }
/** Expert: returns the sequential sub readers that this /** Expert: returns the sequential sub readers that this
* reader is logically composed of. It contrast to previous * reader is logically composed of. This method may not
* Lucene versions may not return null. * return {@code null}.
* If this method returns an empty array, that means this *
* reader is a null reader (for example a MultiReader * <p><b>NOTE:</b> In contrast to previous Lucene versions this method
* that has no sub readers). * is no longer public, code that wants to get all {@link AtomicReader}s
* this composite is composed of should use {@link IndexReader#leaves()}.
* @see IndexReader#leaves()
*/ */
public abstract List<? extends IndexReader> getSequentialSubReaders(); protected abstract List<? extends IndexReader> getSequentialSubReaders();
@Override @Override
public final CompositeReaderContext getTopReaderContext() { public final CompositeReaderContext getContext() {
ensureOpen(); ensureOpen();
// lazy init without thread safety for perf reasons: Building the readerContext twice does not hurt! // lazy init without thread safety for perf reasons: Building the readerContext twice does not hurt!
if (readerContext == null) { if (readerContext == null) {

View File

@ -24,7 +24,6 @@ import java.util.List;
/** /**
* {@link IndexReaderContext} for {@link CompositeReader} instance. * {@link IndexReaderContext} for {@link CompositeReader} instance.
* @lucene.experimental
*/ */
public final class CompositeReaderContext extends IndexReaderContext { public final class CompositeReaderContext extends IndexReaderContext {
private final List<IndexReaderContext> children; private final List<IndexReaderContext> children;

View File

@ -21,6 +21,7 @@ import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.LinkedHashSet; import java.util.LinkedHashSet;
import java.util.List;
import java.util.WeakHashMap; import java.util.WeakHashMap;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
@ -377,9 +378,11 @@ public abstract class IndexReader implements Closeable {
protected abstract void doClose() throws IOException; protected abstract void doClose() throws IOException;
/** /**
* Expert: Returns a the root {@link IndexReaderContext} for this * Expert: Returns the root {@link IndexReaderContext} for this
* {@link IndexReader}'s sub-reader tree. Iff this reader is composed of sub * {@link IndexReader}'s sub-reader tree.
* readers ,ie. this reader being a composite reader, this method returns a * <p>
* Iff this reader is composed of sub
* readers, i.e. this reader being a composite reader, this method returns a
* {@link CompositeReaderContext} holding the reader's direct children as well as a * {@link CompositeReaderContext} holding the reader's direct children as well as a
* view of the reader tree's atomic leaf contexts. All sub- * view of the reader tree's atomic leaf contexts. All sub-
* {@link IndexReaderContext} instances referenced from this readers top-level * {@link IndexReaderContext} instances referenced from this readers top-level
@ -388,14 +391,21 @@ public abstract class IndexReader implements Closeable {
* atomic leaf reader at a time. If this reader is not composed of child * atomic leaf reader at a time. If this reader is not composed of child
* readers, this method returns an {@link AtomicReaderContext}. * readers, this method returns an {@link AtomicReaderContext}.
* <p> * <p>
* Note: Any of the sub-{@link CompositeReaderContext} instances reference from this * Note: Any of the sub-{@link CompositeReaderContext} instances referenced
* top-level context holds a <code>null</code> {@link CompositeReaderContext#leaves()} * from this top-level context do not support {@link CompositeReaderContext#leaves()}.
* reference. Only the top-level context maintains the convenience leaf-view * Only the top-level context maintains the convenience leaf-view
* for performance reasons. * for performance reasons.
*
* @lucene.experimental
*/ */
public abstract IndexReaderContext getTopReaderContext(); public abstract IndexReaderContext getContext();
/**
* Returns the reader's leaves, or itself if this reader is atomic.
* This is a convenience method calling {@code this.getContext().leaves()}.
* @see IndexReaderContext#leaves()
*/
public final List<AtomicReaderContext> leaves() {
return getContext().leaves();
}
/** Expert: Returns a key for this IndexReader, so FieldCache/CachingWrapperFilter can find /** Expert: Returns a key for this IndexReader, so FieldCache/CachingWrapperFilter can find
* it again. * it again.

View File

@ -22,7 +22,6 @@ import java.util.List;
/** /**
* A struct like class that represents a hierarchical relationship between * A struct like class that represents a hierarchical relationship between
* {@link IndexReader} instances. * {@link IndexReader} instances.
* @lucene.experimental
*/ */
public abstract class IndexReaderContext { public abstract class IndexReaderContext {
/** The reader context for this reader's immediate parent, or null if none */ /** The reader context for this reader's immediate parent, or null if none */

View File

@ -260,9 +260,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// to allow users to query an IndexWriter settings. // to allow users to query an IndexWriter settings.
private final LiveIndexWriterConfig config; private final LiveIndexWriterConfig config;
// The PayloadProcessorProvider to use when segments are merged
private PayloadProcessorProvider payloadProcessorProvider;
DirectoryReader getReader() throws IOException { DirectoryReader getReader() throws IOException {
return getReader(true); return getReader(true);
} }
@ -763,8 +760,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
} }
/** /**
* Commits all changes to an index and closes all * Commits all changes to an index, waits for pending merges
* associated files. Note that this may be a costly * to complete, and closes all associated files.
* <p>
* This is a "slow graceful shutdown" which may take a long time
* especially if a big merge is pending: If you only want to close
* resources use {@link #rollback()}. If you only want to commit
* pending changes and close resources see {@link #close(boolean)}.
* <p>
* Note that this may be a costly
* operation, so, try to re-use a single writer instead of * operation, so, try to re-use a single writer instead of
* closing and opening a new one. See {@link #commit()} for * closing and opening a new one. See {@link #commit()} for
* caveats about write caching done by some IO devices. * caveats about write caching done by some IO devices.
@ -1263,7 +1267,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
reader = (AtomicReader) readerIn; reader = (AtomicReader) readerIn;
} else { } else {
// Composite reader: lookup sub-reader and re-base docID: // Composite reader: lookup sub-reader and re-base docID:
List<AtomicReaderContext> leaves = readerIn.getTopReaderContext().leaves(); List<AtomicReaderContext> leaves = readerIn.leaves();
int subIndex = ReaderUtil.subIndex(docID, leaves); int subIndex = ReaderUtil.subIndex(docID, leaves);
reader = leaves.get(subIndex).reader(); reader = leaves.get(subIndex).reader();
docID -= leaves.get(subIndex).docBase; docID -= leaves.get(subIndex).docBase;
@ -2399,8 +2403,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
false, codec, null, null); false, codec, null, null);
SegmentMerger merger = new SegmentMerger(info, infoStream, trackingDir, config.getTermIndexInterval(), SegmentMerger merger = new SegmentMerger(info, infoStream, trackingDir, config.getTermIndexInterval(),
MergeState.CheckAbort.NONE, payloadProcessorProvider, MergeState.CheckAbort.NONE, globalFieldNumberMap, context);
globalFieldNumberMap, context);
for (IndexReader reader : readers) { // add new indexes for (IndexReader reader : readers) { // add new indexes
merger.add(reader); merger.add(reader);
@ -3503,7 +3506,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
final TrackingDirectoryWrapper dirWrapper = new TrackingDirectoryWrapper(directory); final TrackingDirectoryWrapper dirWrapper = new TrackingDirectoryWrapper(directory);
SegmentMerger merger = new SegmentMerger(merge.info.info, infoStream, dirWrapper, config.getTermIndexInterval(), checkAbort, SegmentMerger merger = new SegmentMerger(merge.info.info, infoStream, dirWrapper, config.getTermIndexInterval(), checkAbort,
payloadProcessorProvider, globalFieldNumberMap, context); globalFieldNumberMap, context);
if (infoStream.isEnabled("IW")) { if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "merging " + segString(merge.segments)); infoStream.message("IW", "merging " + segString(merge.segments));
@ -4059,38 +4062,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
deleter.deletePendingFiles(); deleter.deletePendingFiles();
} }
/**
* Sets the {@link PayloadProcessorProvider} to use when merging payloads.
* Note that the given <code>pcp</code> will be invoked for every segment that
* is merged, not only external ones that are given through
* {@link #addIndexes}. If you want only the payloads of the external segments
* to be processed, you can return <code>null</code> whenever a
* {@link PayloadProcessorProvider.ReaderPayloadProcessor} is requested for the {@link Directory} of the
* {@link IndexWriter}.
* <p>
* The default is <code>null</code> which means payloads are processed
* normally (copied) during segment merges. You can also unset it by passing
* <code>null</code>.
* <p>
* <b>NOTE:</b> the set {@link PayloadProcessorProvider} will be in effect
* immediately, potentially for already running merges too. If you want to be
* sure it is used for further operations only, such as {@link #addIndexes} or
* {@link #forceMerge}, you can call {@link #waitForMerges()} before.
*/
public void setPayloadProcessorProvider(PayloadProcessorProvider pcp) {
ensureOpen();
payloadProcessorProvider = pcp;
}
/**
* Returns the {@link PayloadProcessorProvider} that is used during segment
* merges to process payloads.
*/
public PayloadProcessorProvider getPayloadProcessorProvider() {
ensureOpen();
return payloadProcessorProvider;
}
/** /**
* NOTE: this method creates a compound file for all files returned by * NOTE: this method creates a compound file for all files returned by
* info.files(). While, generally, this may include separate norms and * info.files(). While, generally, this may include separate norms and

View File

@ -19,8 +19,6 @@ package org.apache.lucene.index;
import java.util.List; import java.util.List;
import org.apache.lucene.index.PayloadProcessorProvider.PayloadProcessor;
import org.apache.lucene.index.PayloadProcessorProvider.ReaderPayloadProcessor;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
import org.apache.lucene.util.InfoStream; import org.apache.lucene.util.InfoStream;
@ -194,14 +192,6 @@ public class MergeState {
// Updated per field; // Updated per field;
public FieldInfo fieldInfo; public FieldInfo fieldInfo;
// Used to process payloads
// TODO: this is a FactoryFactory here basically
// and we could make a codec(wrapper) to do all of this privately so IW is uninvolved
public PayloadProcessorProvider payloadProcessorProvider;
public ReaderPayloadProcessor[] readerPayloadProcessor;
public ReaderPayloadProcessor currentReaderPayloadProcessor;
public PayloadProcessor[] currentPayloadProcessor;
// TODO: get rid of this? it tells you which segments are 'aligned' (e.g. for bulk merging) // TODO: get rid of this? it tells you which segments are 'aligned' (e.g. for bulk merging)
// but is this really so expensive to compute again in different components, versus once in SM? // but is this really so expensive to compute again in different components, versus once in SM?
public SegmentReader[] matchingSegmentReaders; public SegmentReader[] matchingSegmentReaders;

View File

@ -1,4 +1,4 @@
package org.apache.lucene.util; package org.apache.lucene.index;
/* /*
* Licensed to the Apache Software Foundation (ASF) under one or more * Licensed to the Apache Software Foundation (ASF) under one or more
@ -20,6 +20,8 @@ package org.apache.lucene.util;
import java.util.Iterator; import java.util.Iterator;
import java.util.NoSuchElementException; import java.util.NoSuchElementException;
import org.apache.lucene.util.PriorityQueue;
/** /**
* Provides a merged sorted view from several sorted iterators, each * Provides a merged sorted view from several sorted iterators, each
* iterating over a unique set of elements. * iterating over a unique set of elements.
@ -39,13 +41,13 @@ import java.util.NoSuchElementException;
* </ul> * </ul>
* @lucene.internal * @lucene.internal
*/ */
public class MergedIterator<T extends Comparable<T>> implements Iterator<T> { final class MergedIterator<T extends Comparable<T>> implements Iterator<T> {
private T current; private T current;
private final TermMergeQueue<T> queue; private final TermMergeQueue<T> queue;
private final SubIterator<T>[] top; private final SubIterator<T>[] top;
private int numTop; private int numTop;
@SuppressWarnings("unchecked") @SuppressWarnings({"unchecked","rawtypes"})
public MergedIterator(Iterator<T>... iterators) { public MergedIterator(Iterator<T>... iterators) {
queue = new TermMergeQueue<T>(iterators.length); queue = new TermMergeQueue<T>(iterators.length);
top = new SubIterator[iterators.length]; top = new SubIterator[iterators.length];

View File

@ -36,7 +36,7 @@ import org.apache.lucene.util.packed.PackedInts.Reader;
* *
* <p><b>NOTE</b>: for multi readers, you'll get better * <p><b>NOTE</b>: for multi readers, you'll get better
* performance by gathering the sub readers using * performance by gathering the sub readers using
* {@link IndexReader#getTopReaderContext()} to get the * {@link IndexReader#getContext()} to get the
* atomic leaves and then operate per-AtomicReader, * atomic leaves and then operate per-AtomicReader,
* instead of using this class. * instead of using this class.
* *
@ -128,7 +128,7 @@ public class MultiDocValues extends DocValues {
return puller.pull((AtomicReader) reader, field); return puller.pull((AtomicReader) reader, field);
} }
assert reader instanceof CompositeReader; assert reader instanceof CompositeReader;
final List<AtomicReaderContext> leaves = reader.getTopReaderContext().leaves(); final List<AtomicReaderContext> leaves = reader.leaves();
switch (leaves.size()) { switch (leaves.size()) {
case 0: case 0:
// no fields // no fields

View File

@ -28,7 +28,6 @@ import java.util.concurrent.ConcurrentHashMap;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.MergedIterator;
/** /**
* Exposes flex API, merged from flex API of sub-segments. * Exposes flex API, merged from flex API of sub-segments.
@ -39,7 +38,7 @@ import org.apache.lucene.util.MergedIterator;
* *
* <p><b>NOTE</b>: for composite readers, you'll get better * <p><b>NOTE</b>: for composite readers, you'll get better
* performance by gathering the sub readers using * performance by gathering the sub readers using
* {@link IndexReader#getTopReaderContext()} to get the * {@link IndexReader#getContext()} to get the
* atomic leaves and then operate per-AtomicReader, * atomic leaves and then operate per-AtomicReader,
* instead of using this class. * instead of using this class.
* *
@ -60,7 +59,7 @@ public final class MultiFields extends Fields {
* It's better to get the sub-readers and iterate through them * It's better to get the sub-readers and iterate through them
* yourself. */ * yourself. */
public static Fields getFields(IndexReader reader) throws IOException { public static Fields getFields(IndexReader reader) throws IOException {
final List<AtomicReaderContext> leaves = reader.getTopReaderContext().leaves(); final List<AtomicReaderContext> leaves = reader.leaves();
switch (leaves.size()) { switch (leaves.size()) {
case 0: case 0:
// no fields // no fields
@ -92,7 +91,7 @@ public final class MultiFields extends Fields {
public static Bits getLiveDocs(IndexReader reader) { public static Bits getLiveDocs(IndexReader reader) {
if (reader.hasDeletions()) { if (reader.hasDeletions()) {
final List<AtomicReaderContext> leaves = reader.getTopReaderContext().leaves(); final List<AtomicReaderContext> leaves = reader.leaves();
final int size = leaves.size(); final int size = leaves.size();
assert size > 0 : "A reader with deletions must have at least one leave"; assert size > 0 : "A reader with deletions must have at least one leave";
if (size == 1) { if (size == 1) {
@ -182,7 +181,7 @@ public final class MultiFields extends Fields {
this.subSlices = subSlices; this.subSlices = subSlices;
} }
@SuppressWarnings("unchecked") @SuppressWarnings({"unchecked","rawtypes"})
@Override @Override
public Iterator<String> iterator() { public Iterator<String> iterator() {
Iterator<String> subIterators[] = new Iterator[subs.length]; Iterator<String> subIterators[] = new Iterator[subs.length];
@ -251,7 +250,7 @@ public final class MultiFields extends Fields {
*/ */
public static FieldInfos getMergedFieldInfos(IndexReader reader) { public static FieldInfos getMergedFieldInfos(IndexReader reader) {
final FieldInfos.Builder builder = new FieldInfos.Builder(); final FieldInfos.Builder builder = new FieldInfos.Builder();
for(final AtomicReaderContext ctx : reader.getTopReaderContext().leaves()) { for(final AtomicReaderContext ctx : reader.leaves()) {
builder.add(ctx.reader().getFieldInfos()); builder.add(ctx.reader().getFieldInfos());
} }
return builder.finish(); return builder.finish();

View File

@ -1,81 +0,0 @@
package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
/**
* Provides a {@link ReaderPayloadProcessor} to be used for a {@link Directory}.
* This allows using different {@link ReaderPayloadProcessor}s for different
* source {@link AtomicReader}, for e.g. to perform different processing of payloads of
* different directories.
* <p>
* <b>NOTE:</b> to avoid processing payloads of certain directories, you can
* return <code>null</code> in {@link #getReaderProcessor}.
* <p>
* <b>NOTE:</b> it is possible that the same {@link ReaderPayloadProcessor} will be
* requested for the same {@link Directory} concurrently. Therefore, to avoid
* concurrency issues you should return different instances for different
* threads. Usually, if your {@link ReaderPayloadProcessor} does not maintain state
* this is not a problem. The merge code ensures that the
* {@link ReaderPayloadProcessor} instance you return will be accessed by one
* thread to obtain the {@link PayloadProcessor}s for different terms.
*
* @lucene.experimental
*/
public abstract class PayloadProcessorProvider {
/**
* Returns a {@link PayloadProcessor} for a given {@link Term} which allows
* processing the payloads of different terms differently. If you intent to
* process all your payloads the same way, then you can ignore the given term.
* <p>
* <b>NOTE:</b> if you protect your {@link ReaderPayloadProcessor} from
* concurrency issues, then you shouldn't worry about any such issues when
* {@link PayloadProcessor}s are requested for different terms.
*/
public static abstract class ReaderPayloadProcessor {
/** Returns a {@link PayloadProcessor} for the given term. */
public abstract PayloadProcessor getProcessor(String field, BytesRef text) throws IOException;
}
/**
* Processes the given payload.
*
* @lucene.experimental
*/
public static abstract class PayloadProcessor {
/** Process the incoming payload and stores the result in the given {@link BytesRef}. */
public abstract void processPayload(BytesRef payload) throws IOException;
}
/**
* Returns a {@link ReaderPayloadProcessor} for the given {@link Directory},
* through which {@link PayloadProcessor}s can be obtained for each
* {@link Term}, or <code>null</code> if none should be used.
*/
public abstract ReaderPayloadProcessor getReaderProcessor(AtomicReader reader) throws IOException;
}

View File

@ -56,13 +56,11 @@ final class SegmentMerger {
// note, just like in codec apis Directory 'dir' is NOT the same as segmentInfo.dir!! // note, just like in codec apis Directory 'dir' is NOT the same as segmentInfo.dir!!
SegmentMerger(SegmentInfo segmentInfo, InfoStream infoStream, Directory dir, int termIndexInterval, SegmentMerger(SegmentInfo segmentInfo, InfoStream infoStream, Directory dir, int termIndexInterval,
MergeState.CheckAbort checkAbort, PayloadProcessorProvider payloadProcessorProvider, MergeState.CheckAbort checkAbort, FieldInfos.FieldNumbers fieldNumbers, IOContext context) {
FieldInfos.FieldNumbers fieldNumbers, IOContext context) {
mergeState.segmentInfo = segmentInfo; mergeState.segmentInfo = segmentInfo;
mergeState.infoStream = infoStream; mergeState.infoStream = infoStream;
mergeState.readers = new ArrayList<AtomicReader>(); mergeState.readers = new ArrayList<AtomicReader>();
mergeState.checkAbort = checkAbort; mergeState.checkAbort = checkAbort;
mergeState.payloadProcessorProvider = payloadProcessorProvider;
directory = dir; directory = dir;
this.termIndexInterval = termIndexInterval; this.termIndexInterval = termIndexInterval;
this.codec = segmentInfo.getCodec(); this.codec = segmentInfo.getCodec();
@ -75,7 +73,7 @@ final class SegmentMerger {
* @param reader * @param reader
*/ */
final void add(IndexReader reader) { final void add(IndexReader reader) {
for (final AtomicReaderContext ctx : reader.getTopReaderContext().leaves()) { for (final AtomicReaderContext ctx : reader.leaves()) {
final AtomicReader r = ctx.reader(); final AtomicReader r = ctx.reader();
mergeState.readers.add(r); mergeState.readers.add(r);
} }
@ -274,8 +272,6 @@ final class SegmentMerger {
// Remap docIDs // Remap docIDs
mergeState.docMaps = new MergeState.DocMap[numReaders]; mergeState.docMaps = new MergeState.DocMap[numReaders];
mergeState.docBase = new int[numReaders]; mergeState.docBase = new int[numReaders];
mergeState.readerPayloadProcessor = new PayloadProcessorProvider.ReaderPayloadProcessor[numReaders];
mergeState.currentPayloadProcessor = new PayloadProcessorProvider.PayloadProcessor[numReaders];
int docBase = 0; int docBase = 0;
@ -289,10 +285,6 @@ final class SegmentMerger {
mergeState.docMaps[i] = docMap; mergeState.docMaps[i] = docMap;
docBase += docMap.numDocs(); docBase += docMap.numDocs();
if (mergeState.payloadProcessorProvider != null) {
mergeState.readerPayloadProcessor[i] = mergeState.payloadProcessorProvider.getReaderProcessor(reader);
}
i++; i++;
} }

View File

@ -37,7 +37,7 @@ import org.apache.lucene.index.MultiReader; // javadoc
* <p><b>NOTE</b>: this class almost always results in a * <p><b>NOTE</b>: this class almost always results in a
* performance hit. If this is important to your use case, * performance hit. If this is important to your use case,
* you'll get better performance by gathering the sub readers using * you'll get better performance by gathering the sub readers using
* {@link IndexReader#getTopReaderContext()} to get the * {@link IndexReader#getContext()} to get the
* atomic leaves and then operate per-AtomicReader, * atomic leaves and then operate per-AtomicReader,
* instead of using this class. * instead of using this class.
*/ */

View File

@ -122,7 +122,7 @@ public class IndexSearcher {
* *
* @lucene.experimental */ * @lucene.experimental */
public IndexSearcher(IndexReader r, ExecutorService executor) { public IndexSearcher(IndexReader r, ExecutorService executor) {
this(r.getTopReaderContext(), executor); this(r.getContext(), executor);
} }
/** /**
@ -138,7 +138,7 @@ public class IndexSearcher {
* href="https://issues.apache.org/jira/browse/LUCENE-2239">LUCENE-2239</a>). * href="https://issues.apache.org/jira/browse/LUCENE-2239">LUCENE-2239</a>).
* *
* @see IndexReaderContext * @see IndexReaderContext
* @see IndexReader#getTopReaderContext() * @see IndexReader#getContext()
* @lucene.experimental * @lucene.experimental
*/ */
public IndexSearcher(IndexReaderContext context, ExecutorService executor) { public IndexSearcher(IndexReaderContext context, ExecutorService executor) {
@ -154,7 +154,7 @@ public class IndexSearcher {
* Creates a searcher searching the provided top-level {@link IndexReaderContext}. * Creates a searcher searching the provided top-level {@link IndexReaderContext}.
* *
* @see IndexReaderContext * @see IndexReaderContext
* @see IndexReader#getTopReaderContext() * @see IndexReader#getContext()
* @lucene.experimental * @lucene.experimental
*/ */
public IndexSearcher(IndexReaderContext context) { public IndexSearcher(IndexReaderContext context) {
@ -639,7 +639,7 @@ public class IndexSearcher {
/** /**
* Returns this searchers the top-level {@link IndexReaderContext}. * Returns this searchers the top-level {@link IndexReaderContext}.
* @see IndexReader#getTopReaderContext() * @see IndexReader#getContext()
*/ */
/* sugar for #getReader().getTopReaderContext() */ /* sugar for #getReader().getTopReaderContext() */
public IndexReaderContext getTopReaderContext() { public IndexReaderContext getTopReaderContext() {

View File

@ -50,7 +50,7 @@ public class QueryWrapperFilter extends Filter {
@Override @Override
public DocIdSet getDocIdSet(final AtomicReaderContext context, final Bits acceptDocs) throws IOException { public DocIdSet getDocIdSet(final AtomicReaderContext context, final Bits acceptDocs) throws IOException {
// get a private context that is used to rewrite, createWeight and score eventually // get a private context that is used to rewrite, createWeight and score eventually
final AtomicReaderContext privateContext = context.reader().getTopReaderContext(); final AtomicReaderContext privateContext = context.reader().getContext();
final Weight weight = new IndexSearcher(privateContext).createNormalizedWeight(query); final Weight weight = new IndexSearcher(privateContext).createNormalizedWeight(query);
return new DocIdSet() { return new DocIdSet() {
@Override @Override

View File

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

View File

@ -60,7 +60,7 @@ public class PayloadSpanUtil {
* @param context * @param context
* that contains doc with payloads to extract * that contains doc with payloads to extract
* *
* @see IndexReader#getTopReaderContext() * @see IndexReader#getContext()
*/ */
public PayloadSpanUtil(IndexReaderContext context) { public PayloadSpanUtil(IndexReaderContext context) {
this.context = context; this.context = context;

View File

@ -25,8 +25,10 @@ import java.util.Set;
import org.apache.lucene.index.CompositeReader; import org.apache.lucene.index.CompositeReader;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.search.FieldCache; import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.FieldCache.CacheEntry; import org.apache.lucene.search.FieldCache.CacheEntry;
import org.apache.lucene.store.AlreadyClosedException;
/** /**
* Provides methods for sanity checking that entries in the FieldCache * Provides methods for sanity checking that entries in the FieldCache
@ -272,20 +274,28 @@ public final class FieldCacheSanityChecker {
/** /**
* Checks if the seed is an IndexReader, and if so will walk * Checks if the seed is an IndexReader, and if so will walk
* the hierarchy of subReaders building up a list of the objects * the hierarchy of subReaders building up a list of the objects
* returned by obj.getFieldCacheKey() * returned by {@code seed.getCoreCacheKey()}
*/ */
private List<Object> getAllDescendantReaderKeys(Object seed) { private List<Object> getAllDescendantReaderKeys(Object seed) {
List<Object> all = new ArrayList<Object>(17); // will grow as we iter List<Object> all = new ArrayList<Object>(17); // will grow as we iter
all.add(seed); all.add(seed);
for (int i = 0; i < all.size(); i++) { for (int i = 0; i < all.size(); i++) {
Object obj = all.get(i); final Object obj = all.get(i);
if (obj instanceof CompositeReader) { // TODO: We don't check closed readers here (as getTopReaderContext
List<? extends IndexReader> subs = ((CompositeReader)obj).getSequentialSubReaders(); // throws AlreadyClosedException), what should we do? Reflection?
for (int j = 0; (null != subs) && (j < subs.size()); j++) { if (obj instanceof IndexReader) {
all.add(subs.get(j).getCoreCacheKey()); try {
final List<IndexReaderContext> childs =
((IndexReader) obj).getContext().children();
if (childs != null) { // it is composite reader
for (final IndexReaderContext ctx : childs) {
all.add(ctx.reader().getCoreCacheKey());
}
}
} catch (AlreadyClosedException ace) {
// ignore this reader
} }
} }
} }
// need to skip the first, because it was the seed // need to skip the first, because it was the seed
return all.subList(1, all.size()); return all.subList(1, all.size());

View File

@ -23,9 +23,9 @@ import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.AtomicReader; import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.DocsEnum; import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Terms; import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.TermsEnum;
@ -50,7 +50,8 @@ public class TestReuseDocsEnum extends LuceneTestCase {
writer.commit(); writer.commit();
DirectoryReader open = DirectoryReader.open(dir); DirectoryReader open = DirectoryReader.open(dir);
for (AtomicReader indexReader : open.getSequentialSubReaders()) { for (AtomicReaderContext ctx : open.leaves()) {
AtomicReader indexReader = ctx.reader();
Terms terms = indexReader.terms("body"); Terms terms = indexReader.terms("body");
TermsEnum iterator = terms.iterator(null); TermsEnum iterator = terms.iterator(null);
IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<DocsEnum, Boolean>(); IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<DocsEnum, Boolean>();
@ -76,8 +77,8 @@ public class TestReuseDocsEnum extends LuceneTestCase {
writer.commit(); writer.commit();
DirectoryReader open = DirectoryReader.open(dir); DirectoryReader open = DirectoryReader.open(dir);
for (AtomicReader indexReader : open.getSequentialSubReaders()) { for (AtomicReaderContext ctx : open.leaves()) {
Terms terms = indexReader.terms("body"); Terms terms = ctx.reader().terms("body");
TermsEnum iterator = terms.iterator(null); TermsEnum iterator = terms.iterator(null);
IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<DocsEnum, Boolean>(); IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<DocsEnum, Boolean>();
MatchNoBits bits = new Bits.MatchNoBits(open.maxDoc()); MatchNoBits bits = new Bits.MatchNoBits(open.maxDoc());
@ -121,11 +122,11 @@ public class TestReuseDocsEnum extends LuceneTestCase {
DirectoryReader firstReader = DirectoryReader.open(dir); DirectoryReader firstReader = DirectoryReader.open(dir);
DirectoryReader secondReader = DirectoryReader.open(dir); DirectoryReader secondReader = DirectoryReader.open(dir);
List<? extends AtomicReader> sequentialSubReaders = firstReader.getSequentialSubReaders(); List<AtomicReaderContext> leaves = firstReader.leaves();
List<? extends AtomicReader> sequentialSubReaders2 = secondReader.getSequentialSubReaders(); List<AtomicReaderContext> leaves2 = secondReader.leaves();
for (IndexReader indexReader : sequentialSubReaders) { for (AtomicReaderContext ctx : leaves) {
Terms terms = ((AtomicReader) indexReader).terms("body"); Terms terms = ctx.reader().terms("body");
TermsEnum iterator = terms.iterator(null); TermsEnum iterator = terms.iterator(null);
IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<DocsEnum, Boolean>(); IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<DocsEnum, Boolean>();
MatchNoBits bits = new Bits.MatchNoBits(firstReader.maxDoc()); MatchNoBits bits = new Bits.MatchNoBits(firstReader.maxDoc());
@ -133,7 +134,7 @@ public class TestReuseDocsEnum extends LuceneTestCase {
DocsEnum docs = null; DocsEnum docs = null;
BytesRef term = null; BytesRef term = null;
while ((term = iterator.next()) != null) { while ((term = iterator.next()) != null) {
docs = iterator.docs(null, randomDocsEnum("body", term, sequentialSubReaders2, bits), random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0); docs = iterator.docs(null, randomDocsEnum("body", term, leaves2, bits), random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
enums.put(docs, true); enums.put(docs, true);
} }
assertEquals(terms.size(), enums.size()); assertEquals(terms.size(), enums.size());
@ -142,7 +143,7 @@ public class TestReuseDocsEnum extends LuceneTestCase {
enums.clear(); enums.clear();
docs = null; docs = null;
while ((term = iterator.next()) != null) { while ((term = iterator.next()) != null) {
docs = iterator.docs(bits, randomDocsEnum("body", term, sequentialSubReaders2, bits), random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0); docs = iterator.docs(bits, randomDocsEnum("body", term, leaves2, bits), random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
enums.put(docs, true); enums.put(docs, true);
} }
assertEquals(terms.size(), enums.size()); assertEquals(terms.size(), enums.size());
@ -150,11 +151,11 @@ public class TestReuseDocsEnum extends LuceneTestCase {
IOUtils.close(writer, firstReader, secondReader, dir); IOUtils.close(writer, firstReader, secondReader, dir);
} }
public DocsEnum randomDocsEnum(String field, BytesRef term, List<? extends AtomicReader> readers, Bits bits) throws IOException { public DocsEnum randomDocsEnum(String field, BytesRef term, List<AtomicReaderContext> readers, Bits bits) throws IOException {
if (random().nextInt(10) == 0) { if (random().nextInt(10) == 0) {
return null; return null;
} }
AtomicReader indexReader = (AtomicReader) readers.get(random().nextInt(readers.size())); AtomicReader indexReader = readers.get(random().nextInt(readers.size())).reader();
return indexReader.termDocsEnum(bits, field, term, random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0); return indexReader.termDocsEnum(bits, field, term, random().nextBoolean() ? DocsEnum.FLAG_FREQS : 0);
} }

View File

@ -143,7 +143,7 @@ public class TestCustomNorms extends LuceneTestCase {
IndexReader reader = writer.getReader(); IndexReader reader = writer.getReader();
writer.close(); writer.close();
assertEquals(numAdded, reader.numDocs()); assertEquals(numAdded, reader.numDocs());
IndexReaderContext topReaderContext = reader.getTopReaderContext(); IndexReaderContext topReaderContext = reader.getContext();
for (final AtomicReaderContext ctx : topReaderContext.leaves()) { for (final AtomicReaderContext ctx : topReaderContext.leaves()) {
AtomicReader atomicReader = ctx.reader(); AtomicReader atomicReader = ctx.reader();
Source source = random().nextBoolean() ? atomicReader.normValues("foo").getSource() : atomicReader.normValues("foo").getDirectSource(); Source source = random().nextBoolean() ? atomicReader.normValues("foo").getSource() : atomicReader.normValues("foo").getDirectSource();

View File

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

View File

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

View File

@ -171,8 +171,8 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
TestDirectoryReader.assertIndexEquals(index1, index2_refreshed); TestDirectoryReader.assertIndexEquals(index1, index2_refreshed);
index2_refreshed.close(); index2_refreshed.close();
assertReaderClosed(index2, true, true); assertReaderClosed(index2, true);
assertReaderClosed(index2_refreshed, true, true); assertReaderClosed(index2_refreshed, true);
index2 = test.openReader(); index2 = test.openReader();
@ -190,28 +190,8 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
index1.close(); index1.close();
index2.close(); index2.close();
assertReaderClosed(index1, true, true); assertReaderClosed(index1, true);
assertReaderClosed(index2, true, true); assertReaderClosed(index2, true);
}
private void performTestsWithExceptionInReopen(TestReopen test) throws Exception {
DirectoryReader index1 = test.openReader();
DirectoryReader index2 = test.openReader();
TestDirectoryReader.assertIndexEquals(index1, index2);
try {
refreshReader(index1, test, 0, true);
fail("Expected exception not thrown.");
} catch (Exception e) {
// expected exception
}
// index2 should still be usable and unaffected by the failed reopen() call
TestDirectoryReader.assertIndexEquals(index1, index2);
index1.close();
index2.close();
} }
public void testThreadSafety() throws Exception { public void testThreadSafety() throws Exception {
@ -355,11 +335,11 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
reader.close(); reader.close();
for (final DirectoryReader readerToClose : readersToClose) { for (final DirectoryReader readerToClose : readersToClose) {
assertReaderClosed(readerToClose, true, true); assertReaderClosed(readerToClose, true);
} }
assertReaderClosed(reader, true, true); assertReaderClosed(reader, true);
assertReaderClosed(firstReader, true, true); assertReaderClosed(firstReader, true);
dir.close(); dir.close();
} }
@ -374,7 +354,7 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
DirectoryReader refreshedReader; DirectoryReader refreshedReader;
} }
private abstract static class ReaderThreadTask { abstract static class ReaderThreadTask {
protected volatile boolean stopped; protected volatile boolean stopped;
public void stop() { public void stop() {
this.stopped = true; this.stopped = true;
@ -384,8 +364,8 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
} }
private static class ReaderThread extends Thread { private static class ReaderThread extends Thread {
private ReaderThreadTask task; ReaderThreadTask task;
private Throwable error; Throwable error;
ReaderThread(ReaderThreadTask task) { ReaderThread(ReaderThreadTask task) {
@ -469,9 +449,9 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
DirectoryReader r = DirectoryReader.open(dir); DirectoryReader r = DirectoryReader.open(dir);
if (multiSegment) { if (multiSegment) {
assertTrue(r.getSequentialSubReaders().size() > 1); assertTrue(r.leaves().size() > 1);
} else { } else {
assertTrue(r.getSequentialSubReaders().size() == 1); assertTrue(r.leaves().size() == 1);
} }
r.close(); r.close();
} }
@ -533,46 +513,25 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
} }
} }
static void assertReaderClosed(IndexReader reader, boolean checkSubReaders, boolean checkNormsClosed) { static void assertReaderClosed(IndexReader reader, boolean checkSubReaders) {
assertEquals(0, reader.getRefCount()); assertEquals(0, reader.getRefCount());
if (checkNormsClosed && reader instanceof AtomicReader) {
// TODO: should we really assert something here? we check for open files and this is obselete...
// assertTrue(((SegmentReader) reader).normsClosed());
}
if (checkSubReaders && reader instanceof CompositeReader) { if (checkSubReaders && reader instanceof CompositeReader) {
// we cannot use reader context here, as reader is
// already closed and calling getTopReaderContext() throws AlreadyClosed!
List<? extends IndexReader> subReaders = ((CompositeReader) reader).getSequentialSubReaders(); List<? extends IndexReader> subReaders = ((CompositeReader) reader).getSequentialSubReaders();
for (IndexReader r : subReaders) { for (final IndexReader r : subReaders) {
assertReaderClosed(r, checkSubReaders, checkNormsClosed); assertReaderClosed(r, checkSubReaders);
} }
} }
} }
/* abstract static class TestReopen {
private void assertReaderOpen(DirectoryReader reader) {
reader.ensureOpen();
if (reader instanceof DirectoryReader) {
DirectoryReader[] subReaders = reader.getSequentialSubReaders();
for (int i = 0; i < subReaders.length; i++) {
assertReaderOpen(subReaders[i]);
}
}
}
*/
private void assertRefCountEquals(int refCount, DirectoryReader reader) {
assertEquals("Reader has wrong refCount value.", refCount, reader.getRefCount());
}
private abstract static class TestReopen {
protected abstract DirectoryReader openReader() throws IOException; protected abstract DirectoryReader openReader() throws IOException;
protected abstract void modifyIndex(int i) throws IOException; protected abstract void modifyIndex(int i) throws IOException;
} }
private static class KeepAllCommits implements IndexDeletionPolicy { static class KeepAllCommits implements IndexDeletionPolicy {
public void onInit(List<? extends IndexCommit> commits) { public void onInit(List<? extends IndexCommit> commits) {
} }
public void onCommit(List<? extends IndexCommit> commits) { public void onCommit(List<? extends IndexCommit> commits) {

View File

@ -204,7 +204,7 @@ public class TestDoc extends LuceneTestCase {
final SegmentInfo si = new SegmentInfo(si1.info.dir, Constants.LUCENE_MAIN_VERSION, merged, -1, false, codec, null, null); final SegmentInfo si = new SegmentInfo(si1.info.dir, Constants.LUCENE_MAIN_VERSION, merged, -1, false, codec, null, null);
SegmentMerger merger = new SegmentMerger(si, InfoStream.getDefault(), trackingDir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL, SegmentMerger merger = new SegmentMerger(si, InfoStream.getDefault(), trackingDir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL,
MergeState.CheckAbort.NONE, null, new FieldInfos.FieldNumbers(), context); MergeState.CheckAbort.NONE, new FieldInfos.FieldNumbers(), context);
merger.add(r1); merger.add(r1);
merger.add(r2); merger.add(r2);

View File

@ -154,11 +154,11 @@ public class TestDocTermOrds extends LuceneTestCase {
System.out.println("TEST: reader=" + r); System.out.println("TEST: reader=" + r);
} }
for(IndexReader subR : r.getSequentialSubReaders()) { for(AtomicReaderContext ctx : r.leaves()) {
if (VERBOSE) { if (VERBOSE) {
System.out.println("\nTEST: sub=" + subR); System.out.println("\nTEST: sub=" + ctx.reader());
} }
verify((AtomicReader) subR, idToOrds, termsArray, null); verify(ctx.reader(), idToOrds, termsArray, null);
} }
// Also test top-level reader: its enum does not support // Also test top-level reader: its enum does not support
@ -273,11 +273,11 @@ public class TestDocTermOrds extends LuceneTestCase {
idToOrdsPrefix[id] = newOrdsArray; idToOrdsPrefix[id] = newOrdsArray;
} }
for(IndexReader subR : r.getSequentialSubReaders()) { for(AtomicReaderContext ctx : r.leaves()) {
if (VERBOSE) { if (VERBOSE) {
System.out.println("\nTEST: sub=" + subR); System.out.println("\nTEST: sub=" + ctx.reader());
} }
verify((AtomicReader) subR, idToOrdsPrefix, termsArray, prefixRef); verify(ctx.reader(), idToOrdsPrefix, termsArray, prefixRef);
} }
// Also test top-level reader: its enum does not support // Also test top-level reader: its enum does not support

View File

@ -91,7 +91,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
writer.close(true); writer.close(true);
DirectoryReader reader = DirectoryReader.open(dir, 1); DirectoryReader reader = DirectoryReader.open(dir, 1);
assertEquals(1, reader.getSequentialSubReaders().size()); assertEquals(1, reader.leaves().size());
IndexSearcher searcher = new IndexSearcher(reader); IndexSearcher searcher = new IndexSearcher(reader);

View File

@ -63,7 +63,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
int num = atLeast(13); int num = atLeast(13);
for (int i = 0; i < num; i++) { for (int i = 0; i < num; i++) {
BytesRef bytes = new BytesRef("1"); BytesRef bytes = new BytesRef("1");
IndexReaderContext topReaderContext = reader.getTopReaderContext(); IndexReaderContext topReaderContext = reader.getContext();
for (AtomicReaderContext atomicReaderContext : topReaderContext.leaves()) { for (AtomicReaderContext atomicReaderContext : topReaderContext.leaves()) {
DocsAndPositionsEnum docsAndPosEnum = getDocsAndPositions( DocsAndPositionsEnum docsAndPosEnum = getDocsAndPositions(
atomicReaderContext.reader(), bytes, null); atomicReaderContext.reader(), bytes, null);
@ -138,7 +138,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
int num = atLeast(13); int num = atLeast(13);
for (int i = 0; i < num; i++) { for (int i = 0; i < num; i++) {
BytesRef bytes = new BytesRef("" + term); BytesRef bytes = new BytesRef("" + term);
IndexReaderContext topReaderContext = reader.getTopReaderContext(); IndexReaderContext topReaderContext = reader.getContext();
for (AtomicReaderContext atomicReaderContext : topReaderContext.leaves()) { for (AtomicReaderContext atomicReaderContext : topReaderContext.leaves()) {
DocsAndPositionsEnum docsAndPosEnum = getDocsAndPositions( DocsAndPositionsEnum docsAndPosEnum = getDocsAndPositions(
atomicReaderContext.reader(), bytes, null); atomicReaderContext.reader(), bytes, null);
@ -214,7 +214,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
int num = atLeast(13); int num = atLeast(13);
for (int i = 0; i < num; i++) { for (int i = 0; i < num; i++) {
BytesRef bytes = new BytesRef("" + term); BytesRef bytes = new BytesRef("" + term);
IndexReaderContext topReaderContext = reader.getTopReaderContext(); IndexReaderContext topReaderContext = reader.getContext();
for (AtomicReaderContext context : topReaderContext.leaves()) { for (AtomicReaderContext context : topReaderContext.leaves()) {
int maxDoc = context.reader().maxDoc(); int maxDoc = context.reader().maxDoc();
DocsEnum docsEnum = _TestUtil.docs(random(), context.reader(), fieldName, bytes, null, null, DocsEnum.FLAG_FREQS); DocsEnum docsEnum = _TestUtil.docs(random(), context.reader(), fieldName, bytes, null, null, DocsEnum.FLAG_FREQS);
@ -292,7 +292,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
for (int i = 0; i < num; i++) { for (int i = 0; i < num; i++) {
BytesRef bytes = new BytesRef("even"); BytesRef bytes = new BytesRef("even");
IndexReaderContext topReaderContext = reader.getTopReaderContext(); IndexReaderContext topReaderContext = reader.getContext();
for (AtomicReaderContext atomicReaderContext : topReaderContext.leaves()) { for (AtomicReaderContext atomicReaderContext : topReaderContext.leaves()) {
DocsAndPositionsEnum docsAndPosEnum = getDocsAndPositions( DocsAndPositionsEnum docsAndPosEnum = getDocsAndPositions(
atomicReaderContext.reader(), bytes, null); atomicReaderContext.reader(), bytes, null);

View File

@ -289,8 +289,9 @@ public class TestFieldsReader extends LuceneTestCase {
assertEquals(numDocs, r.numDocs()); assertEquals(numDocs, r.numDocs());
for(IndexReader sub : r.getSequentialSubReaders()) { for(AtomicReaderContext ctx : r.leaves()) {
final int[] ids = FieldCache.DEFAULT.getInts((AtomicReader) sub, "id", false); final AtomicReader sub = ctx.reader();
final int[] ids = FieldCache.DEFAULT.getInts(sub, "id", false);
for(int docID=0;docID<sub.numDocs();docID++) { for(int docID=0;docID<sub.numDocs();docID++) {
final Document doc = sub.document(docID); final Document doc = sub.document(docID);
final Field f = (Field) doc.getField("nf"); final Field f = (Field) doc.getField("nf");

View File

@ -1498,9 +1498,9 @@ public class TestIndexWriter extends LuceneTestCase {
assertNoUnreferencedFiles(dir, "no tv files"); assertNoUnreferencedFiles(dir, "no tv files");
DirectoryReader r0 = DirectoryReader.open(dir); DirectoryReader r0 = DirectoryReader.open(dir);
for (IndexReader r : r0.getSequentialSubReaders()) { for (AtomicReaderContext ctx : r0.leaves()) {
SegmentInfoPerCommit s = ((SegmentReader) r).getSegmentInfo(); SegmentReader sr = (SegmentReader) ctx.reader();
assertFalse(((SegmentReader) r).getFieldInfos().hasVectors()); assertFalse(sr.getFieldInfos().hasVectors());
} }
r0.close(); r0.close();

View File

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

View File

@ -1293,7 +1293,7 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
assertTrue(reader.numDocs() > 0); assertTrue(reader.numDocs() > 0);
SegmentInfos sis = new SegmentInfos(); SegmentInfos sis = new SegmentInfos();
sis.read(dir); sis.read(dir);
for(AtomicReaderContext context : reader.getTopReaderContext().leaves()) { for(AtomicReaderContext context : reader.leaves()) {
assertFalse(context.reader().getFieldInfos().hasVectors()); assertFalse(context.reader().getFieldInfos().hasVectors());
} }
reader.close(); reader.close();

View File

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

View File

@ -315,7 +315,7 @@ public class TestIndexWriterUnicode extends LuceneTestCase {
IndexReader r = writer.getReader(); IndexReader r = writer.getReader();
// Test each sub-segment // Test each sub-segment
for (AtomicReaderContext ctx : r.getTopReaderContext().leaves()) { for (AtomicReaderContext ctx : r.leaves()) {
checkTermsOrder(ctx.reader(), allTerms, false); checkTermsOrder(ctx.reader(), allTerms, false);
} }
checkTermsOrder(r, allTerms, true); checkTermsOrder(r, allTerms, true);

View File

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

View File

@ -1,271 +0,0 @@
package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.PayloadProcessorProvider.ReaderPayloadProcessor;
import org.apache.lucene.index.PayloadProcessorProvider.PayloadProcessor;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.junit.Test;
public class TestPayloadProcessorProvider extends LuceneTestCase {
private static final class PerDirPayloadProcessor extends PayloadProcessorProvider {
private final Map<Directory, ReaderPayloadProcessor> processors;
public PerDirPayloadProcessor(Map<Directory, ReaderPayloadProcessor> processors) {
this.processors = processors;
}
@Override
public ReaderPayloadProcessor getReaderProcessor(AtomicReader reader) {
if (reader instanceof SegmentReader) {
return processors.get(((SegmentReader) reader).directory());
} else {
throw new UnsupportedOperationException("This shouldnot happen in this test: Reader is no SegmentReader");
}
}
}
private static final class PerTermPayloadProcessor extends ReaderPayloadProcessor {
@Override
public PayloadProcessor getProcessor(String field, BytesRef text) {
// don't process payloads of terms other than "p:p1"
if (!field.equals("p") || !text.bytesEquals(new BytesRef("p1"))) {
return null;
}
// All other terms are processed the same way
return new DeletePayloadProcessor();
}
}
/** deletes the incoming payload */
private static final class DeletePayloadProcessor extends PayloadProcessor {
@Override
public void processPayload(BytesRef payload) {
payload.length = 0;
}
}
private static final class PayloadTokenStream extends TokenStream {
private final PayloadAttribute payload = addAttribute(PayloadAttribute.class);
private final CharTermAttribute term = addAttribute(CharTermAttribute.class);
private boolean called = false;
private String t;
public PayloadTokenStream(String t) {
this.t = t;
}
@Override
public boolean incrementToken() {
if (called) {
return false;
}
called = true;
byte[] p = new byte[] { 1 };
payload.setPayload(new BytesRef(p));
term.append(t);
return true;
}
@Override
public void reset() throws IOException {
super.reset();
called = false;
term.setEmpty();
}
}
private static final int NUM_DOCS = 10;
private void populateDirs(Random random, Directory[] dirs, boolean multipleCommits)
throws IOException {
for (int i = 0; i < dirs.length; i++) {
dirs[i] = newDirectory();
populateDocs(random, dirs[i], multipleCommits);
verifyPayloadExists(dirs[i], "p", new BytesRef("p1"), NUM_DOCS);
verifyPayloadExists(dirs[i], "p", new BytesRef("p2"), NUM_DOCS);
}
}
private void populateDocs(Random random, Directory dir, boolean multipleCommits)
throws IOException {
IndexWriter writer = new IndexWriter(
dir,
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random, MockTokenizer.WHITESPACE, false)).
setMergePolicy(newLogMergePolicy(10))
);
TokenStream payloadTS1 = new PayloadTokenStream("p1");
TokenStream payloadTS2 = new PayloadTokenStream("p2");
FieldType customType = new FieldType(TextField.TYPE_NOT_STORED);
customType.setOmitNorms(true);
for (int i = 0; i < NUM_DOCS; i++) {
Document doc = new Document();
doc.add(newField("id", "doc" + i, customType));
doc.add(newTextField("content", "doc content " + i, Field.Store.NO));
if (random.nextBoolean()) {
doc.add(new TextField("p", payloadTS1));
doc.add(new TextField("p", payloadTS2));
} else {
FieldType type = new FieldType(TextField.TYPE_NOT_STORED);
type.setStoreTermVectors(true);
type.setStoreTermVectorPositions(true);
type.setStoreTermVectorPayloads(true);
type.setStoreTermVectorOffsets(random.nextBoolean());
doc.add(new Field("p", payloadTS1, type));
doc.add(new Field("p", payloadTS2, type));
}
writer.addDocument(doc);
if (multipleCommits && (i % 4 == 0)) {
writer.commit();
}
}
writer.close();
}
private void verifyPayloadExists(Directory dir, String field, BytesRef text, int numExpected)
throws IOException {
IndexReader reader = DirectoryReader.open(dir);
try {
int numPayloads = 0;
DocsAndPositionsEnum tpe = MultiFields.getTermPositionsEnum(reader, null, field, text);
while (tpe.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
tpe.nextPosition();
BytesRef payload = tpe.getPayload();
if (payload != null) {
assertEquals(1, payload.length);
assertEquals(1, payload.bytes[0]);
++numPayloads;
}
}
assertEquals(numExpected, numPayloads);
} finally {
reader.close();
}
}
private void doTest(Random random, boolean addToEmptyIndex,
int numExpectedPayloads, boolean multipleCommits) throws IOException {
Directory[] dirs = new Directory[2];
populateDirs(random, dirs, multipleCommits);
Directory dir = newDirectory();
if (!addToEmptyIndex) {
populateDocs(random, dir, multipleCommits);
verifyPayloadExists(dir, "p", new BytesRef("p1"), NUM_DOCS);
verifyPayloadExists(dir, "p", new BytesRef("p2"), NUM_DOCS);
}
// Add two source dirs. By not adding the dest dir, we ensure its payloads
// won't get processed.
Map<Directory, ReaderPayloadProcessor> processors = new HashMap<Directory, ReaderPayloadProcessor>();
for (Directory d : dirs) {
processors.put(d, new PerTermPayloadProcessor());
}
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random, MockTokenizer.WHITESPACE, false)));
writer.setPayloadProcessorProvider(new PerDirPayloadProcessor(processors));
IndexReader[] readers = new IndexReader[dirs.length];
for (int i = 0; i < readers.length; i++) {
readers[i] = DirectoryReader.open(dirs[i]);
}
try {
writer.addIndexes(readers);
} finally {
for (IndexReader r : readers) {
r.close();
}
}
writer.close();
verifyPayloadExists(dir, "p", new BytesRef("p1"), numExpectedPayloads);
// the second term should always have all payloads
numExpectedPayloads = NUM_DOCS * dirs.length
+ (addToEmptyIndex ? 0 : NUM_DOCS);
verifyPayloadExists(dir, "p", new BytesRef("p2"), numExpectedPayloads);
for (Directory d : dirs)
d.close();
dir.close();
}
@Test
public void testAddIndexes() throws Exception {
// addIndexes - single commit in each
doTest(random(), true, 0, false);
// addIndexes - multiple commits in each
doTest(random(), true, 0, true);
}
@Test
public void testAddIndexesIntoExisting() throws Exception {
// addIndexes - single commit in each
doTest(random(), false, NUM_DOCS, false);
// addIndexes - multiple commits in each
doTest(random(), false, NUM_DOCS, true);
}
@Test
public void testRegularMerges() throws Exception {
Directory dir = newDirectory();
populateDocs(random(), dir, true);
verifyPayloadExists(dir, "p", new BytesRef("p1"), NUM_DOCS);
verifyPayloadExists(dir, "p", new BytesRef("p2"), NUM_DOCS);
// Add two source dirs. By not adding the dest dir, we ensure its payloads
// won't get processed.
Map<Directory, ReaderPayloadProcessor> processors = new HashMap<Directory, ReaderPayloadProcessor>();
processors.put(dir, new PerTermPayloadProcessor());
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false)));
writer.setPayloadProcessorProvider(new PerDirPayloadProcessor(processors));
writer.forceMerge(1);
writer.close();
verifyPayloadExists(dir, "p", new BytesRef("p1"), 0);
verifyPayloadExists(dir, "p", new BytesRef("p2"), NUM_DOCS);
dir.close();
}
}

View File

@ -604,7 +604,7 @@ public class TestPayloads extends LuceneTestCase {
field.setTokenStream(ts); field.setTokenStream(ts);
writer.addDocument(doc); writer.addDocument(doc);
DirectoryReader reader = writer.getReader(); DirectoryReader reader = writer.getReader();
AtomicReader sr = reader.getSequentialSubReaders().get(0); AtomicReader sr = SlowCompositeReaderWrapper.wrap(reader);
DocsAndPositionsEnum de = sr.termPositionsEnum(null, "field", new BytesRef("withPayload")); DocsAndPositionsEnum de = sr.termPositionsEnum(null, "field", new BytesRef("withPayload"));
de.nextDoc(); de.nextDoc();
de.nextPosition(); de.nextPosition();

View File

@ -289,9 +289,9 @@ public class TestPostingsOffsets extends LuceneTestCase {
w.close(); w.close();
final String[] terms = new String[] {"a", "b", "c", "d"}; final String[] terms = new String[] {"a", "b", "c", "d"};
for(IndexReader reader : r.getSequentialSubReaders()) { for(AtomicReaderContext ctx : r.leaves()) {
// TODO: improve this // TODO: improve this
AtomicReader sub = (AtomicReader) reader; AtomicReader sub = ctx.reader();
//System.out.println("\nsub=" + sub); //System.out.println("\nsub=" + sub);
final TermsEnum termsEnum = sub.fields().terms("content").iterator(null); final TermsEnum termsEnum = sub.fields().terms("content").iterator(null);
DocsEnum docs = null; DocsEnum docs = null;

View File

@ -25,7 +25,6 @@ import java.util.Set;
import java.util.TreeSet; import java.util.TreeSet;
import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.MergedIterator;
import org.apache.lucene.util._TestUtil; import org.apache.lucene.util._TestUtil;
public class TestPrefixCodedTerms extends LuceneTestCase { public class TestPrefixCodedTerms extends LuceneTestCase {
@ -70,14 +69,10 @@ public class TestPrefixCodedTerms extends LuceneTestCase {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public void testMergeEmpty() { public void testMergeEmpty() {
List<Iterator<Term>> subs = Collections.emptyList(); Iterator<Term> merged = new MergedIterator<Term>();
Iterator<Term> merged = new MergedIterator<Term>(subs.toArray(new Iterator[0]));
assertFalse(merged.hasNext()); assertFalse(merged.hasNext());
subs = new ArrayList<Iterator<Term>>(); merged = new MergedIterator<Term>(new PrefixCodedTerms.Builder().finish().iterator(), new PrefixCodedTerms.Builder().finish().iterator());
subs.add(new PrefixCodedTerms.Builder().finish().iterator());
subs.add(new PrefixCodedTerms.Builder().finish().iterator());
merged = new MergedIterator<Term>(subs.toArray(new Iterator[0]));
assertFalse(merged.hasNext()); assertFalse(merged.hasNext());
} }
@ -93,18 +88,14 @@ public class TestPrefixCodedTerms extends LuceneTestCase {
b2.add(t2); b2.add(t2);
PrefixCodedTerms pb2 = b2.finish(); PrefixCodedTerms pb2 = b2.finish();
List<Iterator<Term>> subs = new ArrayList<Iterator<Term>>(); Iterator<Term> merged = new MergedIterator<Term>(pb1.iterator(), pb2.iterator());
subs.add(pb1.iterator());
subs.add(pb2.iterator());
Iterator<Term> merged = new MergedIterator<Term>(subs.toArray(new Iterator[0]));
assertTrue(merged.hasNext()); assertTrue(merged.hasNext());
assertEquals(t1, merged.next()); assertEquals(t1, merged.next());
assertTrue(merged.hasNext()); assertTrue(merged.hasNext());
assertEquals(t2, merged.next()); assertEquals(t2, merged.next());
} }
@SuppressWarnings("unchecked") @SuppressWarnings({"unchecked","rawtypes"})
public void testMergeRandom() { public void testMergeRandom() {
PrefixCodedTerms pb[] = new PrefixCodedTerms[_TestUtil.nextInt(random(), 2, 10)]; PrefixCodedTerms pb[] = new PrefixCodedTerms[_TestUtil.nextInt(random(), 2, 10)];
Set<Term> superSet = new TreeSet<Term>(); Set<Term> superSet = new TreeSet<Term>();

View File

@ -82,7 +82,7 @@ public class TestSegmentMerger extends LuceneTestCase {
final SegmentInfo si = new SegmentInfo(mergedDir, Constants.LUCENE_MAIN_VERSION, mergedSegment, -1, false, codec, null, null); final SegmentInfo si = new SegmentInfo(mergedDir, Constants.LUCENE_MAIN_VERSION, mergedSegment, -1, false, codec, null, null);
SegmentMerger merger = new SegmentMerger(si, InfoStream.getDefault(), mergedDir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL, SegmentMerger merger = new SegmentMerger(si, InfoStream.getDefault(), mergedDir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL,
MergeState.CheckAbort.NONE, null, new FieldInfos.FieldNumbers(), newIOContext(random())); MergeState.CheckAbort.NONE, new FieldInfos.FieldNumbers(), newIOContext(random()));
merger.add(reader1); merger.add(reader1);
merger.add(reader2); merger.add(reader2);
MergeState mergeState = merger.merge(); MergeState mergeState = merger.merge();

View File

@ -281,10 +281,10 @@ public class TestStressIndexing2 extends LuceneTestCase {
} }
private static void printDocs(DirectoryReader r) throws Throwable { private static void printDocs(DirectoryReader r) throws Throwable {
List<? extends AtomicReader> subs = r.getSequentialSubReaders(); for(AtomicReaderContext ctx : r.leaves()) {
for(IndexReader sub : subs) {
// TODO: improve this // TODO: improve this
Bits liveDocs = ((AtomicReader)sub).getLiveDocs(); AtomicReader sub = ctx.reader();
Bits liveDocs = sub.getLiveDocs();
System.out.println(" " + ((SegmentReader) sub).getSegmentInfo()); System.out.println(" " + ((SegmentReader) sub).getSegmentInfo());
for(int docID=0;docID<sub.maxDoc();docID++) { for(int docID=0;docID<sub.maxDoc();docID++) {
Document doc = sub.document(docID); Document doc = sub.document(docID);

View File

@ -184,9 +184,9 @@ public class TestTermVectorsReader extends LuceneTestCase {
public void test() throws IOException { public void test() throws IOException {
//Check to see the files were created properly in setup //Check to see the files were created properly in setup
DirectoryReader reader = DirectoryReader.open(dir); DirectoryReader reader = DirectoryReader.open(dir);
for (IndexReader r : reader.getSequentialSubReaders()) { for (AtomicReaderContext ctx : reader.leaves()) {
SegmentInfoPerCommit s = ((SegmentReader) r).getSegmentInfo(); SegmentReader sr = (SegmentReader) ctx.reader();
assertTrue(((SegmentReader) r).getFieldInfos().hasVectors()); assertTrue(sr.getFieldInfos().hasVectors());
} }
reader.close(); reader.close();
} }

View File

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

View File

@ -120,8 +120,8 @@ public class TestTypePromotion extends LuceneTestCase {
private void assertValues(TestType type, Directory dir, long[] values, Type[] sourceType) private void assertValues(TestType type, Directory dir, long[] values, Type[] sourceType)
throws IOException { throws IOException {
DirectoryReader reader = DirectoryReader.open(dir); DirectoryReader reader = DirectoryReader.open(dir);
assertEquals(1, reader.getSequentialSubReaders().size()); assertEquals(1, reader.leaves().size());
IndexReaderContext topReaderContext = reader.getTopReaderContext(); IndexReaderContext topReaderContext = reader.getContext();
List<AtomicReaderContext> leaves = topReaderContext.leaves(); List<AtomicReaderContext> leaves = topReaderContext.leaves();
assertEquals(1, leaves.size()); assertEquals(1, leaves.size());
DocValues docValues = leaves.get(0).reader().docValues("promote"); DocValues docValues = leaves.get(0).reader().docValues("promote");
@ -371,8 +371,8 @@ public class TestTypePromotion extends LuceneTestCase {
writer.forceMerge(1); writer.forceMerge(1);
writer.close(); writer.close();
DirectoryReader reader = DirectoryReader.open(dir); DirectoryReader reader = DirectoryReader.open(dir);
assertEquals(1, reader.getSequentialSubReaders().size()); assertEquals(1, reader.leaves().size());
IndexReaderContext topReaderContext = reader.getTopReaderContext(); IndexReaderContext topReaderContext = reader.getContext();
List<AtomicReaderContext> leaves = topReaderContext.leaves(); List<AtomicReaderContext> leaves = topReaderContext.leaves();
DocValues docValues = leaves.get(0).reader().docValues("promote"); DocValues docValues = leaves.get(0).reader().docValues("promote");
assertNotNull(docValues); assertNotNull(docValues);

View File

@ -43,7 +43,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
writer.close(); writer.close();
IndexReader reader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir)); IndexReader reader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
AtomicReaderContext context = (AtomicReaderContext) reader.getTopReaderContext(); AtomicReaderContext context = (AtomicReaderContext) reader.getContext();
MockFilter filter = new MockFilter(); MockFilter filter = new MockFilter();
CachingWrapperFilter cacher = new CachingWrapperFilter(filter); CachingWrapperFilter cacher = new CachingWrapperFilter(filter);
@ -69,7 +69,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
writer.close(); writer.close();
IndexReader reader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir)); IndexReader reader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
AtomicReaderContext context = (AtomicReaderContext) reader.getTopReaderContext(); AtomicReaderContext context = (AtomicReaderContext) reader.getContext();
final Filter filter = new Filter() { final Filter filter = new Filter() {
@Override @Override
@ -92,7 +92,7 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
writer.close(); writer.close();
IndexReader reader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir)); IndexReader reader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
AtomicReaderContext context = (AtomicReaderContext) reader.getTopReaderContext(); AtomicReaderContext context = (AtomicReaderContext) reader.getContext();
final Filter filter = new Filter() { final Filter filter = new Filter() {
@Override @Override
@ -115,8 +115,8 @@ public class TestCachingWrapperFilter extends LuceneTestCase {
} }
private static void assertDocIdSetCacheable(IndexReader reader, Filter filter, boolean shouldCacheable) throws IOException { private static void assertDocIdSetCacheable(IndexReader reader, Filter filter, boolean shouldCacheable) throws IOException {
assertTrue(reader.getTopReaderContext() instanceof AtomicReaderContext); assertTrue(reader.getContext() instanceof AtomicReaderContext);
AtomicReaderContext context = (AtomicReaderContext) reader.getTopReaderContext(); AtomicReaderContext context = (AtomicReaderContext) reader.getContext();
final CachingWrapperFilter cacher = new CachingWrapperFilter(filter); final CachingWrapperFilter cacher = new CachingWrapperFilter(filter);
final DocIdSet originalSet = filter.getDocIdSet(context, context.reader().getLiveDocs()); final DocIdSet originalSet = filter.getDocIdSet(context, context.reader().getLiveDocs());
final DocIdSet cachedSet = cacher.getDocIdSet(context, context.reader().getLiveDocs()); final DocIdSet cachedSet = cacher.getDocIdSet(context, context.reader().getLiveDocs());

View File

@ -197,7 +197,7 @@ public class TestNumericRangeQuery32 extends LuceneTestCase {
@Test @Test
public void testInverseRange() throws Exception { public void testInverseRange() throws Exception {
AtomicReaderContext context = SlowCompositeReaderWrapper.wrap(reader).getTopReaderContext(); AtomicReaderContext context = SlowCompositeReaderWrapper.wrap(reader).getContext();
NumericRangeFilter<Integer> f = NumericRangeFilter.newIntRange("field8", 8, 1000, -1000, true, true); NumericRangeFilter<Integer> f = NumericRangeFilter.newIntRange("field8", 8, 1000, -1000, true, true);
assertSame("A inverse range should return the EMPTY_DOCIDSET instance", DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(context, context.reader().getLiveDocs())); assertSame("A inverse range should return the EMPTY_DOCIDSET instance", DocIdSet.EMPTY_DOCIDSET, f.getDocIdSet(context, context.reader().getLiveDocs()));
f = NumericRangeFilter.newIntRange("field8", 8, Integer.MAX_VALUE, null, false, false); f = NumericRangeFilter.newIntRange("field8", 8, Integer.MAX_VALUE, null, false, false);

View File

@ -211,7 +211,7 @@ public class TestNumericRangeQuery64 extends LuceneTestCase {
@Test @Test
public void testInverseRange() throws Exception { public void testInverseRange() throws Exception {
AtomicReaderContext context = SlowCompositeReaderWrapper.wrap(searcher.getIndexReader()).getTopReaderContext(); AtomicReaderContext context = SlowCompositeReaderWrapper.wrap(searcher.getIndexReader()).getContext();
NumericRangeFilter<Long> f = NumericRangeFilter.newLongRange("field8", 8, 1000L, -1000L, true, true); NumericRangeFilter<Long> f = NumericRangeFilter.newLongRange("field8", 8, 1000L, -1000L, true, true);
assertSame("A inverse range should return the EMPTY_DOCIDSET instance", DocIdSet.EMPTY_DOCIDSET, assertSame("A inverse range should return the EMPTY_DOCIDSET instance", DocIdSet.EMPTY_DOCIDSET,
f.getDocIdSet(context, context.reader().getLiveDocs())); f.getDocIdSet(context, context.reader().getLiveDocs()));

View File

@ -22,8 +22,8 @@ import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import org.apache.lucene.index.CompositeReader;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.index.MultiFields; import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.MultiReader; import org.apache.lucene.index.MultiReader;
import org.apache.lucene.index.Term; import org.apache.lucene.index.Term;
@ -310,13 +310,11 @@ public class TestShardSearching extends ShardSearchingTestBase {
final int numNodes = shardSearcher.nodeVersions.length; final int numNodes = shardSearcher.nodeVersions.length;
int[] base = new int[numNodes]; int[] base = new int[numNodes];
final List<? extends IndexReader> subs = ((CompositeReader) mockSearcher.getIndexReader()).getSequentialSubReaders(); final List<IndexReaderContext> subs = mockSearcher.getTopReaderContext().children();
assertEquals(numNodes, subs.size()); assertEquals(numNodes, subs.size());
int docCount = 0;
for(int nodeID=0;nodeID<numNodes;nodeID++) { for(int nodeID=0;nodeID<numNodes;nodeID++) {
base[nodeID] = docCount; base[nodeID] = subs.get(nodeID).docBaseInParent;
docCount += subs.get(nodeID).maxDoc();
} }
if (VERBOSE) { if (VERBOSE) {

View File

@ -67,12 +67,12 @@ public class TestPayloadSpans extends LuceneTestCase {
SpanTermQuery stq; SpanTermQuery stq;
Spans spans; Spans spans;
stq = new SpanTermQuery(new Term(PayloadHelper.FIELD, "seventy")); stq = new SpanTermQuery(new Term(PayloadHelper.FIELD, "seventy"));
spans = MultiSpansWrapper.wrap(indexReader.getTopReaderContext(), stq); spans = MultiSpansWrapper.wrap(indexReader.getContext(), stq);
assertTrue("spans is null and it shouldn't be", spans != null); assertTrue("spans is null and it shouldn't be", spans != null);
checkSpans(spans, 100, 1, 1, 1); checkSpans(spans, 100, 1, 1, 1);
stq = new SpanTermQuery(new Term(PayloadHelper.NO_PAYLOAD_FIELD, "seventy")); stq = new SpanTermQuery(new Term(PayloadHelper.NO_PAYLOAD_FIELD, "seventy"));
spans = MultiSpansWrapper.wrap(indexReader.getTopReaderContext(), stq); spans = MultiSpansWrapper.wrap(indexReader.getContext(), stq);
assertTrue("spans is null and it shouldn't be", spans != null); assertTrue("spans is null and it shouldn't be", spans != null);
checkSpans(spans, 100, 0, 0, 0); checkSpans(spans, 100, 0, 0, 0);
} }
@ -83,7 +83,7 @@ public class TestPayloadSpans extends LuceneTestCase {
SpanFirstQuery sfq; SpanFirstQuery sfq;
match = new SpanTermQuery(new Term(PayloadHelper.FIELD, "one")); match = new SpanTermQuery(new Term(PayloadHelper.FIELD, "one"));
sfq = new SpanFirstQuery(match, 2); sfq = new SpanFirstQuery(match, 2);
Spans spans = MultiSpansWrapper.wrap(indexReader.getTopReaderContext(), sfq); Spans spans = MultiSpansWrapper.wrap(indexReader.getContext(), sfq);
checkSpans(spans, 109, 1, 1, 1); checkSpans(spans, 109, 1, 1, 1);
//Test more complicated subclause //Test more complicated subclause
SpanQuery[] clauses = new SpanQuery[2]; SpanQuery[] clauses = new SpanQuery[2];
@ -91,11 +91,11 @@ public class TestPayloadSpans extends LuceneTestCase {
clauses[1] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "hundred")); clauses[1] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "hundred"));
match = new SpanNearQuery(clauses, 0, true); match = new SpanNearQuery(clauses, 0, true);
sfq = new SpanFirstQuery(match, 2); sfq = new SpanFirstQuery(match, 2);
checkSpans(MultiSpansWrapper.wrap(indexReader.getTopReaderContext(), sfq), 100, 2, 1, 1); checkSpans(MultiSpansWrapper.wrap(indexReader.getContext(), sfq), 100, 2, 1, 1);
match = new SpanNearQuery(clauses, 0, false); match = new SpanNearQuery(clauses, 0, false);
sfq = new SpanFirstQuery(match, 2); sfq = new SpanFirstQuery(match, 2);
checkSpans(MultiSpansWrapper.wrap(indexReader.getTopReaderContext(), sfq), 100, 2, 1, 1); checkSpans(MultiSpansWrapper.wrap(indexReader.getContext(), sfq), 100, 2, 1, 1);
} }
@ -119,7 +119,7 @@ public class TestPayloadSpans extends LuceneTestCase {
writer.close(); writer.close();
checkSpans(MultiSpansWrapper.wrap(reader.getTopReaderContext(), snq), 1,new int[]{2}); checkSpans(MultiSpansWrapper.wrap(reader.getContext(), snq), 1,new int[]{2});
reader.close(); reader.close();
directory.close(); directory.close();
} }

View File

@ -77,9 +77,9 @@ public class TestLockFactory extends LuceneTestCase {
// exceptions raised: // exceptions raised:
// Verify: NoLockFactory allows two IndexWriters // Verify: NoLockFactory allows two IndexWriters
public void testRAMDirectoryNoLocking() throws IOException { public void testRAMDirectoryNoLocking() throws IOException {
Directory dir = new MockDirectoryWrapper(random(), new RAMDirectory()); MockDirectoryWrapper dir = new MockDirectoryWrapper(random(), new RAMDirectory());
dir.setLockFactory(NoLockFactory.getNoLockFactory()); dir.setLockFactory(NoLockFactory.getNoLockFactory());
dir.setWrapLockFactory(false); // we are gonna explicitly test we get this back
assertTrue("RAMDirectory.setLockFactory did not take", assertTrue("RAMDirectory.setLockFactory did not take",
NoLockFactory.class.isInstance(dir.getLockFactory())); NoLockFactory.class.isInstance(dir.getLockFactory()));

View File

@ -17,7 +17,13 @@ package org.apache.lucene.util.junitcompat;
* limitations under the License. * limitations under the License.
*/ */
import java.io.IOException;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.SingleInstanceLockFactory;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.JUnitCore; import org.junit.runner.JUnitCore;
@ -35,9 +41,38 @@ public class TestFailIfDirectoryNotClosed extends WithNestedTests {
} }
} }
public static class Nested2 extends WithNestedTests.AbstractNestedTest {
public void testDummy() throws IOException {
MockDirectoryWrapper dir = newMockDirectory();
IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, null));
dir.close();
}
}
public static class Nested3 extends WithNestedTests.AbstractNestedTest {
public void testDummy() throws IOException {
MockDirectoryWrapper dir = newMockDirectory();
dir.setLockFactory(new SingleInstanceLockFactory());
IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, null));
dir.close();
}
}
@Test @Test
public void testFailIfDirectoryNotClosed() { public void testFailIfDirectoryNotClosed() {
Result r = JUnitCore.runClasses(Nested1.class); Result r = JUnitCore.runClasses(Nested1.class);
Assert.assertEquals(1, r.getFailureCount()); Assert.assertEquals(1, r.getFailureCount());
} }
@Test
public void testFailIfIndexWriterNotClosed() {
Result r = JUnitCore.runClasses(Nested2.class);
Assert.assertEquals(1, r.getFailureCount());
}
@Test
public void testFailIfIndexWriterNotClosedChangeLockFactory() {
Result r = JUnitCore.runClasses(Nested3.class);
Assert.assertEquals(1, r.getFailureCount());
}
} }