mirror of https://github.com/apache/lucene.git
LUCENE-5569: Rename more locations in test classes and comments
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1627258 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
19cd9a06fd
commit
83cbdfda3d
|
@ -170,7 +170,7 @@ public class CheckIndex {
|
|||
/** Current deletions generation. */
|
||||
public long deletionsGen;
|
||||
|
||||
/** True if we were able to open an AtomicReader on this
|
||||
/** True if we were able to open an LeafReader on this
|
||||
* segment. */
|
||||
public boolean openReaderPassed;
|
||||
|
||||
|
|
|
@ -23,7 +23,7 @@ import org.apache.lucene.store.*;
|
|||
|
||||
/**
|
||||
Instances of this reader type can only
|
||||
be used to get stored fields from the underlying AtomicReaders,
|
||||
be used to get stored fields from the underlying LeafReaders,
|
||||
but it is not possible to directly retrieve postings. To do that, get
|
||||
the {@link LeafReaderContext} for all sub-readers via {@link #leaves()}.
|
||||
Alternatively, you can mimic an {@link LeafReader} (with a serious slowdown),
|
||||
|
|
|
@ -197,7 +197,7 @@ public final class DocValues {
|
|||
}
|
||||
|
||||
// some helpers, for transition from fieldcache apis.
|
||||
// as opposed to the AtomicReader apis (which must be strict for consistency), these are lenient
|
||||
// as opposed to the LeafReader apis (which must be strict for consistency), these are lenient
|
||||
|
||||
/**
|
||||
* Returns NumericDocValues for the reader, or {@link #emptyNumeric()} if it has none.
|
||||
|
|
|
@ -37,7 +37,7 @@ public abstract class FilterDirectoryReader extends DirectoryReader {
|
|||
* Factory class passed to FilterDirectoryReader constructor that allows
|
||||
* subclasses to wrap the filtered DirectoryReader's subreaders. You
|
||||
* can use this to, e.g., wrap the subreaders with specialised
|
||||
* FilterAtomicReader implementations.
|
||||
* FilterLeafReader implementations.
|
||||
*/
|
||||
public static abstract class SubReaderWrapper {
|
||||
|
||||
|
@ -55,7 +55,7 @@ public abstract class FilterDirectoryReader extends DirectoryReader {
|
|||
/**
|
||||
* Wrap one of the parent DirectoryReader's subreaders
|
||||
* @param reader the subreader to wrap
|
||||
* @return a wrapped/filtered AtomicReader
|
||||
* @return a wrapped/filtered LeafReader
|
||||
*/
|
||||
public abstract LeafReader wrap(LeafReader reader);
|
||||
|
||||
|
|
|
@ -25,12 +25,12 @@ import org.apache.lucene.util.AttributeSource;
|
|||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/** A <code>FilterAtomicReader</code> contains another AtomicReader, which it
|
||||
/** A <code>FilterLeafReader</code> contains another LeafReader, which it
|
||||
* uses as its basic source of data, possibly transforming the data along the
|
||||
* way or providing additional functionality. The class
|
||||
* <code>FilterAtomicReader</code> itself simply implements all abstract methods
|
||||
* <code>FilterLeafReader</code> itself simply implements all abstract methods
|
||||
* of <code>IndexReader</code> with versions that pass all requests to the
|
||||
* contained index reader. Subclasses of <code>FilterAtomicReader</code> may
|
||||
* contained index reader. Subclasses of <code>FilterLeafReader</code> may
|
||||
* further override some of these methods and may also provide additional
|
||||
* methods and fields.
|
||||
* <p><b>NOTE</b>: If you override {@link #getLiveDocs()}, you will likely need
|
||||
|
@ -317,12 +317,12 @@ public class FilterLeafReader extends LeafReader {
|
|||
}
|
||||
}
|
||||
|
||||
/** The underlying AtomicReader. */
|
||||
/** The underlying LeafReader. */
|
||||
protected final LeafReader in;
|
||||
|
||||
/**
|
||||
* <p>Construct a FilterAtomicReader based on the specified base reader.
|
||||
* <p>Note that base reader is closed if this FilterAtomicReader is closed.</p>
|
||||
* <p>Construct a FilterLeafReader based on the specified base reader.
|
||||
* <p>Note that base reader is closed if this FilterLeafReader is closed.</p>
|
||||
* @param in specified base reader.
|
||||
*/
|
||||
public FilterLeafReader(LeafReader in) {
|
||||
|
@ -390,7 +390,7 @@ public class FilterLeafReader extends LeafReader {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder buffer = new StringBuilder("FilterAtomicReader(");
|
||||
final StringBuilder buffer = new StringBuilder("FilterLeafReader(");
|
||||
buffer.append(in);
|
||||
buffer.append(')');
|
||||
return buffer.toString();
|
||||
|
|
|
@ -53,7 +53,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
and postings.
|
||||
<li>{@link CompositeReader}: Instances (like {@link DirectoryReader})
|
||||
of this reader can only
|
||||
be used to get stored fields from the underlying AtomicReaders,
|
||||
be used to get stored fields from the underlying LeafReaders,
|
||||
but it is not possible to directly retrieve postings. To do that, get
|
||||
the sub-readers via {@link CompositeReader#getSequentialSubReaders}.
|
||||
Alternatively, you can mimic an {@link LeafReader} (with a serious slowdown),
|
||||
|
@ -88,7 +88,7 @@ public abstract class IndexReader implements Closeable {
|
|||
|
||||
IndexReader() {
|
||||
if (!(this instanceof CompositeReader || this instanceof LeafReader))
|
||||
throw new Error("IndexReader should never be directly extended, subclass AtomicReader or CompositeReader instead.");
|
||||
throw new Error("IndexReader should never be directly extended, subclass LeafReader or CompositeReader instead.");
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.io.IOException;
|
|||
import org.apache.lucene.index.IndexReader.ReaderClosedListener;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
/** {@code AtomicReader} is an abstract class, providing an interface for accessing an
|
||||
/** {@code LeafReader} is an abstract class, providing an interface for accessing an
|
||||
index. Search of an index is done entirely through this abstract interface,
|
||||
so that any subclass which implements it is searchable. IndexReaders implemented
|
||||
by this subclass do not consist of several sub-readers,
|
||||
|
|
|
@ -41,7 +41,7 @@ import org.apache.lucene.util.packed.PackedLongValues;
|
|||
* <p><b>NOTE</b>: for multi readers, you'll get better
|
||||
* performance by gathering the sub readers using
|
||||
* {@link IndexReader#getContext()} to get the
|
||||
* atomic leaves and then operate per-AtomicReader,
|
||||
* atomic leaves and then operate per-LeafReader,
|
||||
* instead of using this class.
|
||||
*
|
||||
* <p><b>NOTE</b>: This is very costly.
|
||||
|
|
|
@ -40,7 +40,7 @@ import org.apache.lucene.util.MergedIterator;
|
|||
* <p><b>NOTE</b>: for composite readers, you'll get better
|
||||
* performance by gathering the sub readers using
|
||||
* {@link IndexReader#getContext()} to get the
|
||||
* atomic leaves and then operate per-AtomicReader,
|
||||
* atomic leaves and then operate per-LeafReader,
|
||||
* instead of using this class.
|
||||
*
|
||||
* @lucene.experimental
|
||||
|
|
|
@ -59,19 +59,19 @@ public class ParallelLeafReader extends LeafReader {
|
|||
private final SortedMap<String,LeafReader> fieldToReader = new TreeMap<>();
|
||||
private final SortedMap<String,LeafReader> tvFieldToReader = new TreeMap<>();
|
||||
|
||||
/** Create a ParallelAtomicReader based on the provided
|
||||
/** Create a ParallelLeafReader based on the provided
|
||||
* readers; auto-closes the given readers on {@link #close()}. */
|
||||
public ParallelLeafReader(LeafReader... readers) throws IOException {
|
||||
this(true, readers);
|
||||
}
|
||||
|
||||
/** Create a ParallelAtomicReader based on the provided
|
||||
/** Create a ParallelLeafReader based on the provided
|
||||
* readers. */
|
||||
public ParallelLeafReader(boolean closeSubReaders, LeafReader... readers) throws IOException {
|
||||
this(closeSubReaders, readers, readers);
|
||||
}
|
||||
|
||||
/** Expert: create a ParallelAtomicReader based on the provided
|
||||
/** Expert: create a ParallelLeafReader based on the provided
|
||||
* readers and storedFieldReaders; when a document is
|
||||
* loaded, only storedFieldsReaders will be used. */
|
||||
public ParallelLeafReader(boolean closeSubReaders, LeafReader[] readers, LeafReader[] storedFieldsReaders) throws IOException {
|
||||
|
@ -141,7 +141,7 @@ public class ParallelLeafReader extends LeafReader {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder buffer = new StringBuilder("ParallelAtomicReader(");
|
||||
final StringBuilder buffer = new StringBuilder("ParallelLeafReader(");
|
||||
for (final Iterator<LeafReader> iter = completeReaderSet.iterator(); iter.hasNext();) {
|
||||
buffer.append(iter.next());
|
||||
if (iter.hasNext()) buffer.append(", ");
|
||||
|
|
|
@ -40,7 +40,7 @@ import org.apache.lucene.index.MultiDocValues.OrdinalMap;
|
|||
* performance hit. If this is important to your use case,
|
||||
* you'll get better performance by gathering the sub readers using
|
||||
* {@link IndexReader#getContext()} to get the
|
||||
* atomic leaves and then operate per-AtomicReader,
|
||||
* atomic leaves and then operate per-LeafReader,
|
||||
* instead of using this class.
|
||||
*/
|
||||
public final class SlowCompositeReaderWrapper extends LeafReader {
|
||||
|
|
|
@ -43,7 +43,7 @@ import java.io.IOException;
|
|||
* final BitSet bits = new BitSet(indexReader.maxDoc());
|
||||
* searcher.search(query, new Collector() {
|
||||
*
|
||||
* public LeafCollector getLeafCollector(AtomicReaderContext context)
|
||||
* public LeafCollector getLeafCollector(LeafReaderContext context)
|
||||
* throws IOException {
|
||||
* final int docBase = context.docBase;
|
||||
* return new LeafCollector() {
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.lucene.util.Bits;
|
|||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
public class TestFilterAtomicReader extends LuceneTestCase {
|
||||
public class TestFilterLeafReader extends LuceneTestCase {
|
||||
|
||||
private static class TestReader extends FilterLeafReader {
|
||||
|
|
@ -383,7 +383,7 @@ public class TestParallelCompositeReader extends LuceneTestCase {
|
|||
ParallelCompositeReader pr = new ParallelCompositeReader(new CompositeReader[] {ir1});
|
||||
|
||||
final String s = pr.toString();
|
||||
assertTrue("toString incorrect: " + s, s.startsWith("ParallelCompositeReader(ParallelAtomicReader("));
|
||||
assertTrue("toString incorrect: " + s, s.startsWith("ParallelCompositeReader(ParallelLeafReader("));
|
||||
|
||||
pr.close();
|
||||
dir1.close();
|
||||
|
@ -395,7 +395,7 @@ public class TestParallelCompositeReader extends LuceneTestCase {
|
|||
ParallelCompositeReader pr = new ParallelCompositeReader(new CompositeReader[] {new MultiReader(ir1)});
|
||||
|
||||
final String s = pr.toString();
|
||||
assertTrue("toString incorrect: " + s, s.startsWith("ParallelCompositeReader(ParallelCompositeReader(ParallelAtomicReader("));
|
||||
assertTrue("toString incorrect: " + s, s.startsWith("ParallelCompositeReader(ParallelCompositeReader(ParallelLeafReader("));
|
||||
|
||||
pr.close();
|
||||
dir1.close();
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
||||
public class TestParallelAtomicReader extends LuceneTestCase {
|
||||
public class TestParallelLeafReader extends LuceneTestCase {
|
||||
|
||||
private IndexSearcher parallel, single;
|
||||
private Directory dir, dir1, dir2;
|
||||
|
@ -128,7 +128,7 @@ public class TestParallelAtomicReader extends LuceneTestCase {
|
|||
|
||||
try {
|
||||
pr.document(0);
|
||||
fail("ParallelAtomicReader should be already closed because inner reader was closed!");
|
||||
fail("ParallelLeafReader should be already closed because inner reader was closed!");
|
||||
} catch (AlreadyClosedException e) {
|
||||
// pass
|
||||
}
|
|
@ -51,10 +51,10 @@ import org.apache.lucene.util.IntsRef;
|
|||
* DirectoryReader reader = DirectoryReader.open(oldDir);
|
||||
* IndexWriterConfig conf = new IndexWriterConfig(VER, ANALYZER);
|
||||
* IndexWriter writer = new IndexWriter(newDir, conf);
|
||||
* List<AtomicReaderContext> leaves = reader.leaves();
|
||||
* AtomicReader wrappedLeaves[] = new AtomicReader[leaves.size()];
|
||||
* List<LeafReaderContext> leaves = reader.leaves();
|
||||
* LeafReader wrappedLeaves[] = new LeafReader[leaves.size()];
|
||||
* for (int i = 0; i < leaves.size(); i++) {
|
||||
* wrappedLeaves[i] = new OrdinalMappingAtomicReader(leaves.get(i).reader(), ordmap);
|
||||
* wrappedLeaves[i] = new OrdinalMappingLeafReader(leaves.get(i).reader(), ordmap);
|
||||
* }
|
||||
* writer.addIndexes(new MultiReader(wrappedLeaves));
|
||||
* writer.commit();
|
||||
|
@ -113,7 +113,7 @@ public class OrdinalMappingLeafReader extends FilterLeafReader {
|
|||
private final Set<String> facetFields;
|
||||
|
||||
/**
|
||||
* Wraps an AtomicReader, mapping ordinals according to the ordinalMap, using
|
||||
* Wraps an LeafReader, mapping ordinals according to the ordinalMap, using
|
||||
* the provided {@link FacetsConfig} which was used to build the wrapped
|
||||
* reader.
|
||||
*/
|
||||
|
|
|
@ -45,7 +45,7 @@ import org.junit.Test;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
public class TestOrdinalMappingAtomicReader extends FacetTestCase {
|
||||
public class TestOrdinalMappingLeafReader extends FacetTestCase {
|
||||
|
||||
private static final int NUM_DOCS = 100;
|
||||
private final FacetsConfig facetConfig = new FacetsConfig();
|
||||
|
@ -126,7 +126,7 @@ public class TestOrdinalMappingAtomicReader extends FacetTestCase {
|
|||
// add a facet under default dim config
|
||||
doc.add(new FacetField("id", Integer.toString(i)));
|
||||
|
||||
// make sure OrdinalMappingAtomicReader ignores non-facet BinaryDocValues fields
|
||||
// make sure OrdinalMappingLeafReader ignores non-facet BinaryDocValues fields
|
||||
doc.add(new BinaryDocValuesField("bdv", new BytesRef(Integer.toString(i))));
|
||||
doc.add(new BinaryDocValuesField("cbdv", new BytesRef(Integer.toString(i*2))));
|
||||
writer.addDocument(facetConfig.build(taxonomyWriter, doc));
|
|
@ -368,7 +368,7 @@ public class WeightedSpanTermExtractor {
|
|||
|
||||
/*
|
||||
* This reader will just delegate every call to a single field in the wrapped
|
||||
* AtomicReader. This way we only need to build this field once rather than
|
||||
* LeafReader. This way we only need to build this field once rather than
|
||||
* N-Times
|
||||
*/
|
||||
static final class DelegatingLeafReader extends FilterLeafReader {
|
||||
|
|
|
@ -146,7 +146,7 @@ public class BlockJoinComparatorSource extends FieldComparatorSource {
|
|||
public FieldComparator<Integer> setNextReader(LeafReaderContext context) throws IOException {
|
||||
final DocIdSet parents = parentsFilter.getDocIdSet(context, null);
|
||||
if (parents == null) {
|
||||
throw new IllegalStateException("AtomicReader " + context.reader() + " contains no parents!");
|
||||
throw new IllegalStateException("LeafReader " + context.reader() + " contains no parents!");
|
||||
}
|
||||
if (!(parents instanceof FixedBitSet)) {
|
||||
throw new IllegalStateException("parentFilter must return FixedBitSet; got " + parents);
|
||||
|
|
|
@ -57,7 +57,7 @@ import org.apache.lucene.util.automaton.CompiledAutomaton;
|
|||
* IndexWriter writer; // writer to which the sorted index will be added
|
||||
* DirectoryReader reader; // reader on the input index
|
||||
* Sort sort; // determines how the documents are sorted
|
||||
* AtomicReader sortingReader = SortingAtomicReader.wrap(SlowCompositeReaderWrapper.wrap(reader), sort);
|
||||
* LeafReader sortingReader = SortingLeafReader.wrap(SlowCompositeReaderWrapper.wrap(reader), sort);
|
||||
* writer.addIndexes(reader);
|
||||
* writer.close();
|
||||
* reader.close();
|
||||
|
|
|
@ -52,7 +52,7 @@ import org.apache.lucene.util.Bits;
|
|||
* The uninversion process happens lazily: upon the first request for the
|
||||
* field's docvalues (e.g. via {@link org.apache.lucene.index.LeafReader#getNumericDocValues(String)}
|
||||
* or similar), it will create the docvalues on-the-fly if needed and cache it,
|
||||
* based on the core cache key of the wrapped AtomicReader.
|
||||
* based on the core cache key of the wrapped LeafReader.
|
||||
*/
|
||||
public class UninvertingReader extends FilterLeafReader {
|
||||
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.junit.BeforeClass;
|
|||
public class SortingLeafReaderTest extends SorterTestBase {
|
||||
|
||||
@BeforeClass
|
||||
public static void beforeClassSortingAtomicReaderTest() throws Exception {
|
||||
public static void beforeClassSortingLeafReaderTest() throws Exception {
|
||||
|
||||
// sort the index by id (as integer, in NUMERIC_DV_FIELD)
|
||||
Sort sort = new Sort(new SortField(NUMERIC_DV_FIELD, SortField.Type.INT));
|
||||
|
|
|
@ -144,7 +144,7 @@ public class ChainedFilter extends Filter {
|
|||
/**
|
||||
* Delegates to each filter in the chain.
|
||||
*
|
||||
* @param context AtomicReaderContext
|
||||
* @param context LeafReaderContext
|
||||
* @param logic Logical operation
|
||||
* @return DocIdSet
|
||||
*/
|
||||
|
@ -161,7 +161,7 @@ public class ChainedFilter extends Filter {
|
|||
/**
|
||||
* Delegates to each filter in the chain.
|
||||
*
|
||||
* @param context AtomicReaderContext
|
||||
* @param context LeafReaderContext
|
||||
* @param logic Logical operation
|
||||
* @return DocIdSet
|
||||
*/
|
||||
|
|
|
@ -143,7 +143,7 @@ public abstract class FunctionValues {
|
|||
// A RangeValueSource can't easily be a ValueSource that takes another ValueSource
|
||||
// because it needs different behavior depending on the type of fields. There is also
|
||||
// a setup cost - parsing and normalizing params, and doing a binary search on the StringIndex.
|
||||
// TODO: change "reader" to AtomicReaderContext
|
||||
// TODO: change "reader" to LeafReaderContext
|
||||
public ValueSourceScorer getRangeScorer(IndexReader reader, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) {
|
||||
float lower;
|
||||
float upper;
|
||||
|
|
|
@ -79,7 +79,7 @@ public abstract class AbstractVisitingPrefixTreeFilter extends AbstractPrefixTre
|
|||
/**
|
||||
* An abstract class designed to make it easy to implement predicates or
|
||||
* other operations on a {@link SpatialPrefixTree} indexed field. An instance
|
||||
* of this class is not designed to be re-used across AtomicReaderContext
|
||||
* of this class is not designed to be re-used across LeafReaderContext
|
||||
* instances so simply create a new one for each call to, say a {@link
|
||||
* org.apache.lucene.search.Filter#getDocIdSet(org.apache.lucene.index.LeafReaderContext, org.apache.lucene.util.Bits)}.
|
||||
* The {@link #getDocIdSet()} method here starts the work. It first checks
|
||||
|
|
|
@ -154,7 +154,7 @@ public final class FieldFilterLeafReader extends FilterLeafReader {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("FieldFilterAtomicReader(reader=");
|
||||
final StringBuilder sb = new StringBuilder("FieldFilterLeafReader(reader=");
|
||||
sb.append(in).append(", fields=");
|
||||
if (negate) sb.append('!');
|
||||
return sb.append(fields).append(')').toString();
|
||||
|
|
|
@ -1431,7 +1431,7 @@ public abstract class LuceneTestCase extends Assert {
|
|||
r = SlowCompositeReaderWrapper.wrap(r);
|
||||
break;
|
||||
case 1:
|
||||
// will create no FC insanity in atomic case, as ParallelAtomicReader has own cache key:
|
||||
// will create no FC insanity in atomic case, as ParallelLeafReader has own cache key:
|
||||
r = (r instanceof LeafReader) ?
|
||||
new ParallelLeafReader((LeafReader) r) :
|
||||
new ParallelCompositeReader((CompositeReader) r);
|
||||
|
@ -1451,7 +1451,7 @@ public abstract class LuceneTestCase extends Assert {
|
|||
Collections.shuffle(allFields, random);
|
||||
final int end = allFields.isEmpty() ? 0 : random.nextInt(allFields.size());
|
||||
final Set<String> fields = new HashSet<>(allFields.subList(0, end));
|
||||
// will create no FC insanity as ParallelAtomicReader has own cache key:
|
||||
// will create no FC insanity as ParallelLeafReader has own cache key:
|
||||
r = new ParallelLeafReader(
|
||||
new FieldFilterLeafReader(ar, fields, false),
|
||||
new FieldFilterLeafReader(ar, fields, true)
|
||||
|
|
|
@ -429,7 +429,7 @@ org.apache.lucene.index.TestDocumentsWriterStallControl=1910,1896,1537,543,2200,
|
|||
org.apache.lucene.index.TestDuelingCodecs=11794,3845,2763,3152,6520,2614,1843
|
||||
org.apache.lucene.index.TestFieldInfos=10,16,19,47,25,8,14
|
||||
org.apache.lucene.index.TestFieldsReader=942,3392,480,688,404,1603,388
|
||||
org.apache.lucene.index.TestFilterAtomicReader=30,41,268,248,20,28,134
|
||||
org.apache.lucene.index.TestFilterLeafReader=30,41,268,248,20,28,134
|
||||
org.apache.lucene.index.TestFlex=145,529,161,594,878,665,65
|
||||
org.apache.lucene.index.TestFlushByRamOrCountsPolicy=5259,3813,4027,3368,5559,1244,4965
|
||||
org.apache.lucene.index.TestForTooMuchCloning=201,234,378,440,92,327,119
|
||||
|
@ -476,7 +476,7 @@ org.apache.lucene.index.TestOmitNorms=379,1609,1493,414,651,1578,1018
|
|||
org.apache.lucene.index.TestOmitPositions=1178,558,273,362,619,373,83
|
||||
org.apache.lucene.index.TestOmitTf=503,300,1123,210,376,208,295
|
||||
org.apache.lucene.index.TestPKIndexSplitter=1876,1437,1779,1962,1522,1627,1592
|
||||
org.apache.lucene.index.TestParallelAtomicReader=373,981,97,186,567,351,258
|
||||
org.apache.lucene.index.TestParallelLeafReader=373,981,97,186,567,351,258
|
||||
org.apache.lucene.index.TestParallelCompositeReader=1119,934,1773,975,678,822,784
|
||||
org.apache.lucene.index.TestParallelReaderEmptyIndex=75,76,150,142,130,342,59
|
||||
org.apache.lucene.index.TestParallelTermEnum=84,12,12,66,67,106,120
|
||||
|
|
|
@ -42,10 +42,10 @@ public class EarlyTerminatingCollectorException extends RuntimeException {
|
|||
* This number represents the sum of:
|
||||
* </p>
|
||||
* <ul>
|
||||
* <li>The total number of documents in all AtomicReaders
|
||||
* <li>The total number of documents in all LeafReaders
|
||||
* that were fully exhausted during collection
|
||||
* </li>
|
||||
* <li>The id of the last doc collected in the last AtomicReader
|
||||
* <li>The id of the last doc collected in the last LeafReader
|
||||
* consulted during collection.
|
||||
* </li>
|
||||
* </ul>
|
||||
|
|
|
@ -199,7 +199,7 @@ public class TestMergePolicyConfig extends SolrTestCaseJ4 {
|
|||
|
||||
/**
|
||||
* Given an IndexReader, asserts that there is at least one AtomcReader leaf,
|
||||
* and that all AtomicReader leaves are SegmentReader's that have a compound
|
||||
* and that all LeafReader leaves are SegmentReader's that have a compound
|
||||
* file status that matches the expected input.
|
||||
*/
|
||||
private static void assertCompoundSegments(IndexReader reader,
|
||||
|
|
Loading…
Reference in New Issue