mirror of https://github.com/apache/lucene.git
LUCENE-5569: *AtomicReader/AtomicReaderContext have been renamed to *LeafReader/LeafReaderContext
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1627178 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
c60e7fe419
commit
85bf318173
|
@ -126,6 +126,9 @@ API Changes
|
|||
extend from IOException.
|
||||
(Ryan Ernst, Robert Muir)
|
||||
|
||||
* LUCENE-5569: *AtomicReader/AtomicReaderContext have been renamed to *LeafReader/LeafReaderContext.
|
||||
(Ryan Ernst)
|
||||
|
||||
Bug Fixes
|
||||
|
||||
* LUCENE-5650: Enforce read-only access to any path outside the temporary
|
||||
|
|
|
@ -22,8 +22,8 @@ import java.util.Random;
|
|||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
|
@ -38,7 +38,6 @@ import org.apache.lucene.util.IOUtils;
|
|||
import org.apache.lucene.util.LineFileDocs;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.junit.BeforeClass;
|
||||
|
||||
// TODO: really this should be in BaseTestPF or somewhere else? useful test!
|
||||
public class TestReuseDocsEnum extends LuceneTestCase {
|
||||
|
@ -53,8 +52,8 @@ public class TestReuseDocsEnum extends LuceneTestCase {
|
|||
writer.commit();
|
||||
|
||||
DirectoryReader open = DirectoryReader.open(dir);
|
||||
for (AtomicReaderContext ctx : open.leaves()) {
|
||||
AtomicReader indexReader = ctx.reader();
|
||||
for (LeafReaderContext ctx : open.leaves()) {
|
||||
LeafReader indexReader = ctx.reader();
|
||||
Terms terms = indexReader.terms("body");
|
||||
TermsEnum iterator = terms.iterator(null);
|
||||
IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<>();
|
||||
|
@ -81,7 +80,7 @@ public class TestReuseDocsEnum extends LuceneTestCase {
|
|||
writer.commit();
|
||||
|
||||
DirectoryReader open = DirectoryReader.open(dir);
|
||||
for (AtomicReaderContext ctx : open.leaves()) {
|
||||
for (LeafReaderContext ctx : open.leaves()) {
|
||||
Terms terms = ctx.reader().terms("body");
|
||||
TermsEnum iterator = terms.iterator(null);
|
||||
IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<>();
|
||||
|
@ -130,10 +129,10 @@ public class TestReuseDocsEnum extends LuceneTestCase {
|
|||
|
||||
DirectoryReader firstReader = DirectoryReader.open(dir);
|
||||
DirectoryReader secondReader = DirectoryReader.open(dir);
|
||||
List<AtomicReaderContext> leaves = firstReader.leaves();
|
||||
List<AtomicReaderContext> leaves2 = secondReader.leaves();
|
||||
List<LeafReaderContext> leaves = firstReader.leaves();
|
||||
List<LeafReaderContext> leaves2 = secondReader.leaves();
|
||||
|
||||
for (AtomicReaderContext ctx : leaves) {
|
||||
for (LeafReaderContext ctx : leaves) {
|
||||
Terms terms = ctx.reader().terms("body");
|
||||
TermsEnum iterator = terms.iterator(null);
|
||||
IdentityHashMap<DocsEnum, Boolean> enums = new IdentityHashMap<>();
|
||||
|
@ -160,11 +159,11 @@ public class TestReuseDocsEnum extends LuceneTestCase {
|
|||
IOUtils.close(firstReader, secondReader, dir);
|
||||
}
|
||||
|
||||
public DocsEnum randomDocsEnum(String field, BytesRef term, List<AtomicReaderContext> readers, Bits bits) throws IOException {
|
||||
public DocsEnum randomDocsEnum(String field, BytesRef term, List<LeafReaderContext> readers, Bits bits) throws IOException {
|
||||
if (random().nextInt(10) == 0) {
|
||||
return null;
|
||||
}
|
||||
AtomicReader indexReader = readers.get(random().nextInt(readers.size())).reader();
|
||||
LeafReader indexReader = readers.get(random().nextInt(readers.size())).reader();
|
||||
Terms terms = indexReader.terms(field);
|
||||
if (terms == null) {
|
||||
return null;
|
||||
|
|
|
@ -992,7 +992,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
for (String name : oldNames) {
|
||||
Directory dir = oldIndexDirs.get(name);
|
||||
DirectoryReader r = DirectoryReader.open(dir);
|
||||
for (AtomicReaderContext context : r.leaves()) {
|
||||
for (LeafReaderContext context : r.leaves()) {
|
||||
air = (SegmentReader) context.reader();
|
||||
Version oldVersion = air.getSegmentInfo().info.getVersion();
|
||||
assertNotNull(oldVersion); // only 3.0 segments can have a null version
|
||||
|
@ -1005,7 +1005,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
|
||||
public void verifyUsesDefaultCodec(Directory dir, String name) throws Exception {
|
||||
DirectoryReader r = DirectoryReader.open(dir);
|
||||
for (AtomicReaderContext context : r.leaves()) {
|
||||
for (LeafReaderContext context : r.leaves()) {
|
||||
SegmentReader air = (SegmentReader) context.reader();
|
||||
Codec codec = air.getSegmentInfo().info.getCodec();
|
||||
assertTrue("codec used in " + name + " (" + codec.getName() + ") is not a default codec (does not begin with Lucene)",
|
||||
|
@ -1229,7 +1229,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
|
||||
public static final String dvUpdatesIndex = "dvupdates.48.zip";
|
||||
|
||||
private void assertNumericDocValues(AtomicReader r, String f, String cf) throws IOException {
|
||||
private void assertNumericDocValues(LeafReader r, String f, String cf) throws IOException {
|
||||
NumericDocValues ndvf = r.getNumericDocValues(f);
|
||||
NumericDocValues ndvcf = r.getNumericDocValues(cf);
|
||||
for (int i = 0; i < r.maxDoc(); i++) {
|
||||
|
@ -1237,7 +1237,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
private void assertBinaryDocValues(AtomicReader r, String f, String cf) throws IOException {
|
||||
private void assertBinaryDocValues(LeafReader r, String f, String cf) throws IOException {
|
||||
BinaryDocValues bdvf = r.getBinaryDocValues(f);
|
||||
BinaryDocValues bdvcf = r.getBinaryDocValues(cf);
|
||||
for (int i = 0; i < r.maxDoc(); i++) {
|
||||
|
@ -1247,8 +1247,8 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
|
||||
private void verifyDocValues(Directory dir) throws IOException {
|
||||
DirectoryReader reader = DirectoryReader.open(dir);
|
||||
for (AtomicReaderContext context : reader.leaves()) {
|
||||
AtomicReader r = context.reader();
|
||||
for (LeafReaderContext context : reader.leaves()) {
|
||||
LeafReader r = context.reader();
|
||||
assertNumericDocValues(r, "ndv1", "ndv1_c");
|
||||
assertNumericDocValues(r, "ndv2", "ndv2_c");
|
||||
assertBinaryDocValues(r, "bdv1", "bdv1_c");
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.classification;
|
|||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.MultiFields;
|
||||
import org.apache.lucene.index.StorableField;
|
||||
import org.apache.lucene.index.StoredDocument;
|
||||
|
@ -34,7 +34,6 @@ import org.apache.lucene.search.ScoreDoc;
|
|||
import org.apache.lucene.search.WildcardQuery;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.IntsRefBuilder;
|
||||
import org.apache.lucene.util.fst.Builder;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
|
@ -80,7 +79,7 @@ public class BooleanPerceptronClassifier implements Classifier<Boolean> {
|
|||
/**
|
||||
* Default constructor, no batch updates of FST, perceptron threshold is
|
||||
* calculated via underlying index metrics during
|
||||
* {@link #train(org.apache.lucene.index.AtomicReader, String, String, org.apache.lucene.analysis.Analyzer)
|
||||
* {@link #train(org.apache.lucene.index.LeafReader, String, String, org.apache.lucene.analysis.Analyzer)
|
||||
* training}
|
||||
*/
|
||||
public BooleanPerceptronClassifier() {
|
||||
|
@ -118,18 +117,18 @@ public class BooleanPerceptronClassifier implements Classifier<Boolean> {
|
|||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void train(AtomicReader atomicReader, String textFieldName,
|
||||
public void train(LeafReader leafReader, String textFieldName,
|
||||
String classFieldName, Analyzer analyzer) throws IOException {
|
||||
train(atomicReader, textFieldName, classFieldName, analyzer, null);
|
||||
train(leafReader, textFieldName, classFieldName, analyzer, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void train(AtomicReader atomicReader, String textFieldName,
|
||||
public void train(LeafReader leafReader, String textFieldName,
|
||||
String classFieldName, Analyzer analyzer, Query query) throws IOException {
|
||||
this.textTerms = MultiFields.getTerms(atomicReader, textFieldName);
|
||||
this.textTerms = MultiFields.getTerms(leafReader, textFieldName);
|
||||
|
||||
if (textTerms == null) {
|
||||
throw new IOException("term vectors need to be available for field " + textFieldName);
|
||||
|
@ -140,7 +139,7 @@ public class BooleanPerceptronClassifier implements Classifier<Boolean> {
|
|||
|
||||
if (threshold == null || threshold == 0d) {
|
||||
// automatic assign a threshold
|
||||
long sumDocFreq = atomicReader.getSumDocFreq(textFieldName);
|
||||
long sumDocFreq = leafReader.getSumDocFreq(textFieldName);
|
||||
if (sumDocFreq != -1) {
|
||||
this.threshold = (double) sumDocFreq / 2d;
|
||||
} else {
|
||||
|
@ -160,7 +159,7 @@ public class BooleanPerceptronClassifier implements Classifier<Boolean> {
|
|||
}
|
||||
updateFST(weights);
|
||||
|
||||
IndexSearcher indexSearcher = new IndexSearcher(atomicReader);
|
||||
IndexSearcher indexSearcher = new IndexSearcher(leafReader);
|
||||
|
||||
int batchCount = 0;
|
||||
|
||||
|
@ -185,7 +184,7 @@ public class BooleanPerceptronClassifier implements Classifier<Boolean> {
|
|||
Boolean correctClass = Boolean.valueOf(field.stringValue());
|
||||
long modifier = correctClass.compareTo(assignedClass);
|
||||
if (modifier != 0) {
|
||||
reuse = updateWeights(atomicReader, reuse, scoreDoc.doc, assignedClass,
|
||||
reuse = updateWeights(leafReader, reuse, scoreDoc.doc, assignedClass,
|
||||
weights, modifier, batchCount % batchSize == 0);
|
||||
}
|
||||
batchCount++;
|
||||
|
@ -194,17 +193,17 @@ public class BooleanPerceptronClassifier implements Classifier<Boolean> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void train(AtomicReader atomicReader, String[] textFieldNames, String classFieldName, Analyzer analyzer, Query query) throws IOException {
|
||||
public void train(LeafReader leafReader, String[] textFieldNames, String classFieldName, Analyzer analyzer, Query query) throws IOException {
|
||||
throw new IOException("training with multiple fields not supported by boolean perceptron classifier");
|
||||
}
|
||||
|
||||
private TermsEnum updateWeights(AtomicReader atomicReader, TermsEnum reuse,
|
||||
private TermsEnum updateWeights(LeafReader leafReader, TermsEnum reuse,
|
||||
int docId, Boolean assignedClass, SortedMap<String,Double> weights,
|
||||
double modifier, boolean updateFST) throws IOException {
|
||||
TermsEnum cte = textTerms.iterator(reuse);
|
||||
|
||||
// get the doc term vectors
|
||||
Terms terms = atomicReader.getTermVector(docId, textFieldName);
|
||||
Terms terms = leafReader.getTermVector(docId, textFieldName);
|
||||
|
||||
if (terms == null) {
|
||||
throw new IOException("term vectors must be stored for field "
|
||||
|
|
|
@ -9,7 +9,7 @@ import java.util.Map;
|
|||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.MultiFields;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
|
@ -60,7 +60,7 @@ public class CachingNaiveBayesClassifier extends SimpleNaiveBayesClassifier {
|
|||
|
||||
/**
|
||||
* Creates a new NaiveBayes classifier with inside caching. Note that you must
|
||||
* call {@link #train(AtomicReader, String, String, Analyzer) train()} before
|
||||
* call {@link #train(org.apache.lucene.index.LeafReader, String, String, Analyzer) train()} before
|
||||
* you can classify any documents. If you want less memory usage you could
|
||||
* call {@link #reInitCache(int, boolean) reInitCache()}.
|
||||
*/
|
||||
|
@ -71,30 +71,30 @@ public class CachingNaiveBayesClassifier extends SimpleNaiveBayesClassifier {
|
|||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void train(AtomicReader atomicReader, String textFieldName, String classFieldName, Analyzer analyzer) throws IOException {
|
||||
train(atomicReader, textFieldName, classFieldName, analyzer, null);
|
||||
public void train(LeafReader leafReader, String textFieldName, String classFieldName, Analyzer analyzer) throws IOException {
|
||||
train(leafReader, textFieldName, classFieldName, analyzer, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void train(AtomicReader atomicReader, String textFieldName, String classFieldName, Analyzer analyzer, Query query) throws IOException {
|
||||
train(atomicReader, new String[]{textFieldName}, classFieldName, analyzer, query);
|
||||
public void train(LeafReader leafReader, String textFieldName, String classFieldName, Analyzer analyzer, Query query) throws IOException {
|
||||
train(leafReader, new String[]{textFieldName}, classFieldName, analyzer, query);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void train(AtomicReader atomicReader, String[] textFieldNames, String classFieldName, Analyzer analyzer, Query query) throws IOException {
|
||||
super.train(atomicReader, textFieldNames, classFieldName, analyzer, query);
|
||||
public void train(LeafReader leafReader, String[] textFieldNames, String classFieldName, Analyzer analyzer, Query query) throws IOException {
|
||||
super.train(leafReader, textFieldNames, classFieldName, analyzer, query);
|
||||
// building the cache
|
||||
reInitCache(0, true);
|
||||
}
|
||||
|
||||
private List<ClassificationResult<BytesRef>> assignClassNormalizedList(String inputDocument) throws IOException {
|
||||
if (atomicReader == null) {
|
||||
if (leafReader == null) {
|
||||
throw new IOException("You must first call Classifier#train");
|
||||
}
|
||||
|
||||
|
@ -241,7 +241,7 @@ public class CachingNaiveBayesClassifier extends SimpleNaiveBayesClassifier {
|
|||
// build the cache for the word
|
||||
Map<String, Long> frequencyMap = new HashMap<>();
|
||||
for (String textFieldName : textFieldNames) {
|
||||
TermsEnum termsEnum = atomicReader.terms(textFieldName).iterator(null);
|
||||
TermsEnum termsEnum = leafReader.terms(textFieldName).iterator(null);
|
||||
while (termsEnum.next() != null) {
|
||||
BytesRef term = termsEnum.term();
|
||||
String termText = term.utf8ToString();
|
||||
|
@ -258,7 +258,7 @@ public class CachingNaiveBayesClassifier extends SimpleNaiveBayesClassifier {
|
|||
}
|
||||
|
||||
// fill the class list
|
||||
Terms terms = MultiFields.getTerms(atomicReader, classFieldName);
|
||||
Terms terms = MultiFields.getTerms(leafReader, classFieldName);
|
||||
TermsEnum termsEnum = terms.iterator(null);
|
||||
while ((termsEnum.next()) != null) {
|
||||
cclasses.add(BytesRef.deepCopyOf(termsEnum.term()));
|
||||
|
@ -267,11 +267,11 @@ public class CachingNaiveBayesClassifier extends SimpleNaiveBayesClassifier {
|
|||
for (BytesRef cclass : cclasses) {
|
||||
double avgNumberOfUniqueTerms = 0;
|
||||
for (String textFieldName : textFieldNames) {
|
||||
terms = MultiFields.getTerms(atomicReader, textFieldName);
|
||||
terms = MultiFields.getTerms(leafReader, textFieldName);
|
||||
long numPostings = terms.getSumDocFreq(); // number of term/doc pairs
|
||||
avgNumberOfUniqueTerms += numPostings / (double) terms.getDocCount();
|
||||
}
|
||||
int docsWithC = atomicReader.docFreq(new Term(classFieldName, cclass));
|
||||
int docsWithC = leafReader.docFreq(new Term(classFieldName, cclass));
|
||||
classTermFreq.put(cclass, avgNumberOfUniqueTerms * docsWithC);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,7 +20,7 @@ import java.io.IOException;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
|
@ -63,39 +63,39 @@ public interface Classifier<T> {
|
|||
/**
|
||||
* Train the classifier using the underlying Lucene index
|
||||
*
|
||||
* @param atomicReader the reader to use to access the Lucene index
|
||||
* @param leafReader the reader to use to access the Lucene index
|
||||
* @param textFieldName the name of the field used to compare documents
|
||||
* @param classFieldName the name of the field containing the class assigned to documents
|
||||
* @param analyzer the analyzer used to tokenize / filter the unseen text
|
||||
* @throws IOException If there is a low-level I/O error.
|
||||
*/
|
||||
public void train(AtomicReader atomicReader, String textFieldName, String classFieldName, Analyzer analyzer)
|
||||
public void train(LeafReader leafReader, String textFieldName, String classFieldName, Analyzer analyzer)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Train the classifier using the underlying Lucene index
|
||||
*
|
||||
* @param atomicReader the reader to use to access the Lucene index
|
||||
* @param leafReader the reader to use to access the Lucene index
|
||||
* @param textFieldName the name of the field used to compare documents
|
||||
* @param classFieldName the name of the field containing the class assigned to documents
|
||||
* @param analyzer the analyzer used to tokenize / filter the unseen text
|
||||
* @param query the query to filter which documents use for training
|
||||
* @throws IOException If there is a low-level I/O error.
|
||||
*/
|
||||
public void train(AtomicReader atomicReader, String textFieldName, String classFieldName, Analyzer analyzer, Query query)
|
||||
public void train(LeafReader leafReader, String textFieldName, String classFieldName, Analyzer analyzer, Query query)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Train the classifier using the underlying Lucene index
|
||||
*
|
||||
* @param atomicReader the reader to use to access the Lucene index
|
||||
* @param leafReader the reader to use to access the Lucene index
|
||||
* @param textFieldNames the names of the fields to be used to compare documents
|
||||
* @param classFieldName the name of the field containing the class assigned to documents
|
||||
* @param analyzer the analyzer used to tokenize / filter the unseen text
|
||||
* @param query the query to filter which documents use for training
|
||||
* @throws IOException If there is a low-level I/O error.
|
||||
*/
|
||||
public void train(AtomicReader atomicReader, String[] textFieldNames, String classFieldName, Analyzer analyzer, Query query)
|
||||
public void train(LeafReader leafReader, String[] textFieldNames, String classFieldName, Analyzer analyzer, Query query)
|
||||
throws IOException;
|
||||
|
||||
}
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
package org.apache.lucene.classification;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.queries.mlt.MoreLikeThis;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
|
@ -166,29 +166,29 @@ public class KNearestNeighborClassifier implements Classifier<BytesRef> {
|
|||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void train(AtomicReader atomicReader, String textFieldName, String classFieldName, Analyzer analyzer) throws IOException {
|
||||
train(atomicReader, textFieldName, classFieldName, analyzer, null);
|
||||
public void train(LeafReader leafReader, String textFieldName, String classFieldName, Analyzer analyzer) throws IOException {
|
||||
train(leafReader, textFieldName, classFieldName, analyzer, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void train(AtomicReader atomicReader, String textFieldName, String classFieldName, Analyzer analyzer, Query query) throws IOException {
|
||||
train(atomicReader, new String[]{textFieldName}, classFieldName, analyzer, query);
|
||||
public void train(LeafReader leafReader, String textFieldName, String classFieldName, Analyzer analyzer, Query query) throws IOException {
|
||||
train(leafReader, new String[]{textFieldName}, classFieldName, analyzer, query);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void train(AtomicReader atomicReader, String[] textFieldNames, String classFieldName, Analyzer analyzer, Query query) throws IOException {
|
||||
public void train(LeafReader leafReader, String[] textFieldNames, String classFieldName, Analyzer analyzer, Query query) throws IOException {
|
||||
this.textFieldNames = textFieldNames;
|
||||
this.classFieldName = classFieldName;
|
||||
mlt = new MoreLikeThis(atomicReader);
|
||||
mlt = new MoreLikeThis(leafReader);
|
||||
mlt.setAnalyzer(analyzer);
|
||||
mlt.setFieldNames(textFieldNames);
|
||||
indexSearcher = new IndexSearcher(atomicReader);
|
||||
indexSearcher = new IndexSearcher(leafReader);
|
||||
if (minDocsFreq > 0) {
|
||||
mlt.setMinDocFreq(minDocsFreq);
|
||||
}
|
||||
|
|
|
@ -26,7 +26,7 @@ import java.util.List;
|
|||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.MultiFields;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
|
@ -48,10 +48,10 @@ import org.apache.lucene.util.BytesRef;
|
|||
public class SimpleNaiveBayesClassifier implements Classifier<BytesRef> {
|
||||
|
||||
/**
|
||||
* {@link org.apache.lucene.index.AtomicReader} used to access the {@link org.apache.lucene.classification.Classifier}'s
|
||||
* {@link org.apache.lucene.index.LeafReader} used to access the {@link org.apache.lucene.classification.Classifier}'s
|
||||
* index
|
||||
*/
|
||||
protected AtomicReader atomicReader;
|
||||
protected LeafReader leafReader;
|
||||
|
||||
/**
|
||||
* names of the fields to be used as input text
|
||||
|
@ -80,7 +80,7 @@ public class SimpleNaiveBayesClassifier implements Classifier<BytesRef> {
|
|||
|
||||
/**
|
||||
* Creates a new NaiveBayes classifier.
|
||||
* Note that you must call {@link #train(AtomicReader, String, String, Analyzer) train()} before you can
|
||||
* Note that you must call {@link #train(org.apache.lucene.index.LeafReader, String, String, Analyzer) train()} before you can
|
||||
* classify any documents.
|
||||
*/
|
||||
public SimpleNaiveBayesClassifier() {
|
||||
|
@ -90,27 +90,27 @@ public class SimpleNaiveBayesClassifier implements Classifier<BytesRef> {
|
|||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void train(AtomicReader atomicReader, String textFieldName, String classFieldName, Analyzer analyzer) throws IOException {
|
||||
train(atomicReader, textFieldName, classFieldName, analyzer, null);
|
||||
public void train(LeafReader leafReader, String textFieldName, String classFieldName, Analyzer analyzer) throws IOException {
|
||||
train(leafReader, textFieldName, classFieldName, analyzer, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void train(AtomicReader atomicReader, String textFieldName, String classFieldName, Analyzer analyzer, Query query)
|
||||
public void train(LeafReader leafReader, String textFieldName, String classFieldName, Analyzer analyzer, Query query)
|
||||
throws IOException {
|
||||
train(atomicReader, new String[]{textFieldName}, classFieldName, analyzer, query);
|
||||
train(leafReader, new String[]{textFieldName}, classFieldName, analyzer, query);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void train(AtomicReader atomicReader, String[] textFieldNames, String classFieldName, Analyzer analyzer, Query query)
|
||||
public void train(LeafReader leafReader, String[] textFieldNames, String classFieldName, Analyzer analyzer, Query query)
|
||||
throws IOException {
|
||||
this.atomicReader = atomicReader;
|
||||
this.indexSearcher = new IndexSearcher(this.atomicReader);
|
||||
this.leafReader = leafReader;
|
||||
this.indexSearcher = new IndexSearcher(this.leafReader);
|
||||
this.textFieldNames = textFieldNames;
|
||||
this.classFieldName = classFieldName;
|
||||
this.analyzer = analyzer;
|
||||
|
@ -155,12 +155,12 @@ public class SimpleNaiveBayesClassifier implements Classifier<BytesRef> {
|
|||
}
|
||||
|
||||
private List<ClassificationResult<BytesRef>> assignClassNormalizedList(String inputDocument) throws IOException {
|
||||
if (atomicReader == null) {
|
||||
if (leafReader == null) {
|
||||
throw new IOException("You must first call Classifier#train");
|
||||
}
|
||||
List<ClassificationResult<BytesRef>> dataList = new ArrayList<>();
|
||||
|
||||
Terms terms = MultiFields.getTerms(atomicReader, classFieldName);
|
||||
Terms terms = MultiFields.getTerms(leafReader, classFieldName);
|
||||
TermsEnum termsEnum = terms.iterator(null);
|
||||
BytesRef next;
|
||||
String[] tokenizedDoc = tokenizeDoc(inputDocument);
|
||||
|
@ -203,7 +203,7 @@ public class SimpleNaiveBayesClassifier implements Classifier<BytesRef> {
|
|||
* @throws IOException if accessing to term vectors or search fails
|
||||
*/
|
||||
protected int countDocsWithClass() throws IOException {
|
||||
int docCount = MultiFields.getTerms(this.atomicReader, this.classFieldName).getDocCount();
|
||||
int docCount = MultiFields.getTerms(this.leafReader, this.classFieldName).getDocCount();
|
||||
if (docCount == -1) { // in case codec doesn't support getDocCount
|
||||
TotalHitCountCollector totalHitCountCollector = new TotalHitCountCollector();
|
||||
BooleanQuery q = new BooleanQuery();
|
||||
|
@ -265,11 +265,11 @@ public class SimpleNaiveBayesClassifier implements Classifier<BytesRef> {
|
|||
private double getTextTermFreqForClass(BytesRef c) throws IOException {
|
||||
double avgNumberOfUniqueTerms = 0;
|
||||
for (String textFieldName : textFieldNames) {
|
||||
Terms terms = MultiFields.getTerms(atomicReader, textFieldName);
|
||||
Terms terms = MultiFields.getTerms(leafReader, textFieldName);
|
||||
long numPostings = terms.getSumDocFreq(); // number of term/doc pairs
|
||||
avgNumberOfUniqueTerms += numPostings / (double) terms.getDocCount(); // avg # of unique terms per doc
|
||||
}
|
||||
int docsWithC = atomicReader.docFreq(new Term(classFieldName, c));
|
||||
int docsWithC = leafReader.docFreq(new Term(classFieldName, c));
|
||||
return avgNumberOfUniqueTerms * docsWithC; // avg # of unique terms in text fields per doc * # docs with c
|
||||
}
|
||||
|
||||
|
@ -294,6 +294,6 @@ public class SimpleNaiveBayesClassifier implements Classifier<BytesRef> {
|
|||
}
|
||||
|
||||
private int docCount(BytesRef countedClass) throws IOException {
|
||||
return atomicReader.docFreq(new Term(classFieldName, countedClass));
|
||||
return leafReader.docFreq(new Term(classFieldName, countedClass));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,7 +22,7 @@ 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.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.StorableField;
|
||||
|
@ -56,7 +56,7 @@ public class DatasetSplitter {
|
|||
/**
|
||||
* Split a given index into 3 indexes for training, test and cross validation tasks respectively
|
||||
*
|
||||
* @param originalIndex an {@link AtomicReader} on the source index
|
||||
* @param originalIndex an {@link org.apache.lucene.index.LeafReader} on the source index
|
||||
* @param trainingIndex a {@link Directory} used to write the training index
|
||||
* @param testIndex a {@link Directory} used to write the test index
|
||||
* @param crossValidationIndex a {@link Directory} used to write the cross validation index
|
||||
|
@ -64,7 +64,7 @@ public class DatasetSplitter {
|
|||
* @param fieldNames names of fields that need to be put in the new indexes or <code>null</code> if all should be used
|
||||
* @throws IOException if any writing operation fails on any of the indexes
|
||||
*/
|
||||
public void split(AtomicReader originalIndex, Directory trainingIndex, Directory testIndex, Directory crossValidationIndex,
|
||||
public void split(LeafReader originalIndex, Directory trainingIndex, Directory testIndex, Directory crossValidationIndex,
|
||||
Analyzer analyzer, String... fieldNames) throws IOException {
|
||||
|
||||
// create IWs for train / test / cv IDXs
|
||||
|
|
|
@ -21,7 +21,7 @@ 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.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.SlowCompositeReaderWrapper;
|
||||
import org.apache.lucene.search.Query;
|
||||
|
@ -83,18 +83,18 @@ public abstract class ClassificationTestBase<T> extends LuceneTestCase {
|
|||
}
|
||||
|
||||
protected void checkCorrectClassification(Classifier<T> classifier, String inputDoc, T expectedResult, Analyzer analyzer, String textFieldName, String classFieldName, Query query) throws Exception {
|
||||
AtomicReader atomicReader = null;
|
||||
LeafReader leafReader = null;
|
||||
try {
|
||||
populateSampleIndex(analyzer);
|
||||
atomicReader = SlowCompositeReaderWrapper.wrap(indexWriter.getReader());
|
||||
classifier.train(atomicReader, textFieldName, classFieldName, analyzer, query);
|
||||
leafReader = SlowCompositeReaderWrapper.wrap(indexWriter.getReader());
|
||||
classifier.train(leafReader, textFieldName, classFieldName, analyzer, query);
|
||||
ClassificationResult<T> classificationResult = classifier.assignClass(inputDoc);
|
||||
assertNotNull(classificationResult.getAssignedClass());
|
||||
assertEquals("got an assigned class of " + classificationResult.getAssignedClass(), expectedResult, classificationResult.getAssignedClass());
|
||||
assertTrue("got a not positive score " + classificationResult.getScore(), classificationResult.getScore() > 0);
|
||||
} finally {
|
||||
if (atomicReader != null)
|
||||
atomicReader.close();
|
||||
if (leafReader != null)
|
||||
leafReader.close();
|
||||
}
|
||||
}
|
||||
protected void checkOnlineClassification(Classifier<T> classifier, String inputDoc, T expectedResult, Analyzer analyzer, String textFieldName, String classFieldName) throws Exception {
|
||||
|
@ -102,11 +102,11 @@ public abstract class ClassificationTestBase<T> extends LuceneTestCase {
|
|||
}
|
||||
|
||||
protected void checkOnlineClassification(Classifier<T> classifier, String inputDoc, T expectedResult, Analyzer analyzer, String textFieldName, String classFieldName, Query query) throws Exception {
|
||||
AtomicReader atomicReader = null;
|
||||
LeafReader leafReader = null;
|
||||
try {
|
||||
populateSampleIndex(analyzer);
|
||||
atomicReader = SlowCompositeReaderWrapper.wrap(indexWriter.getReader());
|
||||
classifier.train(atomicReader, textFieldName, classFieldName, analyzer, query);
|
||||
leafReader = SlowCompositeReaderWrapper.wrap(indexWriter.getReader());
|
||||
classifier.train(leafReader, textFieldName, classFieldName, analyzer, query);
|
||||
ClassificationResult<T> classificationResult = classifier.assignClass(inputDoc);
|
||||
assertNotNull(classificationResult.getAssignedClass());
|
||||
assertEquals("got an assigned class of " + classificationResult.getAssignedClass(), expectedResult, classificationResult.getAssignedClass());
|
||||
|
@ -117,8 +117,8 @@ public abstract class ClassificationTestBase<T> extends LuceneTestCase {
|
|||
assertEquals(Double.valueOf(classificationResult.getScore()), Double.valueOf(secondClassificationResult.getScore()));
|
||||
|
||||
} finally {
|
||||
if (atomicReader != null)
|
||||
atomicReader.close();
|
||||
if (leafReader != null)
|
||||
leafReader.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -199,18 +199,18 @@ public abstract class ClassificationTestBase<T> extends LuceneTestCase {
|
|||
}
|
||||
|
||||
protected void checkPerformance(Classifier<T> classifier, Analyzer analyzer, String classFieldName) throws Exception {
|
||||
AtomicReader atomicReader = null;
|
||||
LeafReader leafReader = null;
|
||||
long trainStart = System.currentTimeMillis();
|
||||
try {
|
||||
populatePerformanceIndex(analyzer);
|
||||
atomicReader = SlowCompositeReaderWrapper.wrap(indexWriter.getReader());
|
||||
classifier.train(atomicReader, textFieldName, classFieldName, analyzer);
|
||||
leafReader = SlowCompositeReaderWrapper.wrap(indexWriter.getReader());
|
||||
classifier.train(leafReader, textFieldName, classFieldName, analyzer);
|
||||
long trainEnd = System.currentTimeMillis();
|
||||
long trainTime = trainEnd - trainStart;
|
||||
assertTrue("training took more than 2 mins : " + trainTime / 1000 + "s", trainTime < 120000);
|
||||
} finally {
|
||||
if (atomicReader != null)
|
||||
atomicReader.close();
|
||||
if (leafReader != null)
|
||||
leafReader.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -23,7 +23,7 @@ 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.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
|
@ -44,7 +44,7 @@ import java.util.Random;
|
|||
*/
|
||||
public class DataSplitterTest extends LuceneTestCase {
|
||||
|
||||
private AtomicReader originalIndex;
|
||||
private LeafReader originalIndex;
|
||||
private RandomIndexWriter indexWriter;
|
||||
private Directory dir;
|
||||
|
||||
|
@ -103,7 +103,7 @@ public class DataSplitterTest extends LuceneTestCase {
|
|||
assertSplit(originalIndex, 0.2, 0.35, idFieldName, textFieldName);
|
||||
}
|
||||
|
||||
public static void assertSplit(AtomicReader originalIndex, double testRatio, double crossValidationRatio, String... fieldNames) throws Exception {
|
||||
public static void assertSplit(LeafReader originalIndex, double testRatio, double crossValidationRatio, String... fieldNames) throws Exception {
|
||||
|
||||
BaseDirectoryWrapper trainingIndex = newDirectory();
|
||||
BaseDirectoryWrapper testIndex = newDirectory();
|
||||
|
|
|
@ -24,7 +24,7 @@ import java.util.Iterator;
|
|||
import java.util.List;
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FilteredTermsEnum;
|
||||
|
@ -137,7 +137,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
if (type == DocValuesType.NUMERIC) {
|
||||
List<NumericDocValues> toMerge = new ArrayList<>();
|
||||
List<Bits> docsWithField = new ArrayList<>();
|
||||
for (AtomicReader reader : mergeState.readers) {
|
||||
for (LeafReader reader : mergeState.readers) {
|
||||
NumericDocValues values = reader.getNumericDocValues(field.name);
|
||||
Bits bits = reader.getDocsWithField(field.name);
|
||||
if (values == null) {
|
||||
|
@ -151,7 +151,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
} else if (type == DocValuesType.BINARY) {
|
||||
List<BinaryDocValues> toMerge = new ArrayList<>();
|
||||
List<Bits> docsWithField = new ArrayList<>();
|
||||
for (AtomicReader reader : mergeState.readers) {
|
||||
for (LeafReader reader : mergeState.readers) {
|
||||
BinaryDocValues values = reader.getBinaryDocValues(field.name);
|
||||
Bits bits = reader.getDocsWithField(field.name);
|
||||
if (values == null) {
|
||||
|
@ -164,7 +164,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
mergeBinaryField(field, mergeState, toMerge, docsWithField);
|
||||
} else if (type == DocValuesType.SORTED) {
|
||||
List<SortedDocValues> toMerge = new ArrayList<>();
|
||||
for (AtomicReader reader : mergeState.readers) {
|
||||
for (LeafReader reader : mergeState.readers) {
|
||||
SortedDocValues values = reader.getSortedDocValues(field.name);
|
||||
if (values == null) {
|
||||
values = DocValues.emptySorted();
|
||||
|
@ -174,7 +174,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
mergeSortedField(field, mergeState, toMerge);
|
||||
} else if (type == DocValuesType.SORTED_SET) {
|
||||
List<SortedSetDocValues> toMerge = new ArrayList<>();
|
||||
for (AtomicReader reader : mergeState.readers) {
|
||||
for (LeafReader reader : mergeState.readers) {
|
||||
SortedSetDocValues values = reader.getSortedSetDocValues(field.name);
|
||||
if (values == null) {
|
||||
values = DocValues.emptySortedSet();
|
||||
|
@ -184,7 +184,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
mergeSortedSetField(field, mergeState, toMerge);
|
||||
} else if (type == DocValuesType.SORTED_NUMERIC) {
|
||||
List<SortedNumericDocValues> toMerge = new ArrayList<>();
|
||||
for (AtomicReader reader : mergeState.readers) {
|
||||
for (LeafReader reader : mergeState.readers) {
|
||||
SortedNumericDocValues values = reader.getSortedNumericDocValues(field.name);
|
||||
if (values == null) {
|
||||
values = DocValues.emptySortedNumeric(reader.maxDoc());
|
||||
|
@ -216,7 +216,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
int docIDUpto;
|
||||
long nextValue;
|
||||
boolean nextHasValue;
|
||||
AtomicReader currentReader;
|
||||
LeafReader currentReader;
|
||||
NumericDocValues currentValues;
|
||||
Bits currentLiveDocs;
|
||||
Bits currentDocsWithField;
|
||||
|
@ -297,7 +297,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
int docIDUpto;
|
||||
BytesRef nextValue;
|
||||
BytesRef nextPointer; // points to null if missing, or nextValue
|
||||
AtomicReader currentReader;
|
||||
LeafReader currentReader;
|
||||
BinaryDocValues currentValues;
|
||||
Bits currentLiveDocs;
|
||||
Bits currentDocsWithField;
|
||||
|
@ -368,7 +368,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
* iterables that filter deleted documents.
|
||||
*/
|
||||
public void mergeSortedNumericField(FieldInfo fieldInfo, final MergeState mergeState, List<SortedNumericDocValues> toMerge) throws IOException {
|
||||
final AtomicReader readers[] = mergeState.readers.toArray(new AtomicReader[toMerge.size()]);
|
||||
final LeafReader readers[] = mergeState.readers.toArray(new LeafReader[toMerge.size()]);
|
||||
final SortedNumericDocValues dvs[] = toMerge.toArray(new SortedNumericDocValues[toMerge.size()]);
|
||||
|
||||
// step 3: add field
|
||||
|
@ -381,7 +381,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
int readerUpto = -1;
|
||||
int docIDUpto;
|
||||
int nextValue;
|
||||
AtomicReader currentReader;
|
||||
LeafReader currentReader;
|
||||
Bits currentLiveDocs;
|
||||
boolean nextIsSet;
|
||||
|
||||
|
@ -444,7 +444,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
int readerUpto = -1;
|
||||
int docIDUpto;
|
||||
long nextValue;
|
||||
AtomicReader currentReader;
|
||||
LeafReader currentReader;
|
||||
Bits currentLiveDocs;
|
||||
boolean nextIsSet;
|
||||
int valueUpto;
|
||||
|
@ -519,14 +519,14 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
* an Iterable that merges ordinals and values and filters deleted documents .
|
||||
*/
|
||||
public void mergeSortedField(FieldInfo fieldInfo, final MergeState mergeState, List<SortedDocValues> toMerge) throws IOException {
|
||||
final AtomicReader readers[] = mergeState.readers.toArray(new AtomicReader[toMerge.size()]);
|
||||
final LeafReader readers[] = mergeState.readers.toArray(new LeafReader[toMerge.size()]);
|
||||
final SortedDocValues dvs[] = toMerge.toArray(new SortedDocValues[toMerge.size()]);
|
||||
|
||||
// step 1: iterate thru each sub and mark terms still in use
|
||||
TermsEnum liveTerms[] = new TermsEnum[dvs.length];
|
||||
long[] weights = new long[liveTerms.length];
|
||||
for (int sub = 0; sub < liveTerms.length; sub++) {
|
||||
AtomicReader reader = readers[sub];
|
||||
LeafReader reader = readers[sub];
|
||||
SortedDocValues dv = dvs[sub];
|
||||
Bits liveDocs = reader.getLiveDocs();
|
||||
if (liveDocs == null) {
|
||||
|
@ -591,7 +591,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
int readerUpto = -1;
|
||||
int docIDUpto;
|
||||
int nextValue;
|
||||
AtomicReader currentReader;
|
||||
LeafReader currentReader;
|
||||
Bits currentLiveDocs;
|
||||
LongValues currentMap;
|
||||
boolean nextIsSet;
|
||||
|
@ -658,14 +658,14 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
* an Iterable that merges ordinals and values and filters deleted documents .
|
||||
*/
|
||||
public void mergeSortedSetField(FieldInfo fieldInfo, final MergeState mergeState, List<SortedSetDocValues> toMerge) throws IOException {
|
||||
final AtomicReader readers[] = mergeState.readers.toArray(new AtomicReader[toMerge.size()]);
|
||||
final LeafReader readers[] = mergeState.readers.toArray(new LeafReader[toMerge.size()]);
|
||||
final SortedSetDocValues dvs[] = toMerge.toArray(new SortedSetDocValues[toMerge.size()]);
|
||||
|
||||
// step 1: iterate thru each sub and mark terms still in use
|
||||
TermsEnum liveTerms[] = new TermsEnum[dvs.length];
|
||||
long[] weights = new long[liveTerms.length];
|
||||
for (int sub = 0; sub < liveTerms.length; sub++) {
|
||||
AtomicReader reader = readers[sub];
|
||||
LeafReader reader = readers[sub];
|
||||
SortedSetDocValues dv = dvs[sub];
|
||||
Bits liveDocs = reader.getLiveDocs();
|
||||
if (liveDocs == null) {
|
||||
|
@ -731,7 +731,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
int readerUpto = -1;
|
||||
int docIDUpto;
|
||||
int nextValue;
|
||||
AtomicReader currentReader;
|
||||
LeafReader currentReader;
|
||||
Bits currentLiveDocs;
|
||||
boolean nextIsSet;
|
||||
|
||||
|
@ -798,7 +798,7 @@ public abstract class DocValuesConsumer implements Closeable {
|
|||
int readerUpto = -1;
|
||||
int docIDUpto;
|
||||
long nextValue;
|
||||
AtomicReader currentReader;
|
||||
LeafReader currentReader;
|
||||
Bits currentLiveDocs;
|
||||
LongValues currentMap;
|
||||
boolean nextIsSet;
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.MappedMultiFields;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
|
@ -91,7 +91,7 @@ public abstract class FieldsConsumer implements Closeable {
|
|||
int docBase = 0;
|
||||
|
||||
for(int readerIndex=0;readerIndex<mergeState.readers.size();readerIndex++) {
|
||||
final AtomicReader reader = mergeState.readers.get(readerIndex);
|
||||
final LeafReader reader = mergeState.readers.get(readerIndex);
|
||||
final Fields f = reader.fields();
|
||||
final int maxDoc = reader.maxDoc();
|
||||
if (f != null) {
|
||||
|
|
|
@ -24,7 +24,7 @@ import java.util.Iterator;
|
|||
import java.util.List;
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
|
@ -75,7 +75,7 @@ public abstract class NormsConsumer implements Closeable {
|
|||
for (FieldInfo field : mergeState.fieldInfos) {
|
||||
if (field.hasNorms()) {
|
||||
List<NumericDocValues> toMerge = new ArrayList<>();
|
||||
for (AtomicReader reader : mergeState.readers) {
|
||||
for (LeafReader reader : mergeState.readers) {
|
||||
NumericDocValues norms = reader.getNormValues(field.name);
|
||||
if (norms == null) {
|
||||
norms = DocValues.emptyNumeric();
|
||||
|
@ -104,7 +104,7 @@ public abstract class NormsConsumer implements Closeable {
|
|||
int readerUpto = -1;
|
||||
int docIDUpto;
|
||||
long nextValue;
|
||||
AtomicReader currentReader;
|
||||
LeafReader currentReader;
|
||||
NumericDocValues currentValues;
|
||||
Bits currentLiveDocs;
|
||||
boolean nextIsSet;
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.apache.lucene.index.MergeState;
|
|||
import org.apache.lucene.index.StorableField;
|
||||
import org.apache.lucene.index.StoredDocument;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
|
||||
/**
|
||||
* Codec API for writing stored fields:
|
||||
|
@ -82,7 +82,7 @@ public abstract class StoredFieldsWriter implements Closeable {
|
|||
* merging (bulk-byte copying, etc). */
|
||||
public int merge(MergeState mergeState) throws IOException {
|
||||
int docCount = 0;
|
||||
for (AtomicReader reader : mergeState.readers) {
|
||||
for (LeafReader reader : mergeState.readers) {
|
||||
final int maxDoc = reader.maxDoc();
|
||||
final Bits liveDocs = reader.getLiveDocs();
|
||||
for (int i = 0; i < maxDoc; i++) {
|
||||
|
|
|
@ -21,7 +21,7 @@ import java.io.Closeable;
|
|||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
|
@ -178,7 +178,7 @@ public abstract class TermVectorsWriter implements Closeable {
|
|||
public int merge(MergeState mergeState) throws IOException {
|
||||
int docCount = 0;
|
||||
for (int i = 0; i < mergeState.readers.size(); i++) {
|
||||
final AtomicReader reader = mergeState.readers.get(i);
|
||||
final LeafReader reader = mergeState.readers.get(i);
|
||||
final int maxDoc = reader.maxDoc();
|
||||
final Bits liveDocs = reader.getLiveDocs();
|
||||
|
||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.lucene.codecs.CodecUtil;
|
|||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.codecs.compressing.CompressingStoredFieldsReader.ChunkIterator;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
|
@ -337,7 +337,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
|
|||
|
||||
MatchingReaders matching = new MatchingReaders(mergeState);
|
||||
|
||||
for (AtomicReader reader : mergeState.readers) {
|
||||
for (LeafReader reader : mergeState.readers) {
|
||||
final SegmentReader matchingSegmentReader = matching.matchingSegmentReaders[idx++];
|
||||
CompressingStoredFieldsReader matchingFieldsReader = null;
|
||||
if (matchingSegmentReader != null) {
|
||||
|
|
|
@ -28,7 +28,7 @@ import java.util.TreeSet;
|
|||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.TermVectorsReader;
|
||||
import org.apache.lucene.codecs.TermVectorsWriter;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.Fields;
|
||||
|
@ -732,7 +732,7 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
|
|||
|
||||
MatchingReaders matching = new MatchingReaders(mergeState);
|
||||
|
||||
for (AtomicReader reader : mergeState.readers) {
|
||||
for (LeafReader reader : mergeState.readers) {
|
||||
final SegmentReader matchingSegmentReader = matching.matchingSegmentReaders[idx++];
|
||||
CompressingTermVectorsReader matchingVectorsReader = null;
|
||||
if (matchingSegmentReader != null) {
|
||||
|
|
|
@ -17,7 +17,7 @@ package org.apache.lucene.codecs.compressing;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
|
@ -50,7 +50,7 @@ class MatchingReaders {
|
|||
// FieldInfos, then we can do a bulk copy of the
|
||||
// stored fields:
|
||||
for (int i = 0; i < numReaders; i++) {
|
||||
AtomicReader reader = mergeState.readers.get(i);
|
||||
LeafReader reader = mergeState.readers.get(i);
|
||||
// TODO: we may be able to broaden this to
|
||||
// non-SegmentReaders, since FieldInfos is now
|
||||
// required? But... this'd also require exposing
|
||||
|
|
|
@ -43,7 +43,7 @@ import org.apache.lucene.util.Accountables;
|
|||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
import static org.apache.lucene.index.FilterAtomicReader.FilterFields;
|
||||
import static org.apache.lucene.index.FilterLeafReader.FilterFields;
|
||||
|
||||
/**
|
||||
* Enables per field postings support.
|
||||
|
|
|
@ -17,14 +17,12 @@ package org.apache.lucene.document;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.AtomicReader; // javadocs
|
||||
|
||||
/**
|
||||
* Syntactic sugar for encoding doubles as NumericDocValues
|
||||
* via {@link Double#doubleToRawLongBits(double)}.
|
||||
* <p>
|
||||
* Per-document double values can be retrieved via
|
||||
* {@link AtomicReader#getNumericDocValues(String)}.
|
||||
* {@link org.apache.lucene.index.LeafReader#getNumericDocValues(String)}.
|
||||
* <p>
|
||||
* <b>NOTE</b>: In most all cases this will be rather inefficient,
|
||||
* requiring eight bytes per document. Consider encoding double
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.document;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.analysis.NumericTokenStream; // javadocs
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.search.NumericRangeFilter; // javadocs
|
||||
import org.apache.lucene.search.NumericRangeQuery; // javadocs
|
||||
|
@ -57,7 +56,7 @@ import org.apache.lucene.util.NumericUtils;
|
|||
* NumericRangeFilter}. To sort according to a
|
||||
* <code>DoubleField</code>, use the normal numeric sort types, eg
|
||||
* {@link org.apache.lucene.search.SortField.Type#DOUBLE}. <code>DoubleField</code>
|
||||
* values can also be loaded directly from {@link AtomicReader#getNumericDocValues}.</p>
|
||||
* values can also be loaded directly from {@link org.apache.lucene.index.LeafReader#getNumericDocValues}.</p>
|
||||
*
|
||||
* <p>You may add the same field name as an <code>DoubleField</code> to
|
||||
* the same document more than once. Range querying and
|
||||
|
|
|
@ -17,14 +17,12 @@ package org.apache.lucene.document;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.AtomicReader; // javadocs
|
||||
|
||||
/**
|
||||
* Syntactic sugar for encoding floats as NumericDocValues
|
||||
* via {@link Float#floatToRawIntBits(float)}.
|
||||
* <p>
|
||||
* Per-document floating point values can be retrieved via
|
||||
* {@link AtomicReader#getNumericDocValues(String)}.
|
||||
* {@link org.apache.lucene.index.LeafReader#getNumericDocValues(String)}.
|
||||
* <p>
|
||||
* <b>NOTE</b>: In most all cases this will be rather inefficient,
|
||||
* requiring four bytes per document. Consider encoding floating
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.document;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.analysis.NumericTokenStream; // javadocs
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.search.NumericRangeFilter; // javadocs
|
||||
import org.apache.lucene.search.NumericRangeQuery; // javadocs
|
||||
|
@ -57,7 +56,7 @@ import org.apache.lucene.util.NumericUtils;
|
|||
* NumericRangeFilter}. To sort according to a
|
||||
* <code>FloatField</code>, use the normal numeric sort types, eg
|
||||
* {@link org.apache.lucene.search.SortField.Type#FLOAT}. <code>FloatField</code>
|
||||
* values can also be loaded directly from {@link AtomicReader#getNumericDocValues}.</p>
|
||||
* values can also be loaded directly from {@link org.apache.lucene.index.LeafReader#getNumericDocValues}.</p>
|
||||
*
|
||||
* <p>You may add the same field name as an <code>FloatField</code> to
|
||||
* the same document more than once. Range querying and
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.document;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.analysis.NumericTokenStream; // javadocs
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.search.NumericRangeFilter; // javadocs
|
||||
import org.apache.lucene.search.NumericRangeQuery; // javadocs
|
||||
|
@ -57,7 +56,7 @@ import org.apache.lucene.util.NumericUtils;
|
|||
* NumericRangeFilter}. To sort according to a
|
||||
* <code>IntField</code>, use the normal numeric sort types, eg
|
||||
* {@link org.apache.lucene.search.SortField.Type#INT}. <code>IntField</code>
|
||||
* values can also be loaded directly from {@link AtomicReader#getNumericDocValues}.</p>
|
||||
* values can also be loaded directly from {@link org.apache.lucene.index.LeafReader#getNumericDocValues}.</p>
|
||||
*
|
||||
* <p>You may add the same field name as an <code>IntField</code> to
|
||||
* the same document more than once. Range querying and
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.document;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.analysis.NumericTokenStream; // javadocs
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.search.NumericRangeFilter; // javadocs
|
||||
import org.apache.lucene.search.NumericRangeQuery; // javadocs
|
||||
|
@ -67,7 +66,7 @@ import org.apache.lucene.util.NumericUtils;
|
|||
* NumericRangeFilter}. To sort according to a
|
||||
* <code>LongField</code>, use the normal numeric sort types, eg
|
||||
* {@link org.apache.lucene.search.SortField.Type#LONG}. <code>LongField</code>
|
||||
* values can also be loaded directly from {@link AtomicReader#getNumericDocValues}.</p>
|
||||
* values can also be loaded directly from {@link org.apache.lucene.index.LeafReader#getNumericDocValues}.</p>
|
||||
*
|
||||
* <p>You may add the same field name as an <code>LongField</code> to
|
||||
* the same document more than once. Range querying and
|
||||
|
|
|
@ -542,7 +542,7 @@ class BufferedUpdatesStream implements Accountable {
|
|||
// Delete by query
|
||||
private static long applyQueryDeletes(Iterable<QueryAndLimit> queriesIter, ReadersAndUpdates rld, final SegmentReader reader) throws IOException {
|
||||
long delCount = 0;
|
||||
final AtomicReaderContext readerContext = reader.getContext();
|
||||
final LeafReaderContext readerContext = reader.getContext();
|
||||
boolean any = false;
|
||||
for (QueryAndLimit ent : queriesIter) {
|
||||
Query query = ent.query;
|
||||
|
|
|
@ -712,7 +712,7 @@ public class CheckIndex {
|
|||
* Test field norms.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public static Status.FieldNormStatus testFieldNorms(AtomicReader reader, PrintStream infoStream, boolean failFast) throws IOException {
|
||||
public static Status.FieldNormStatus testFieldNorms(LeafReader reader, PrintStream infoStream, boolean failFast) throws IOException {
|
||||
final Status.FieldNormStatus status = new Status.FieldNormStatus();
|
||||
|
||||
try {
|
||||
|
@ -1306,7 +1306,7 @@ public class CheckIndex {
|
|||
* Test the term index.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public static Status.TermIndexStatus testPostings(AtomicReader reader, PrintStream infoStream) throws IOException {
|
||||
public static Status.TermIndexStatus testPostings(LeafReader reader, PrintStream infoStream) throws IOException {
|
||||
return testPostings(reader, infoStream, false, false);
|
||||
}
|
||||
|
||||
|
@ -1314,7 +1314,7 @@ public class CheckIndex {
|
|||
* Test the term index.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public static Status.TermIndexStatus testPostings(AtomicReader reader, PrintStream infoStream, boolean verbose, boolean failFast) throws IOException {
|
||||
public static Status.TermIndexStatus testPostings(LeafReader reader, PrintStream infoStream, boolean verbose, boolean failFast) throws IOException {
|
||||
|
||||
// TODO: we should go and verify term vectors match, if
|
||||
// crossCheckTermVectors is on...
|
||||
|
@ -1356,7 +1356,7 @@ public class CheckIndex {
|
|||
* Test stored fields.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public static Status.StoredFieldStatus testStoredFields(AtomicReader reader, PrintStream infoStream, boolean failFast) throws IOException {
|
||||
public static Status.StoredFieldStatus testStoredFields(LeafReader reader, PrintStream infoStream, boolean failFast) throws IOException {
|
||||
final Status.StoredFieldStatus status = new Status.StoredFieldStatus();
|
||||
|
||||
try {
|
||||
|
@ -1401,7 +1401,7 @@ public class CheckIndex {
|
|||
* Test docvalues.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public static Status.DocValuesStatus testDocValues(AtomicReader reader,
|
||||
public static Status.DocValuesStatus testDocValues(LeafReader reader,
|
||||
PrintStream infoStream,
|
||||
boolean failFast) throws IOException {
|
||||
final Status.DocValuesStatus status = new Status.DocValuesStatus();
|
||||
|
@ -1443,7 +1443,7 @@ public class CheckIndex {
|
|||
return status;
|
||||
}
|
||||
|
||||
private static void checkBinaryDocValues(String fieldName, AtomicReader reader, BinaryDocValues dv, Bits docsWithField) {
|
||||
private static void checkBinaryDocValues(String fieldName, LeafReader reader, BinaryDocValues dv, Bits docsWithField) {
|
||||
for (int i = 0; i < reader.maxDoc(); i++) {
|
||||
final BytesRef term = dv.get(i);
|
||||
assert term.isValid();
|
||||
|
@ -1453,7 +1453,7 @@ public class CheckIndex {
|
|||
}
|
||||
}
|
||||
|
||||
private static void checkSortedDocValues(String fieldName, AtomicReader reader, SortedDocValues dv, Bits docsWithField) {
|
||||
private static void checkSortedDocValues(String fieldName, LeafReader reader, SortedDocValues dv, Bits docsWithField) {
|
||||
checkBinaryDocValues(fieldName, reader, dv, docsWithField);
|
||||
final int maxOrd = dv.getValueCount()-1;
|
||||
FixedBitSet seenOrds = new FixedBitSet(dv.getValueCount());
|
||||
|
@ -1493,7 +1493,7 @@ public class CheckIndex {
|
|||
}
|
||||
}
|
||||
|
||||
private static void checkSortedSetDocValues(String fieldName, AtomicReader reader, SortedSetDocValues dv, Bits docsWithField) {
|
||||
private static void checkSortedSetDocValues(String fieldName, LeafReader reader, SortedSetDocValues dv, Bits docsWithField) {
|
||||
final long maxOrd = dv.getValueCount()-1;
|
||||
LongBitSet seenOrds = new LongBitSet(dv.getValueCount());
|
||||
long maxOrd2 = -1;
|
||||
|
@ -1563,7 +1563,7 @@ public class CheckIndex {
|
|||
}
|
||||
}
|
||||
|
||||
private static void checkSortedNumericDocValues(String fieldName, AtomicReader reader, SortedNumericDocValues ndv, Bits docsWithField) {
|
||||
private static void checkSortedNumericDocValues(String fieldName, LeafReader reader, SortedNumericDocValues ndv, Bits docsWithField) {
|
||||
for (int i = 0; i < reader.maxDoc(); i++) {
|
||||
ndv.setDocument(i);
|
||||
int count = ndv.count();
|
||||
|
@ -1587,7 +1587,7 @@ public class CheckIndex {
|
|||
}
|
||||
}
|
||||
|
||||
private static void checkNumericDocValues(String fieldName, AtomicReader reader, NumericDocValues ndv, Bits docsWithField) {
|
||||
private static void checkNumericDocValues(String fieldName, LeafReader reader, NumericDocValues ndv, Bits docsWithField) {
|
||||
for (int i = 0; i < reader.maxDoc(); i++) {
|
||||
long value = ndv.get(i);
|
||||
if (docsWithField.get(i) == false && value != 0) {
|
||||
|
@ -1596,7 +1596,7 @@ public class CheckIndex {
|
|||
}
|
||||
}
|
||||
|
||||
private static void checkDocValues(FieldInfo fi, AtomicReader reader, PrintStream infoStream, DocValuesStatus status) throws Exception {
|
||||
private static void checkDocValues(FieldInfo fi, LeafReader reader, PrintStream infoStream, DocValuesStatus status) throws Exception {
|
||||
Bits docsWithField = reader.getDocsWithField(fi.name);
|
||||
if (docsWithField == null) {
|
||||
throw new RuntimeException(fi.name + " docsWithField does not exist");
|
||||
|
@ -1659,7 +1659,7 @@ public class CheckIndex {
|
|||
}
|
||||
}
|
||||
|
||||
private static void checkNorms(FieldInfo fi, AtomicReader reader, PrintStream infoStream) throws IOException {
|
||||
private static void checkNorms(FieldInfo fi, LeafReader reader, PrintStream infoStream) throws IOException {
|
||||
switch(fi.getNormType()) {
|
||||
case NUMERIC:
|
||||
checkNumericDocValues(fi.name, reader, reader.getNormValues(fi.name), new Bits.MatchAllBits(reader.maxDoc()));
|
||||
|
@ -1673,7 +1673,7 @@ public class CheckIndex {
|
|||
* Test term vectors.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public static Status.TermVectorStatus testTermVectors(AtomicReader reader, PrintStream infoStream) throws IOException {
|
||||
public static Status.TermVectorStatus testTermVectors(LeafReader reader, PrintStream infoStream) throws IOException {
|
||||
return testTermVectors(reader, infoStream, false, false, false);
|
||||
}
|
||||
|
||||
|
@ -1681,7 +1681,7 @@ public class CheckIndex {
|
|||
* Test term vectors.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public static Status.TermVectorStatus testTermVectors(AtomicReader reader, PrintStream infoStream, boolean verbose, boolean crossCheckTermVectors, boolean failFast) throws IOException {
|
||||
public static Status.TermVectorStatus testTermVectors(LeafReader reader, PrintStream infoStream, boolean verbose, boolean crossCheckTermVectors, boolean failFast) throws IOException {
|
||||
final Status.TermVectorStatus status = new Status.TermVectorStatus();
|
||||
final FieldInfos fieldInfos = reader.getFieldInfos();
|
||||
final Bits onlyDocIsDeleted = new FixedBitSet(1);
|
||||
|
|
|
@ -19,15 +19,14 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.search.SearcherManager; // javadocs
|
||||
import org.apache.lucene.store.*;
|
||||
|
||||
/**
|
||||
Instances of this reader type can only
|
||||
be used to get stored fields from the underlying AtomicReaders,
|
||||
but it is not possible to directly retrieve postings. To do that, get
|
||||
the {@link AtomicReaderContext} for all sub-readers via {@link #leaves()}.
|
||||
Alternatively, you can mimic an {@link AtomicReader} (with a serious slowdown),
|
||||
the {@link LeafReaderContext} for all sub-readers via {@link #leaves()}.
|
||||
Alternatively, you can mimic an {@link LeafReader} (with a serious slowdown),
|
||||
by wrapping composite readers with {@link SlowCompositeReaderWrapper}.
|
||||
|
||||
<p>IndexReader instances for indexes on disk are usually constructed
|
||||
|
@ -91,7 +90,7 @@ public abstract class CompositeReader extends IndexReader {
|
|||
* return {@code null}.
|
||||
*
|
||||
* <p><b>NOTE:</b> In contrast to previous Lucene versions this method
|
||||
* is no longer public, code that wants to get all {@link AtomicReader}s
|
||||
* is no longer public, code that wants to get all {@link LeafReader}s
|
||||
* this composite is composed of should use {@link IndexReader#leaves()}.
|
||||
* @see IndexReader#leaves()
|
||||
*/
|
||||
|
|
|
@ -27,7 +27,7 @@ import java.util.List;
|
|||
*/
|
||||
public final class CompositeReaderContext extends IndexReaderContext {
|
||||
private final List<IndexReaderContext> children;
|
||||
private final List<AtomicReaderContext> leaves;
|
||||
private final List<LeafReaderContext> leaves;
|
||||
private final CompositeReader reader;
|
||||
|
||||
static CompositeReaderContext create(CompositeReader reader) {
|
||||
|
@ -46,13 +46,13 @@ public final class CompositeReaderContext extends IndexReaderContext {
|
|||
/**
|
||||
* Creates a {@link CompositeReaderContext} for top-level readers with parent set to <code>null</code>
|
||||
*/
|
||||
CompositeReaderContext(CompositeReader reader, List<IndexReaderContext> children, List<AtomicReaderContext> leaves) {
|
||||
CompositeReaderContext(CompositeReader reader, List<IndexReaderContext> children, List<LeafReaderContext> leaves) {
|
||||
this(null, reader, 0, 0, children, leaves);
|
||||
}
|
||||
|
||||
private CompositeReaderContext(CompositeReaderContext parent, CompositeReader reader,
|
||||
int ordInParent, int docbaseInParent, List<IndexReaderContext> children,
|
||||
List<AtomicReaderContext> leaves) {
|
||||
List<LeafReaderContext> leaves) {
|
||||
super(parent, ordInParent, docbaseInParent);
|
||||
this.children = Collections.unmodifiableList(children);
|
||||
this.leaves = leaves == null ? null : Collections.unmodifiableList(leaves);
|
||||
|
@ -60,7 +60,7 @@ public final class CompositeReaderContext extends IndexReaderContext {
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<AtomicReaderContext> leaves() throws UnsupportedOperationException {
|
||||
public List<LeafReaderContext> leaves() throws UnsupportedOperationException {
|
||||
if (!isTopLevel)
|
||||
throw new UnsupportedOperationException("This is not a top-level context.");
|
||||
assert leaves != null;
|
||||
|
@ -80,7 +80,7 @@ public final class CompositeReaderContext extends IndexReaderContext {
|
|||
|
||||
private static final class Builder {
|
||||
private final CompositeReader reader;
|
||||
private final List<AtomicReaderContext> leaves = new ArrayList<>();
|
||||
private final List<LeafReaderContext> leaves = new ArrayList<>();
|
||||
private int leafDocBase = 0;
|
||||
|
||||
public Builder(CompositeReader reader) {
|
||||
|
@ -92,9 +92,9 @@ public final class CompositeReaderContext extends IndexReaderContext {
|
|||
}
|
||||
|
||||
private IndexReaderContext build(CompositeReaderContext parent, IndexReader reader, int ord, int docBase) {
|
||||
if (reader instanceof AtomicReader) {
|
||||
final AtomicReader ar = (AtomicReader) reader;
|
||||
final AtomicReaderContext atomic = new AtomicReaderContext(parent, ar, ord, docBase, leaves.size(), leafDocBase);
|
||||
if (reader instanceof LeafReader) {
|
||||
final LeafReader ar = (LeafReader) reader;
|
||||
final LeafReaderContext atomic = new LeafReaderContext(parent, ar, ord, docBase, leaves.size(), leafDocBase);
|
||||
leaves.add(atomic);
|
||||
leafDocBase += reader.maxDoc();
|
||||
return atomic;
|
||||
|
|
|
@ -49,7 +49,7 @@ import org.apache.lucene.store.Directory;
|
|||
<code>IndexReader</code> instance; use your own
|
||||
(non-Lucene) objects instead.
|
||||
*/
|
||||
public abstract class DirectoryReader extends BaseCompositeReader<AtomicReader> {
|
||||
public abstract class DirectoryReader extends BaseCompositeReader<LeafReader> {
|
||||
|
||||
/** The index directory. */
|
||||
protected final Directory directory;
|
||||
|
@ -307,7 +307,7 @@ public abstract class DirectoryReader extends BaseCompositeReader<AtomicReader>
|
|||
* Subclasses of {@code DirectoryReader} should take care to not allow
|
||||
* modification of this internal array, e.g. {@link #doOpenIfChanged()}.
|
||||
*/
|
||||
protected DirectoryReader(Directory directory, AtomicReader[] segmentReaders) {
|
||||
protected DirectoryReader(Directory directory, LeafReader[] segmentReaders) {
|
||||
super(segmentReaders);
|
||||
this.directory = directory;
|
||||
}
|
||||
|
|
|
@ -202,7 +202,7 @@ public final class DocValues {
|
|||
/**
|
||||
* Returns NumericDocValues for the reader, or {@link #emptyNumeric()} if it has none.
|
||||
*/
|
||||
public static NumericDocValues getNumeric(AtomicReader in, String field) throws IOException {
|
||||
public static NumericDocValues getNumeric(LeafReader in, String field) throws IOException {
|
||||
NumericDocValues dv = in.getNumericDocValues(field);
|
||||
if (dv == null) {
|
||||
return emptyNumeric();
|
||||
|
@ -214,7 +214,7 @@ public final class DocValues {
|
|||
/**
|
||||
* Returns BinaryDocValues for the reader, or {@link #emptyBinary} if it has none.
|
||||
*/
|
||||
public static BinaryDocValues getBinary(AtomicReader in, String field) throws IOException {
|
||||
public static BinaryDocValues getBinary(LeafReader in, String field) throws IOException {
|
||||
BinaryDocValues dv = in.getBinaryDocValues(field);
|
||||
if (dv == null) {
|
||||
dv = in.getSortedDocValues(field);
|
||||
|
@ -228,7 +228,7 @@ public final class DocValues {
|
|||
/**
|
||||
* Returns SortedDocValues for the reader, or {@link #emptySorted} if it has none.
|
||||
*/
|
||||
public static SortedDocValues getSorted(AtomicReader in, String field) throws IOException {
|
||||
public static SortedDocValues getSorted(LeafReader in, String field) throws IOException {
|
||||
SortedDocValues dv = in.getSortedDocValues(field);
|
||||
if (dv == null) {
|
||||
return emptySorted();
|
||||
|
@ -240,7 +240,7 @@ public final class DocValues {
|
|||
/**
|
||||
* Returns SortedNumericDocValues for the reader, or {@link #emptySortedNumeric} if it has none.
|
||||
*/
|
||||
public static SortedNumericDocValues getSortedNumeric(AtomicReader in, String field) throws IOException {
|
||||
public static SortedNumericDocValues getSortedNumeric(LeafReader in, String field) throws IOException {
|
||||
SortedNumericDocValues dv = in.getSortedNumericDocValues(field);
|
||||
if (dv == null) {
|
||||
NumericDocValues single = in.getNumericDocValues(field);
|
||||
|
@ -256,7 +256,7 @@ public final class DocValues {
|
|||
/**
|
||||
* Returns SortedSetDocValues for the reader, or {@link #emptySortedSet} if it has none.
|
||||
*/
|
||||
public static SortedSetDocValues getSortedSet(AtomicReader in, String field) throws IOException {
|
||||
public static SortedSetDocValues getSortedSet(LeafReader in, String field) throws IOException {
|
||||
SortedSetDocValues dv = in.getSortedSetDocValues(field);
|
||||
if (dv == null) {
|
||||
SortedDocValues sorted = in.getSortedDocValues(field);
|
||||
|
@ -271,7 +271,7 @@ public final class DocValues {
|
|||
/**
|
||||
* Returns Bits for the reader, or {@link Bits} matching nothing if it has none.
|
||||
*/
|
||||
public static Bits getDocsWithField(AtomicReader in, String field) throws IOException {
|
||||
public static Bits getDocsWithField(LeafReader in, String field) throws IOException {
|
||||
Bits dv = in.getDocsWithField(field);
|
||||
if (dv == null) {
|
||||
return new Bits.MatchNoBits(in.maxDoc());
|
||||
|
|
|
@ -41,8 +41,8 @@ public abstract class FilterDirectoryReader extends DirectoryReader {
|
|||
*/
|
||||
public static abstract class SubReaderWrapper {
|
||||
|
||||
private AtomicReader[] wrap(List<? extends AtomicReader> readers) {
|
||||
AtomicReader[] wrapped = new AtomicReader[readers.size()];
|
||||
private LeafReader[] wrap(List<? extends LeafReader> readers) {
|
||||
LeafReader[] wrapped = new LeafReader[readers.size()];
|
||||
for (int i = 0; i < readers.size(); i++) {
|
||||
wrapped[i] = wrap(readers.get(i));
|
||||
}
|
||||
|
@ -57,7 +57,7 @@ public abstract class FilterDirectoryReader extends DirectoryReader {
|
|||
* @param reader the subreader to wrap
|
||||
* @return a wrapped/filtered AtomicReader
|
||||
*/
|
||||
public abstract AtomicReader wrap(AtomicReader reader);
|
||||
public abstract LeafReader wrap(LeafReader reader);
|
||||
|
||||
}
|
||||
|
||||
|
@ -71,7 +71,7 @@ public abstract class FilterDirectoryReader extends DirectoryReader {
|
|||
public StandardReaderWrapper() {}
|
||||
|
||||
@Override
|
||||
public AtomicReader wrap(AtomicReader reader) {
|
||||
public LeafReader wrap(LeafReader reader) {
|
||||
return reader;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,7 +35,7 @@ import org.apache.lucene.util.BytesRef;
|
|||
* methods and fields.
|
||||
* <p><b>NOTE</b>: If you override {@link #getLiveDocs()}, you will likely need
|
||||
* to override {@link #numDocs()} as well and vice-versa.
|
||||
* <p><b>NOTE</b>: If this {@link FilterAtomicReader} does not change the
|
||||
* <p><b>NOTE</b>: If this {@link FilterLeafReader} does not change the
|
||||
* content the contained reader, you could consider overriding
|
||||
* {@link #getCoreCacheKey()} so that
|
||||
* {@link CachingWrapperFilter} shares the same entries for this atomic reader
|
||||
|
@ -43,13 +43,13 @@ import org.apache.lucene.util.BytesRef;
|
|||
* overridden as well if the {@link #getLiveDocs() live docs} are not changed
|
||||
* either.
|
||||
*/
|
||||
public class FilterAtomicReader extends AtomicReader {
|
||||
public class FilterLeafReader extends LeafReader {
|
||||
|
||||
/** Get the wrapped instance by <code>reader</code> as long as this reader is
|
||||
* an intance of {@link FilterAtomicReader}. */
|
||||
public static AtomicReader unwrap(AtomicReader reader) {
|
||||
while (reader instanceof FilterAtomicReader) {
|
||||
reader = ((FilterAtomicReader) reader).in;
|
||||
* an intance of {@link FilterLeafReader}. */
|
||||
public static LeafReader unwrap(LeafReader reader) {
|
||||
while (reader instanceof FilterLeafReader) {
|
||||
reader = ((FilterLeafReader) reader).in;
|
||||
}
|
||||
return reader;
|
||||
}
|
||||
|
@ -318,14 +318,14 @@ public class FilterAtomicReader extends AtomicReader {
|
|||
}
|
||||
|
||||
/** The underlying AtomicReader. */
|
||||
protected final AtomicReader in;
|
||||
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>
|
||||
* @param in specified base reader.
|
||||
*/
|
||||
public FilterAtomicReader(AtomicReader in) {
|
||||
public FilterLeafReader(LeafReader in) {
|
||||
super();
|
||||
this.in = in;
|
||||
in.registerParentReader(this);
|
|
@ -48,7 +48,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
||||
<p>There are two different types of IndexReaders:
|
||||
<ul>
|
||||
<li>{@link AtomicReader}: These indexes do not consist of several sub-readers,
|
||||
<li>{@link LeafReader}: These indexes do not consist of several sub-readers,
|
||||
they are atomic. They support retrieval of stored fields, doc values, terms,
|
||||
and postings.
|
||||
<li>{@link CompositeReader}: Instances (like {@link DirectoryReader})
|
||||
|
@ -56,7 +56,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
be used to get stored fields from the underlying AtomicReaders,
|
||||
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 AtomicReader} (with a serious slowdown),
|
||||
Alternatively, you can mimic an {@link LeafReader} (with a serious slowdown),
|
||||
by wrapping composite readers with {@link SlowCompositeReaderWrapper}.
|
||||
</ul>
|
||||
|
||||
|
@ -87,7 +87,7 @@ public abstract class IndexReader implements Closeable {
|
|||
private final AtomicInteger refCount = new AtomicInteger(1);
|
||||
|
||||
IndexReader() {
|
||||
if (!(this instanceof CompositeReader || this instanceof AtomicReader))
|
||||
if (!(this instanceof CompositeReader || this instanceof LeafReader))
|
||||
throw new Error("IndexReader should never be directly extended, subclass AtomicReader or CompositeReader instead.");
|
||||
}
|
||||
|
||||
|
@ -128,7 +128,7 @@ public abstract class IndexReader implements Closeable {
|
|||
}
|
||||
|
||||
/** Expert: This method is called by {@code IndexReader}s which wrap other readers
|
||||
* (e.g. {@link CompositeReader} or {@link FilterAtomicReader}) to register the parent
|
||||
* (e.g. {@link CompositeReader} or {@link FilterLeafReader}) to register the parent
|
||||
* at the child (this reader) on construction of the parent. When this reader is closed,
|
||||
* it will mark all registered parents as closed, too. The references to parent readers
|
||||
* are weak only, so they can be GCed once they are no longer in use.
|
||||
|
@ -420,7 +420,7 @@ public abstract class IndexReader implements Closeable {
|
|||
* context are private to this reader and are not shared with another context
|
||||
* tree. For example, IndexSearcher uses this API to drive searching by one
|
||||
* 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 LeafReaderContext}.
|
||||
* <p>
|
||||
* Note: Any of the sub-{@link CompositeReaderContext} instances referenced
|
||||
* from this top-level context do not support {@link CompositeReaderContext#leaves()}.
|
||||
|
@ -434,7 +434,7 @@ public abstract class IndexReader implements Closeable {
|
|||
* This is a convenience method calling {@code this.getContext().leaves()}.
|
||||
* @see IndexReaderContext#leaves()
|
||||
*/
|
||||
public final List<AtomicReaderContext> leaves() {
|
||||
public final List<LeafReaderContext> leaves() {
|
||||
return getContext().leaves();
|
||||
}
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ public abstract class IndexReaderContext {
|
|||
public final int ordInParent;
|
||||
|
||||
IndexReaderContext(CompositeReaderContext parent, int ordInParent, int docBaseInParent) {
|
||||
if (!(this instanceof CompositeReaderContext || this instanceof AtomicReaderContext))
|
||||
if (!(this instanceof CompositeReaderContext || this instanceof LeafReaderContext))
|
||||
throw new Error("This class should never be extended by custom code!");
|
||||
this.parent = parent;
|
||||
this.docBaseInParent = docBaseInParent;
|
||||
|
@ -47,14 +47,14 @@ public abstract class IndexReaderContext {
|
|||
|
||||
/**
|
||||
* Returns the context's leaves if this context is a top-level context.
|
||||
* For convenience, if this is an {@link AtomicReaderContext} this
|
||||
* For convenience, if this is an {@link LeafReaderContext} this
|
||||
* returns itself as the only leaf.
|
||||
* <p>Note: this is convenience method since leaves can always be obtained by
|
||||
* walking the context tree using {@link #children()}.
|
||||
* @throws UnsupportedOperationException if this is not a top-level context.
|
||||
* @see #children()
|
||||
*/
|
||||
public abstract List<AtomicReaderContext> leaves() throws UnsupportedOperationException;
|
||||
public abstract List<LeafReaderContext> leaves() throws UnsupportedOperationException;
|
||||
|
||||
/**
|
||||
* Returns the context's children iff this context is a composite context
|
||||
|
|
|
@ -1224,13 +1224,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* reader you must use {@link #deleteDocuments(Term...)}). */
|
||||
public synchronized boolean tryDeleteDocument(IndexReader readerIn, int docID) throws IOException {
|
||||
|
||||
final AtomicReader reader;
|
||||
if (readerIn instanceof AtomicReader) {
|
||||
final LeafReader reader;
|
||||
if (readerIn instanceof LeafReader) {
|
||||
// Reader is already atomic: use the incoming docID:
|
||||
reader = (AtomicReader) readerIn;
|
||||
reader = (LeafReader) readerIn;
|
||||
} else {
|
||||
// Composite reader: lookup sub-reader and re-base docID:
|
||||
List<AtomicReaderContext> leaves = readerIn.leaves();
|
||||
List<LeafReaderContext> leaves = readerIn.leaves();
|
||||
int subIndex = ReaderUtil.subIndex(docID, leaves);
|
||||
reader = leaves.get(subIndex).reader();
|
||||
docID -= leaves.get(subIndex).docBase;
|
||||
|
@ -2481,10 +2481,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
flush(false, true);
|
||||
|
||||
String mergedName = newSegmentName();
|
||||
final List<AtomicReader> mergeReaders = new ArrayList<>();
|
||||
final List<LeafReader> mergeReaders = new ArrayList<>();
|
||||
for (IndexReader indexReader : readers) {
|
||||
numDocs += indexReader.numDocs();
|
||||
for (AtomicReaderContext ctx : indexReader.leaves()) {
|
||||
for (LeafReaderContext ctx : indexReader.leaves()) {
|
||||
mergeReaders.add(ctx.reader());
|
||||
}
|
||||
}
|
||||
|
@ -3945,7 +3945,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
try {
|
||||
if (!merger.shouldMerge()) {
|
||||
// would result in a 0 document segment: nothing to merge!
|
||||
mergeState = new MergeState(new ArrayList<AtomicReader>(), merge.info.info, infoStream, checkAbort);
|
||||
mergeState = new MergeState(new ArrayList<LeafReader>(), merge.info.info, infoStream, checkAbort);
|
||||
} else {
|
||||
mergeState = merger.merge();
|
||||
}
|
||||
|
@ -4375,10 +4375,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
protected IndexReaderWarmer() {
|
||||
}
|
||||
|
||||
/** Invoked on the {@link AtomicReader} for the newly
|
||||
/** Invoked on the {@link LeafReader} for the newly
|
||||
* merged segment, before that segment is made visible
|
||||
* to near-real-time readers. */
|
||||
public abstract void warm(AtomicReader reader) throws IOException;
|
||||
public abstract void warm(LeafReader reader) throws IOException;
|
||||
}
|
||||
|
||||
private void tragicEvent(Throwable tragedy, String location) {
|
||||
|
|
|
@ -110,7 +110,7 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
|
|||
* ram buffers use <code>false</code> */
|
||||
public final static boolean DEFAULT_USE_COMPOUND_FILE_SYSTEM = true;
|
||||
|
||||
/** Default value for calling {@link AtomicReader#checkIntegrity()} before
|
||||
/** Default value for calling {@link LeafReader#checkIntegrity()} before
|
||||
* merging segments (set to <code>false</code>). You can set this
|
||||
* to <code>true</code> for additional safety. */
|
||||
public final static boolean DEFAULT_CHECK_INTEGRITY_AT_MERGE = false;
|
||||
|
|
|
@ -44,27 +44,27 @@ import org.apache.lucene.util.Bits;
|
|||
<code>IndexReader</code> instance; use your own
|
||||
(non-Lucene) objects instead.
|
||||
*/
|
||||
public abstract class AtomicReader extends IndexReader {
|
||||
public abstract class LeafReader extends IndexReader {
|
||||
|
||||
private final AtomicReaderContext readerContext = new AtomicReaderContext(this);
|
||||
private final LeafReaderContext readerContext = new LeafReaderContext(this);
|
||||
|
||||
/** Sole constructor. (For invocation by subclass
|
||||
* constructors, typically implicit.) */
|
||||
protected AtomicReader() {
|
||||
protected LeafReader() {
|
||||
super();
|
||||
}
|
||||
|
||||
@Override
|
||||
public final AtomicReaderContext getContext() {
|
||||
public final LeafReaderContext getContext() {
|
||||
ensureOpen();
|
||||
return readerContext;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when the shared core for this {@link AtomicReader}
|
||||
* Called when the shared core for this {@link LeafReader}
|
||||
* is closed.
|
||||
* <p>
|
||||
* If this {@link AtomicReader} impl has the ability to share
|
||||
* If this {@link LeafReader} impl has the ability to share
|
||||
* resources across instances that might only vary through
|
||||
* deleted documents and doc values updates, then this listener
|
||||
* will only be called when the shared core is closed.
|
||||
|
@ -112,9 +112,9 @@ public abstract class AtomicReader extends IndexReader {
|
|||
}
|
||||
|
||||
/** Add a {@link CoreClosedListener} as a {@link ReaderClosedListener}. This
|
||||
* method is typically useful for {@link AtomicReader} implementations that
|
||||
* method is typically useful for {@link LeafReader} implementations that
|
||||
* don't have the concept of a core that is shared across several
|
||||
* {@link AtomicReader} instances in which case the {@link CoreClosedListener}
|
||||
* {@link LeafReader} instances in which case the {@link CoreClosedListener}
|
||||
* is called when closing the reader. */
|
||||
protected static void addCoreClosedListenerAsReaderClosedListener(IndexReader reader, CoreClosedListener listener) {
|
||||
reader.addReaderClosedListener(new CoreClosedListenerWrapper(listener));
|
|
@ -21,22 +21,22 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
|
||||
/**
|
||||
* {@link IndexReaderContext} for {@link AtomicReader} instances.
|
||||
* {@link IndexReaderContext} for {@link LeafReader} instances.
|
||||
*/
|
||||
public final class AtomicReaderContext extends IndexReaderContext {
|
||||
public final class LeafReaderContext extends IndexReaderContext {
|
||||
/** The readers ord in the top-level's leaves array */
|
||||
public final int ord;
|
||||
/** The readers absolute doc base */
|
||||
public final int docBase;
|
||||
|
||||
private final AtomicReader reader;
|
||||
private final List<AtomicReaderContext> leaves;
|
||||
private final LeafReader reader;
|
||||
private final List<LeafReaderContext> leaves;
|
||||
|
||||
/**
|
||||
* Creates a new {@link AtomicReaderContext}
|
||||
* Creates a new {@link LeafReaderContext}
|
||||
*/
|
||||
AtomicReaderContext(CompositeReaderContext parent, AtomicReader reader,
|
||||
int ord, int docBase, int leafOrd, int leafDocBase) {
|
||||
LeafReaderContext(CompositeReaderContext parent, LeafReader reader,
|
||||
int ord, int docBase, int leafOrd, int leafDocBase) {
|
||||
super(parent, ord, docBase);
|
||||
this.ord = leafOrd;
|
||||
this.docBase = leafDocBase;
|
||||
|
@ -44,12 +44,12 @@ public final class AtomicReaderContext extends IndexReaderContext {
|
|||
this.leaves = isTopLevel ? Collections.singletonList(this) : null;
|
||||
}
|
||||
|
||||
AtomicReaderContext(AtomicReader atomicReader) {
|
||||
this(null, atomicReader, 0, 0, 0, 0);
|
||||
LeafReaderContext(LeafReader leafReader) {
|
||||
this(null, leafReader, 0, 0, 0, 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AtomicReaderContext> leaves() {
|
||||
public List<LeafReaderContext> leaves() {
|
||||
if (!isTopLevel) {
|
||||
throw new UnsupportedOperationException("This is not a top-level context.");
|
||||
}
|
||||
|
@ -63,7 +63,7 @@ public final class AtomicReaderContext extends IndexReaderContext {
|
|||
}
|
||||
|
||||
@Override
|
||||
public AtomicReader reader() {
|
||||
public LeafReader reader() {
|
||||
return reader;
|
||||
}
|
||||
}
|
|
@ -464,7 +464,7 @@ public class LiveIndexWriterConfig {
|
|||
}
|
||||
|
||||
/**
|
||||
* Sets if {@link IndexWriter} should call {@link AtomicReader#checkIntegrity()}
|
||||
* Sets if {@link IndexWriter} should call {@link LeafReader#checkIntegrity()}
|
||||
* on existing segments before merging them into a new one.
|
||||
* <p>
|
||||
* Use <code>true</code> to enable this safety check, which can help
|
||||
|
@ -477,7 +477,7 @@ public class LiveIndexWriterConfig {
|
|||
return this;
|
||||
}
|
||||
|
||||
/** Returns true if {@link AtomicReader#checkIntegrity()} is called before
|
||||
/** Returns true if {@link LeafReader#checkIntegrity()} is called before
|
||||
* merging segments. */
|
||||
public boolean getCheckIntegrityAtMerge() {
|
||||
return checkIntegrityAtMerge;
|
||||
|
|
|
@ -21,9 +21,9 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
import static org.apache.lucene.index.FilterAtomicReader.FilterFields;
|
||||
import static org.apache.lucene.index.FilterAtomicReader.FilterTerms;
|
||||
import static org.apache.lucene.index.FilterAtomicReader.FilterTermsEnum;
|
||||
import static org.apache.lucene.index.FilterLeafReader.FilterFields;
|
||||
import static org.apache.lucene.index.FilterLeafReader.FilterTerms;
|
||||
import static org.apache.lucene.index.FilterLeafReader.FilterTermsEnum;
|
||||
|
||||
/** A {@link Fields} implementation that merges multiple
|
||||
* Fields into one, and maps around deleted documents.
|
||||
|
|
|
@ -134,12 +134,12 @@ public abstract class MergePolicy {
|
|||
* reorders doc IDs, it must override {@link #getDocMap} too so that
|
||||
* deletes that happened during the merge can be applied to the newly
|
||||
* merged segment. */
|
||||
public List<AtomicReader> getMergeReaders() throws IOException {
|
||||
public List<LeafReader> getMergeReaders() throws IOException {
|
||||
if (readers == null) {
|
||||
throw new IllegalStateException("IndexWriter has not initialized readers from the segment infos yet");
|
||||
}
|
||||
final List<AtomicReader> readers = new ArrayList<>(this.readers.size());
|
||||
for (AtomicReader reader : this.readers) {
|
||||
final List<LeafReader> readers = new ArrayList<>(this.readers.size());
|
||||
for (LeafReader reader : this.readers) {
|
||||
if (reader.numDocs() > 0) {
|
||||
readers.add(reader);
|
||||
}
|
||||
|
|
|
@ -59,7 +59,7 @@ public class MergeState {
|
|||
|
||||
/** Creates a {@link DocMap} instance appropriate for
|
||||
* this reader. */
|
||||
public static DocMap build(AtomicReader reader) {
|
||||
public static DocMap build(LeafReader reader) {
|
||||
final int maxDoc = reader.maxDoc();
|
||||
if (!reader.hasDeletions()) {
|
||||
return new NoDelDocMap(maxDoc);
|
||||
|
@ -137,7 +137,7 @@ public class MergeState {
|
|||
public FieldInfos fieldInfos;
|
||||
|
||||
/** Readers being merged. */
|
||||
public final List<AtomicReader> readers;
|
||||
public final List<LeafReader> readers;
|
||||
|
||||
/** Maps docIDs around deletions. */
|
||||
public DocMap[] docMaps;
|
||||
|
@ -157,7 +157,7 @@ public class MergeState {
|
|||
public int checkAbortCount;
|
||||
|
||||
/** Sole constructor. */
|
||||
MergeState(List<AtomicReader> readers, SegmentInfo segmentInfo, InfoStream infoStream, CheckAbort checkAbort) {
|
||||
MergeState(List<LeafReader> readers, SegmentInfo segmentInfo, InfoStream infoStream, CheckAbort checkAbort) {
|
||||
this.readers = readers;
|
||||
this.segmentInfo = segmentInfo;
|
||||
this.infoStream = infoStream;
|
||||
|
|
|
@ -57,11 +57,11 @@ public class MultiDocValues {
|
|||
/** Returns a NumericDocValues for a reader's norms (potentially merging on-the-fly).
|
||||
* <p>
|
||||
* This is a slow way to access normalization values. Instead, access them per-segment
|
||||
* with {@link AtomicReader#getNormValues(String)}
|
||||
* with {@link LeafReader#getNormValues(String)}
|
||||
* </p>
|
||||
*/
|
||||
public static NumericDocValues getNormValues(final IndexReader r, final String field) throws IOException {
|
||||
final List<AtomicReaderContext> leaves = r.leaves();
|
||||
final List<LeafReaderContext> leaves = r.leaves();
|
||||
final int size = leaves.size();
|
||||
if (size == 0) {
|
||||
return null;
|
||||
|
@ -77,7 +77,7 @@ public class MultiDocValues {
|
|||
final NumericDocValues[] values = new NumericDocValues[size];
|
||||
final int[] starts = new int[size+1];
|
||||
for (int i = 0; i < size; i++) {
|
||||
AtomicReaderContext context = leaves.get(i);
|
||||
LeafReaderContext context = leaves.get(i);
|
||||
NumericDocValues v = context.reader().getNormValues(field);
|
||||
if (v == null) {
|
||||
v = DocValues.emptyNumeric();
|
||||
|
@ -103,11 +103,11 @@ public class MultiDocValues {
|
|||
/** Returns a NumericDocValues for a reader's docvalues (potentially merging on-the-fly)
|
||||
* <p>
|
||||
* This is a slow way to access numeric values. Instead, access them per-segment
|
||||
* with {@link AtomicReader#getNumericDocValues(String)}
|
||||
* with {@link LeafReader#getNumericDocValues(String)}
|
||||
* </p>
|
||||
* */
|
||||
public static NumericDocValues getNumericValues(final IndexReader r, final String field) throws IOException {
|
||||
final List<AtomicReaderContext> leaves = r.leaves();
|
||||
final List<LeafReaderContext> leaves = r.leaves();
|
||||
final int size = leaves.size();
|
||||
if (size == 0) {
|
||||
return null;
|
||||
|
@ -119,7 +119,7 @@ public class MultiDocValues {
|
|||
final NumericDocValues[] values = new NumericDocValues[size];
|
||||
final int[] starts = new int[size+1];
|
||||
for (int i = 0; i < size; i++) {
|
||||
AtomicReaderContext context = leaves.get(i);
|
||||
LeafReaderContext context = leaves.get(i);
|
||||
NumericDocValues v = context.reader().getNumericDocValues(field);
|
||||
if (v == null) {
|
||||
v = DocValues.emptyNumeric();
|
||||
|
@ -147,11 +147,11 @@ public class MultiDocValues {
|
|||
/** Returns a Bits for a reader's docsWithField (potentially merging on-the-fly)
|
||||
* <p>
|
||||
* This is a slow way to access this bitset. Instead, access them per-segment
|
||||
* with {@link AtomicReader#getDocsWithField(String)}
|
||||
* with {@link LeafReader#getDocsWithField(String)}
|
||||
* </p>
|
||||
* */
|
||||
public static Bits getDocsWithField(final IndexReader r, final String field) throws IOException {
|
||||
final List<AtomicReaderContext> leaves = r.leaves();
|
||||
final List<LeafReaderContext> leaves = r.leaves();
|
||||
final int size = leaves.size();
|
||||
if (size == 0) {
|
||||
return null;
|
||||
|
@ -164,7 +164,7 @@ public class MultiDocValues {
|
|||
final Bits[] values = new Bits[size];
|
||||
final int[] starts = new int[size+1];
|
||||
for (int i = 0; i < size; i++) {
|
||||
AtomicReaderContext context = leaves.get(i);
|
||||
LeafReaderContext context = leaves.get(i);
|
||||
Bits v = context.reader().getDocsWithField(field);
|
||||
if (v == null) {
|
||||
v = new Bits.MatchNoBits(context.reader().maxDoc());
|
||||
|
@ -192,11 +192,11 @@ public class MultiDocValues {
|
|||
/** Returns a BinaryDocValues for a reader's docvalues (potentially merging on-the-fly)
|
||||
* <p>
|
||||
* This is a slow way to access binary values. Instead, access them per-segment
|
||||
* with {@link AtomicReader#getBinaryDocValues(String)}
|
||||
* with {@link LeafReader#getBinaryDocValues(String)}
|
||||
* </p>
|
||||
*/
|
||||
public static BinaryDocValues getBinaryValues(final IndexReader r, final String field) throws IOException {
|
||||
final List<AtomicReaderContext> leaves = r.leaves();
|
||||
final List<LeafReaderContext> leaves = r.leaves();
|
||||
final int size = leaves.size();
|
||||
|
||||
if (size == 0) {
|
||||
|
@ -209,7 +209,7 @@ public class MultiDocValues {
|
|||
final BinaryDocValues[] values = new BinaryDocValues[size];
|
||||
final int[] starts = new int[size+1];
|
||||
for (int i = 0; i < size; i++) {
|
||||
AtomicReaderContext context = leaves.get(i);
|
||||
LeafReaderContext context = leaves.get(i);
|
||||
BinaryDocValues v = context.reader().getBinaryDocValues(field);
|
||||
if (v == null) {
|
||||
v = DocValues.emptyBinary();
|
||||
|
@ -237,11 +237,11 @@ public class MultiDocValues {
|
|||
/** Returns a SortedNumericDocValues for a reader's docvalues (potentially merging on-the-fly)
|
||||
* <p>
|
||||
* This is a slow way to access sorted numeric values. Instead, access them per-segment
|
||||
* with {@link AtomicReader#getSortedNumericDocValues(String)}
|
||||
* with {@link LeafReader#getSortedNumericDocValues(String)}
|
||||
* </p>
|
||||
* */
|
||||
public static SortedNumericDocValues getSortedNumericValues(final IndexReader r, final String field) throws IOException {
|
||||
final List<AtomicReaderContext> leaves = r.leaves();
|
||||
final List<LeafReaderContext> leaves = r.leaves();
|
||||
final int size = leaves.size();
|
||||
if (size == 0) {
|
||||
return null;
|
||||
|
@ -253,7 +253,7 @@ public class MultiDocValues {
|
|||
final SortedNumericDocValues[] values = new SortedNumericDocValues[size];
|
||||
final int[] starts = new int[size+1];
|
||||
for (int i = 0; i < size; i++) {
|
||||
AtomicReaderContext context = leaves.get(i);
|
||||
LeafReaderContext context = leaves.get(i);
|
||||
SortedNumericDocValues v = context.reader().getSortedNumericDocValues(field);
|
||||
if (v == null) {
|
||||
v = DocValues.emptySortedNumeric(context.reader().maxDoc());
|
||||
|
@ -294,11 +294,11 @@ public class MultiDocValues {
|
|||
/** Returns a SortedDocValues for a reader's docvalues (potentially doing extremely slow things).
|
||||
* <p>
|
||||
* This is an extremely slow way to access sorted values. Instead, access them per-segment
|
||||
* with {@link AtomicReader#getSortedDocValues(String)}
|
||||
* with {@link LeafReader#getSortedDocValues(String)}
|
||||
* </p>
|
||||
*/
|
||||
public static SortedDocValues getSortedValues(final IndexReader r, final String field) throws IOException {
|
||||
final List<AtomicReaderContext> leaves = r.leaves();
|
||||
final List<LeafReaderContext> leaves = r.leaves();
|
||||
final int size = leaves.size();
|
||||
|
||||
if (size == 0) {
|
||||
|
@ -311,7 +311,7 @@ public class MultiDocValues {
|
|||
final SortedDocValues[] values = new SortedDocValues[size];
|
||||
final int[] starts = new int[size+1];
|
||||
for (int i = 0; i < size; i++) {
|
||||
AtomicReaderContext context = leaves.get(i);
|
||||
LeafReaderContext context = leaves.get(i);
|
||||
SortedDocValues v = context.reader().getSortedDocValues(field);
|
||||
if (v == null) {
|
||||
v = DocValues.emptySorted();
|
||||
|
@ -334,11 +334,11 @@ public class MultiDocValues {
|
|||
/** Returns a SortedSetDocValues for a reader's docvalues (potentially doing extremely slow things).
|
||||
* <p>
|
||||
* This is an extremely slow way to access sorted values. Instead, access them per-segment
|
||||
* with {@link AtomicReader#getSortedSetDocValues(String)}
|
||||
* with {@link LeafReader#getSortedSetDocValues(String)}
|
||||
* </p>
|
||||
*/
|
||||
public static SortedSetDocValues getSortedSetValues(final IndexReader r, final String field) throws IOException {
|
||||
final List<AtomicReaderContext> leaves = r.leaves();
|
||||
final List<LeafReaderContext> leaves = r.leaves();
|
||||
final int size = leaves.size();
|
||||
|
||||
if (size == 0) {
|
||||
|
@ -351,7 +351,7 @@ public class MultiDocValues {
|
|||
final SortedSetDocValues[] values = new SortedSetDocValues[size];
|
||||
final int[] starts = new int[size+1];
|
||||
for (int i = 0; i < size; i++) {
|
||||
AtomicReaderContext context = leaves.get(i);
|
||||
LeafReaderContext context = leaves.get(i);
|
||||
SortedSetDocValues v = context.reader().getSortedSetDocValues(field);
|
||||
if (v == null) {
|
||||
v = DocValues.emptySortedSet();
|
||||
|
|
|
@ -60,7 +60,7 @@ public final class MultiFields extends Fields {
|
|||
* It's better to get the sub-readers and iterate through them
|
||||
* yourself. */
|
||||
public static Fields getFields(IndexReader reader) throws IOException {
|
||||
final List<AtomicReaderContext> leaves = reader.leaves();
|
||||
final List<LeafReaderContext> leaves = reader.leaves();
|
||||
switch (leaves.size()) {
|
||||
case 0:
|
||||
// no fields
|
||||
|
@ -71,8 +71,8 @@ public final class MultiFields extends Fields {
|
|||
default:
|
||||
final List<Fields> fields = new ArrayList<>();
|
||||
final List<ReaderSlice> slices = new ArrayList<>();
|
||||
for (final AtomicReaderContext ctx : leaves) {
|
||||
final AtomicReader r = ctx.reader();
|
||||
for (final LeafReaderContext ctx : leaves) {
|
||||
final LeafReader r = ctx.reader();
|
||||
final Fields f = r.fields();
|
||||
if (f != null) {
|
||||
fields.add(f);
|
||||
|
@ -101,7 +101,7 @@ public final class MultiFields extends Fields {
|
|||
* yourself. */
|
||||
public static Bits getLiveDocs(IndexReader reader) {
|
||||
if (reader.hasDeletions()) {
|
||||
final List<AtomicReaderContext> leaves = reader.leaves();
|
||||
final List<LeafReaderContext> leaves = reader.leaves();
|
||||
final int size = leaves.size();
|
||||
assert size > 0 : "A reader with deletions must have at least one leave";
|
||||
if (size == 1) {
|
||||
|
@ -111,7 +111,7 @@ public final class MultiFields extends Fields {
|
|||
final int[] starts = new int[size + 1];
|
||||
for (int i = 0; i < size; i++) {
|
||||
// record all liveDocs, even if they are null
|
||||
final AtomicReaderContext ctx = leaves.get(i);
|
||||
final LeafReaderContext ctx = leaves.get(i);
|
||||
liveDocs[i] = ctx.reader().getLiveDocs();
|
||||
starts[i] = ctx.docBase;
|
||||
}
|
||||
|
@ -254,7 +254,7 @@ public final class MultiFields extends Fields {
|
|||
*/
|
||||
public static FieldInfos getMergedFieldInfos(IndexReader reader) {
|
||||
final FieldInfos.Builder builder = new FieldInfos.Builder();
|
||||
for(final AtomicReaderContext ctx : reader.leaves()) {
|
||||
for(final LeafReaderContext ctx : reader.leaves()) {
|
||||
builder.add(ctx.reader().getFieldInfos());
|
||||
}
|
||||
return builder.finish();
|
||||
|
|
|
@ -96,7 +96,7 @@ public class ParallelCompositeReader extends BaseCompositeReader<IndexReader> {
|
|||
for (int i = 0; i < noSubs; i++) {
|
||||
final IndexReader r = firstSubReaders.get(i);
|
||||
childMaxDoc[i] = r.maxDoc();
|
||||
childAtomic[i] = r instanceof AtomicReader;
|
||||
childAtomic[i] = r instanceof LeafReader;
|
||||
}
|
||||
validate(readers, maxDoc, childMaxDoc, childAtomic);
|
||||
validate(storedFieldsReaders, maxDoc, childMaxDoc, childAtomic);
|
||||
|
@ -104,18 +104,18 @@ public class ParallelCompositeReader extends BaseCompositeReader<IndexReader> {
|
|||
// hierarchically build the same subreader structure as the first CompositeReader with Parallel*Readers:
|
||||
final IndexReader[] subReaders = new IndexReader[noSubs];
|
||||
for (int i = 0; i < subReaders.length; i++) {
|
||||
if (firstSubReaders.get(i) instanceof AtomicReader) {
|
||||
final AtomicReader[] atomicSubs = new AtomicReader[readers.length];
|
||||
if (firstSubReaders.get(i) instanceof LeafReader) {
|
||||
final LeafReader[] atomicSubs = new LeafReader[readers.length];
|
||||
for (int j = 0; j < readers.length; j++) {
|
||||
atomicSubs[j] = (AtomicReader) readers[j].getSequentialSubReaders().get(i);
|
||||
atomicSubs[j] = (LeafReader) readers[j].getSequentialSubReaders().get(i);
|
||||
}
|
||||
final AtomicReader[] storedSubs = new AtomicReader[storedFieldsReaders.length];
|
||||
final LeafReader[] storedSubs = new LeafReader[storedFieldsReaders.length];
|
||||
for (int j = 0; j < storedFieldsReaders.length; j++) {
|
||||
storedSubs[j] = (AtomicReader) storedFieldsReaders[j].getSequentialSubReaders().get(i);
|
||||
storedSubs[j] = (LeafReader) storedFieldsReaders[j].getSequentialSubReaders().get(i);
|
||||
}
|
||||
// We pass true for closeSubs and we prevent closing of subreaders in doClose():
|
||||
// By this the synthetic throw-away readers used here are completely invisible to ref-counting
|
||||
subReaders[i] = new ParallelAtomicReader(true, atomicSubs, storedSubs) {
|
||||
subReaders[i] = new ParallelLeafReader(true, atomicSubs, storedSubs) {
|
||||
@Override
|
||||
protected void doClose() {}
|
||||
};
|
||||
|
@ -157,7 +157,7 @@ public class ParallelCompositeReader extends BaseCompositeReader<IndexReader> {
|
|||
if (r.maxDoc() != childMaxDoc[subIDX]) {
|
||||
throw new IllegalArgumentException("All readers must have same corresponding subReader maxDoc");
|
||||
}
|
||||
if (!(childAtomic[subIDX] ? (r instanceof AtomicReader) : (r instanceof CompositeReader))) {
|
||||
if (!(childAtomic[subIDX] ? (r instanceof LeafReader) : (r instanceof CompositeReader))) {
|
||||
throw new IllegalArgumentException("All readers must have same corresponding subReader types (atomic or composite)");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,7 +29,7 @@ import java.util.TreeMap;
|
|||
import org.apache.lucene.util.Bits;
|
||||
|
||||
|
||||
/** An {@link AtomicReader} which reads multiple, parallel indexes. Each index
|
||||
/** An {@link LeafReader} which reads multiple, parallel indexes. Each index
|
||||
* added must have the same number of documents, but typically each contains
|
||||
* different fields. Deletions are taken from the first reader.
|
||||
* Each document contains the union of the fields of all documents
|
||||
|
@ -47,41 +47,41 @@ import org.apache.lucene.util.Bits;
|
|||
* same order to the other indexes. <em>Failure to do so will result in
|
||||
* undefined behavior</em>.
|
||||
*/
|
||||
public class ParallelAtomicReader extends AtomicReader {
|
||||
public class ParallelLeafReader extends LeafReader {
|
||||
private final FieldInfos fieldInfos;
|
||||
private final ParallelFields fields = new ParallelFields();
|
||||
private final AtomicReader[] parallelReaders, storedFieldsReaders;
|
||||
private final Set<AtomicReader> completeReaderSet =
|
||||
Collections.newSetFromMap(new IdentityHashMap<AtomicReader,Boolean>());
|
||||
private final LeafReader[] parallelReaders, storedFieldsReaders;
|
||||
private final Set<LeafReader> completeReaderSet =
|
||||
Collections.newSetFromMap(new IdentityHashMap<LeafReader,Boolean>());
|
||||
private final boolean closeSubReaders;
|
||||
private final int maxDoc, numDocs;
|
||||
private final boolean hasDeletions;
|
||||
private final SortedMap<String,AtomicReader> fieldToReader = new TreeMap<>();
|
||||
private final SortedMap<String,AtomicReader> tvFieldToReader = new TreeMap<>();
|
||||
private final SortedMap<String,LeafReader> fieldToReader = new TreeMap<>();
|
||||
private final SortedMap<String,LeafReader> tvFieldToReader = new TreeMap<>();
|
||||
|
||||
/** Create a ParallelAtomicReader based on the provided
|
||||
* readers; auto-closes the given readers on {@link #close()}. */
|
||||
public ParallelAtomicReader(AtomicReader... readers) throws IOException {
|
||||
public ParallelLeafReader(LeafReader... readers) throws IOException {
|
||||
this(true, readers);
|
||||
}
|
||||
|
||||
/** Create a ParallelAtomicReader based on the provided
|
||||
* readers. */
|
||||
public ParallelAtomicReader(boolean closeSubReaders, AtomicReader... readers) throws IOException {
|
||||
public ParallelLeafReader(boolean closeSubReaders, LeafReader... readers) throws IOException {
|
||||
this(closeSubReaders, readers, readers);
|
||||
}
|
||||
|
||||
/** Expert: create a ParallelAtomicReader based on the provided
|
||||
* readers and storedFieldReaders; when a document is
|
||||
* loaded, only storedFieldsReaders will be used. */
|
||||
public ParallelAtomicReader(boolean closeSubReaders, AtomicReader[] readers, AtomicReader[] storedFieldsReaders) throws IOException {
|
||||
public ParallelLeafReader(boolean closeSubReaders, LeafReader[] readers, LeafReader[] storedFieldsReaders) throws IOException {
|
||||
this.closeSubReaders = closeSubReaders;
|
||||
if (readers.length == 0 && storedFieldsReaders.length > 0)
|
||||
throw new IllegalArgumentException("There must be at least one main reader if storedFieldsReaders are used.");
|
||||
this.parallelReaders = readers.clone();
|
||||
this.storedFieldsReaders = storedFieldsReaders.clone();
|
||||
if (parallelReaders.length > 0) {
|
||||
final AtomicReader first = parallelReaders[0];
|
||||
final LeafReader first = parallelReaders[0];
|
||||
this.maxDoc = first.maxDoc();
|
||||
this.numDocs = first.numDocs();
|
||||
this.hasDeletions = first.hasDeletions();
|
||||
|
@ -93,7 +93,7 @@ public class ParallelAtomicReader extends AtomicReader {
|
|||
Collections.addAll(completeReaderSet, this.storedFieldsReaders);
|
||||
|
||||
// check compatibility:
|
||||
for(AtomicReader reader : completeReaderSet) {
|
||||
for(LeafReader reader : completeReaderSet) {
|
||||
if (reader.maxDoc() != maxDoc) {
|
||||
throw new IllegalArgumentException("All readers must have same maxDoc: "+maxDoc+"!="+reader.maxDoc());
|
||||
}
|
||||
|
@ -102,7 +102,7 @@ public class ParallelAtomicReader extends AtomicReader {
|
|||
// TODO: make this read-only in a cleaner way?
|
||||
FieldInfos.Builder builder = new FieldInfos.Builder();
|
||||
// build FieldInfos and fieldToReader map:
|
||||
for (final AtomicReader reader : this.parallelReaders) {
|
||||
for (final LeafReader reader : this.parallelReaders) {
|
||||
final FieldInfos readerFieldInfos = reader.getFieldInfos();
|
||||
for (FieldInfo fieldInfo : readerFieldInfos) {
|
||||
// NOTE: first reader having a given field "wins":
|
||||
|
@ -118,7 +118,7 @@ public class ParallelAtomicReader extends AtomicReader {
|
|||
fieldInfos = builder.finish();
|
||||
|
||||
// build Fields instance
|
||||
for (final AtomicReader reader : this.parallelReaders) {
|
||||
for (final LeafReader reader : this.parallelReaders) {
|
||||
final Fields readerFields = reader.fields();
|
||||
if (readerFields != null) {
|
||||
for (String field : readerFields) {
|
||||
|
@ -131,7 +131,7 @@ public class ParallelAtomicReader extends AtomicReader {
|
|||
}
|
||||
|
||||
// do this finally so any Exceptions occurred before don't affect refcounts:
|
||||
for (AtomicReader reader : completeReaderSet) {
|
||||
for (LeafReader reader : completeReaderSet) {
|
||||
if (!closeSubReaders) {
|
||||
reader.incRef();
|
||||
}
|
||||
|
@ -142,7 +142,7 @@ public class ParallelAtomicReader extends AtomicReader {
|
|||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder buffer = new StringBuilder("ParallelAtomicReader(");
|
||||
for (final Iterator<AtomicReader> iter = completeReaderSet.iterator(); iter.hasNext();) {
|
||||
for (final Iterator<LeafReader> iter = completeReaderSet.iterator(); iter.hasNext();) {
|
||||
buffer.append(iter.next());
|
||||
if (iter.hasNext()) buffer.append(", ");
|
||||
}
|
||||
|
@ -226,7 +226,7 @@ public class ParallelAtomicReader extends AtomicReader {
|
|||
@Override
|
||||
public void document(int docID, StoredFieldVisitor visitor) throws IOException {
|
||||
ensureOpen();
|
||||
for (final AtomicReader reader: storedFieldsReaders) {
|
||||
for (final LeafReader reader: storedFieldsReaders) {
|
||||
reader.document(docID, visitor);
|
||||
}
|
||||
}
|
||||
|
@ -235,7 +235,7 @@ public class ParallelAtomicReader extends AtomicReader {
|
|||
public Fields getTermVectors(int docID) throws IOException {
|
||||
ensureOpen();
|
||||
ParallelFields fields = null;
|
||||
for (Map.Entry<String,AtomicReader> ent : tvFieldToReader.entrySet()) {
|
||||
for (Map.Entry<String,LeafReader> ent : tvFieldToReader.entrySet()) {
|
||||
String fieldName = ent.getKey();
|
||||
Terms vector = ent.getValue().getTermVector(docID, fieldName);
|
||||
if (vector != null) {
|
||||
|
@ -252,7 +252,7 @@ public class ParallelAtomicReader extends AtomicReader {
|
|||
@Override
|
||||
protected synchronized void doClose() throws IOException {
|
||||
IOException ioe = null;
|
||||
for (AtomicReader reader : completeReaderSet) {
|
||||
for (LeafReader reader : completeReaderSet) {
|
||||
try {
|
||||
if (closeSubReaders) {
|
||||
reader.close();
|
||||
|
@ -270,49 +270,49 @@ public class ParallelAtomicReader extends AtomicReader {
|
|||
@Override
|
||||
public NumericDocValues getNumericDocValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
AtomicReader reader = fieldToReader.get(field);
|
||||
LeafReader reader = fieldToReader.get(field);
|
||||
return reader == null ? null : reader.getNumericDocValues(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BinaryDocValues getBinaryDocValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
AtomicReader reader = fieldToReader.get(field);
|
||||
LeafReader reader = fieldToReader.get(field);
|
||||
return reader == null ? null : reader.getBinaryDocValues(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedDocValues getSortedDocValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
AtomicReader reader = fieldToReader.get(field);
|
||||
LeafReader reader = fieldToReader.get(field);
|
||||
return reader == null ? null : reader.getSortedDocValues(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedNumericDocValues getSortedNumericDocValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
AtomicReader reader = fieldToReader.get(field);
|
||||
LeafReader reader = fieldToReader.get(field);
|
||||
return reader == null ? null : reader.getSortedNumericDocValues(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
AtomicReader reader = fieldToReader.get(field);
|
||||
LeafReader reader = fieldToReader.get(field);
|
||||
return reader == null ? null : reader.getSortedSetDocValues(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getDocsWithField(String field) throws IOException {
|
||||
ensureOpen();
|
||||
AtomicReader reader = fieldToReader.get(field);
|
||||
LeafReader reader = fieldToReader.get(field);
|
||||
return reader == null ? null : reader.getDocsWithField(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NumericDocValues getNormValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
AtomicReader reader = fieldToReader.get(field);
|
||||
LeafReader reader = fieldToReader.get(field);
|
||||
NumericDocValues values = reader == null ? null : reader.getNormValues(field);
|
||||
return values;
|
||||
}
|
||||
|
@ -320,7 +320,7 @@ public class ParallelAtomicReader extends AtomicReader {
|
|||
@Override
|
||||
public void checkIntegrity() throws IOException {
|
||||
ensureOpen();
|
||||
for (AtomicReader reader : completeReaderSet) {
|
||||
for (LeafReader reader : completeReaderSet) {
|
||||
reader.checkIntegrity();
|
||||
}
|
||||
}
|
|
@ -70,7 +70,7 @@ public final class ReaderUtil {
|
|||
* Returns index of the searcher/reader for document <code>n</code> in the
|
||||
* array used to construct this searcher/reader.
|
||||
*/
|
||||
public static int subIndex(int n, List<AtomicReaderContext> leaves) { // find
|
||||
public static int subIndex(int n, List<LeafReaderContext> leaves) { // find
|
||||
// searcher/reader for doc n:
|
||||
int size = leaves.size();
|
||||
int lo = 0; // search starts array
|
||||
|
|
|
@ -26,13 +26,12 @@ import java.util.Set;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.NormsProducer;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.codecs.TermVectorsReader;
|
||||
import org.apache.lucene.index.AtomicReader.CoreClosedListener;
|
||||
import org.apache.lucene.index.LeafReader.CoreClosedListener;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
|
|
@ -50,11 +50,11 @@ final class SegmentMerger {
|
|||
private final FieldInfos.Builder fieldInfosBuilder;
|
||||
|
||||
// note, just like in codec apis Directory 'dir' is NOT the same as segmentInfo.dir!!
|
||||
SegmentMerger(List<AtomicReader> readers, SegmentInfo segmentInfo, InfoStream infoStream, Directory dir,
|
||||
SegmentMerger(List<LeafReader> readers, SegmentInfo segmentInfo, InfoStream infoStream, Directory dir,
|
||||
MergeState.CheckAbort checkAbort, FieldInfos.FieldNumbers fieldNumbers, IOContext context, boolean validate) throws IOException {
|
||||
// validate incoming readers
|
||||
if (validate) {
|
||||
for (AtomicReader reader : readers) {
|
||||
for (LeafReader reader : readers) {
|
||||
reader.checkIntegrity();
|
||||
}
|
||||
}
|
||||
|
@ -182,7 +182,7 @@ final class SegmentMerger {
|
|||
}
|
||||
|
||||
public void mergeFieldInfos() throws IOException {
|
||||
for (AtomicReader reader : mergeState.readers) {
|
||||
for (LeafReader reader : mergeState.readers) {
|
||||
FieldInfos readerFieldInfos = reader.getFieldInfos();
|
||||
for (FieldInfo fi : readerFieldInfos) {
|
||||
fieldInfosBuilder.add(fi);
|
||||
|
@ -250,7 +250,7 @@ final class SegmentMerger {
|
|||
int i = 0;
|
||||
while(i < mergeState.readers.size()) {
|
||||
|
||||
final AtomicReader reader = mergeState.readers.get(i);
|
||||
final LeafReader reader = mergeState.readers.get(i);
|
||||
|
||||
mergeState.docBase[i] = docBase;
|
||||
final MergeState.DocMap docMap = MergeState.DocMap.build(reader);
|
||||
|
|
|
@ -50,7 +50,7 @@ import org.apache.lucene.util.RamUsageEstimator;
|
|||
* may share the same core data.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class SegmentReader extends AtomicReader implements Accountable {
|
||||
public final class SegmentReader extends LeafReader implements Accountable {
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED =
|
||||
RamUsageEstimator.shallowSizeOfInstance(SegmentReader.class)
|
||||
|
|
|
@ -38,7 +38,7 @@ public class SimpleMergedSegmentWarmer extends IndexReaderWarmer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void warm(AtomicReader reader) throws IOException {
|
||||
public void warm(LeafReader reader) throws IOException {
|
||||
long startTime = System.currentTimeMillis();
|
||||
int indexedCount = 0;
|
||||
int docValuesCount = 0;
|
||||
|
|
|
@ -23,12 +23,10 @@ import java.util.Map;
|
|||
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
import org.apache.lucene.index.DirectoryReader; // javadoc
|
||||
import org.apache.lucene.index.FieldInfo.DocValuesType;
|
||||
import org.apache.lucene.index.MultiDocValues.MultiSortedDocValues;
|
||||
import org.apache.lucene.index.MultiDocValues.MultiSortedSetDocValues;
|
||||
import org.apache.lucene.index.MultiDocValues.OrdinalMap;
|
||||
import org.apache.lucene.index.MultiReader; // javadoc
|
||||
|
||||
/**
|
||||
* This class forces a composite reader (eg a {@link
|
||||
|
@ -45,22 +43,22 @@ import org.apache.lucene.index.MultiReader; // javadoc
|
|||
* atomic leaves and then operate per-AtomicReader,
|
||||
* instead of using this class.
|
||||
*/
|
||||
public final class SlowCompositeReaderWrapper extends AtomicReader {
|
||||
public final class SlowCompositeReaderWrapper extends LeafReader {
|
||||
|
||||
private final CompositeReader in;
|
||||
private final Fields fields;
|
||||
private final Bits liveDocs;
|
||||
|
||||
/** This method is sugar for getting an {@link AtomicReader} from
|
||||
/** This method is sugar for getting an {@link LeafReader} from
|
||||
* an {@link IndexReader} of any kind. If the reader is already atomic,
|
||||
* it is returned unchanged, otherwise wrapped by this class.
|
||||
*/
|
||||
public static AtomicReader wrap(IndexReader reader) throws IOException {
|
||||
public static LeafReader wrap(IndexReader reader) throws IOException {
|
||||
if (reader instanceof CompositeReader) {
|
||||
return new SlowCompositeReaderWrapper((CompositeReader) reader);
|
||||
} else {
|
||||
assert reader instanceof AtomicReader;
|
||||
return (AtomicReader) reader;
|
||||
assert reader instanceof LeafReader;
|
||||
return (LeafReader) reader;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -143,7 +141,7 @@ public final class SlowCompositeReaderWrapper extends AtomicReader {
|
|||
final SortedDocValues[] values = new SortedDocValues[size];
|
||||
final int[] starts = new int[size+1];
|
||||
for (int i = 0; i < size; i++) {
|
||||
AtomicReaderContext context = in.leaves().get(i);
|
||||
LeafReaderContext context = in.leaves().get(i);
|
||||
SortedDocValues v = context.reader().getSortedDocValues(field);
|
||||
if (v == null) {
|
||||
v = DocValues.emptySorted();
|
||||
|
@ -182,7 +180,7 @@ public final class SlowCompositeReaderWrapper extends AtomicReader {
|
|||
final SortedSetDocValues[] values = new SortedSetDocValues[size];
|
||||
final int[] starts = new int[size+1];
|
||||
for (int i = 0; i < size; i++) {
|
||||
AtomicReaderContext context = in.leaves().get(i);
|
||||
LeafReaderContext context = in.leaves().get(i);
|
||||
SortedSetDocValues v = context.reader().getSortedSetDocValues(field);
|
||||
if (v == null) {
|
||||
v = DocValues.emptySortedSet();
|
||||
|
@ -259,7 +257,7 @@ public final class SlowCompositeReaderWrapper extends AtomicReader {
|
|||
@Override
|
||||
public void checkIntegrity() throws IOException {
|
||||
ensureOpen();
|
||||
for (AtomicReaderContext ctx : in.leaves()) {
|
||||
for (LeafReaderContext ctx : in.leaves()) {
|
||||
ctx.reader().checkIntegrity();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -37,7 +37,7 @@ final class StandardDirectoryReader extends DirectoryReader {
|
|||
private final boolean applyAllDeletes;
|
||||
|
||||
/** called only from static open() methods */
|
||||
StandardDirectoryReader(Directory directory, AtomicReader[] readers, IndexWriter writer,
|
||||
StandardDirectoryReader(Directory directory, LeafReader[] readers, IndexWriter writer,
|
||||
SegmentInfos sis, boolean applyAllDeletes) {
|
||||
super(directory, readers);
|
||||
this.writer = writer;
|
||||
|
@ -128,7 +128,7 @@ final class StandardDirectoryReader extends DirectoryReader {
|
|||
}
|
||||
|
||||
/** This constructor is only used for {@link #doOpenIfChanged(SegmentInfos)} */
|
||||
private static DirectoryReader open(Directory directory, SegmentInfos infos, List<? extends AtomicReader> oldReaders) throws IOException {
|
||||
private static DirectoryReader open(Directory directory, SegmentInfos infos, List<? extends LeafReader> oldReaders) throws IOException {
|
||||
|
||||
// we put the old SegmentReaders in a map, that allows us
|
||||
// to lookup a reader using its segment name
|
||||
|
@ -234,7 +234,7 @@ final class StandardDirectoryReader extends DirectoryReader {
|
|||
if (writer != null) {
|
||||
buffer.append(":nrt");
|
||||
}
|
||||
for (final AtomicReader r : getSequentialSubReaders()) {
|
||||
for (final LeafReader r : getSequentialSubReaders()) {
|
||||
buffer.append(' ');
|
||||
buffer.append(r);
|
||||
}
|
||||
|
@ -351,7 +351,7 @@ final class StandardDirectoryReader extends DirectoryReader {
|
|||
@Override
|
||||
protected void doClose() throws IOException {
|
||||
Throwable firstExc = null;
|
||||
for (final AtomicReader r : getSequentialSubReaders()) {
|
||||
for (final LeafReader r : getSequentialSubReaders()) {
|
||||
// try to close each reader, even if an exception is thrown
|
||||
try {
|
||||
r.decRef();
|
||||
|
|
|
@ -85,7 +85,7 @@ public final class TermContext {
|
|||
final BytesRef bytes = term.bytes();
|
||||
final TermContext perReaderTermState = new TermContext(context);
|
||||
//if (DEBUG) System.out.println("prts.build term=" + term);
|
||||
for (final AtomicReaderContext ctx : context.leaves()) {
|
||||
for (final LeafReaderContext ctx : context.leaves()) {
|
||||
//if (DEBUG) System.out.println(" r=" + leaves[i].reader);
|
||||
final Fields fields = ctx.reader().fields();
|
||||
if (fields != null) {
|
||||
|
|
|
@ -254,7 +254,7 @@ its {@link org.apache.lucene.search.similarities.Similarity#computeNorm} method.
|
|||
</p>
|
||||
<p>
|
||||
Additional user-supplied statistics can be added to the document as DocValues fields and
|
||||
accessed via {@link org.apache.lucene.index.AtomicReader#getNumericDocValues}.
|
||||
accessed via {@link org.apache.lucene.index.LeafReader#getNumericDocValues}.
|
||||
</p>
|
||||
<p>
|
||||
</body>
|
||||
|
|
|
@ -24,7 +24,7 @@ import java.util.Iterator;
|
|||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.BooleanClause.Occur;
|
||||
|
@ -228,7 +228,7 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc)
|
||||
public Explanation explain(LeafReaderContext context, int doc)
|
||||
throws IOException {
|
||||
final int minShouldMatch =
|
||||
BooleanQuery.this.getMinimumNumberShouldMatch();
|
||||
|
@ -305,7 +305,7 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder,
|
||||
public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder,
|
||||
Bits acceptDocs) throws IOException {
|
||||
|
||||
if (scoreDocsInOrder || minNrShouldMatch > 1) {
|
||||
|
@ -340,7 +340,7 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs)
|
||||
public Scorer scorer(LeafReaderContext context, Bits acceptDocs)
|
||||
throws IOException {
|
||||
// initially the user provided value,
|
||||
// but if minNrShouldMatch == optional.size(),
|
||||
|
|
|
@ -18,12 +18,8 @@ package org.apache.lucene.search;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.search.BooleanQuery.BooleanWeight;
|
||||
|
||||
/* Description from Doug Cutting (excerpted from
|
||||
|
|
|
@ -17,7 +17,7 @@ package org.apache.lucene.search;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
|
@ -83,7 +83,7 @@ public abstract class CachingCollector extends FilterCollector {
|
|||
private static class NoScoreCachingCollector extends CachingCollector {
|
||||
|
||||
List<Boolean> acceptDocsOutOfOrders;
|
||||
List<AtomicReaderContext> contexts;
|
||||
List<LeafReaderContext> contexts;
|
||||
List<int[]> docs;
|
||||
int maxDocsToCache;
|
||||
NoScoreCachingLeafCollector lastCollector;
|
||||
|
@ -100,7 +100,7 @@ public abstract class CachingCollector extends FilterCollector {
|
|||
return new NoScoreCachingLeafCollector(in, maxDocsToCache);
|
||||
}
|
||||
|
||||
public LeafCollector getLeafCollector(AtomicReaderContext context) throws IOException {
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
|
||||
postCollection();
|
||||
final LeafCollector in = this.in.getLeafCollector(context);
|
||||
if (contexts != null) {
|
||||
|
@ -151,7 +151,7 @@ public abstract class CachingCollector extends FilterCollector {
|
|||
}
|
||||
assert docs.size() == contexts.size();
|
||||
for (int i = 0; i < contexts.size(); ++i) {
|
||||
final AtomicReaderContext context = contexts.get(i);
|
||||
final LeafReaderContext context = contexts.get(i);
|
||||
final boolean docsInOrder = !acceptDocsOutOfOrders.get(i);
|
||||
final LeafCollector collector = other.getLeafCollector(context);
|
||||
if (!collector.acceptsDocsOutOfOrder() && !docsInOrder) {
|
||||
|
|
|
@ -26,8 +26,8 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.WeakHashMap;
|
||||
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.Accountables;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -61,12 +61,12 @@ public class CachingWrapperFilter extends Filter implements Accountable {
|
|||
* Provide the DocIdSet to be cached, using the DocIdSet provided
|
||||
* by the wrapped Filter. <p>This implementation returns the given {@link DocIdSet},
|
||||
* if {@link DocIdSet#isCacheable} returns <code>true</code>, else it calls
|
||||
* {@link #cacheImpl(DocIdSetIterator,AtomicReader)}
|
||||
* {@link #cacheImpl(DocIdSetIterator, org.apache.lucene.index.LeafReader)}
|
||||
* <p>Note: This method returns {@linkplain DocIdSet#EMPTY} if the given docIdSet
|
||||
* is <code>null</code> or if {@link DocIdSet#iterator()} return <code>null</code>. The empty
|
||||
* instance is use as a placeholder in the cache instead of the <code>null</code> value.
|
||||
*/
|
||||
protected DocIdSet docIdSetToCache(DocIdSet docIdSet, AtomicReader reader) throws IOException {
|
||||
protected DocIdSet docIdSetToCache(DocIdSet docIdSet, LeafReader reader) throws IOException {
|
||||
if (docIdSet == null) {
|
||||
// this is better than returning null, as the nonnull result can be cached
|
||||
return EMPTY;
|
||||
|
@ -88,7 +88,7 @@ public class CachingWrapperFilter extends Filter implements Accountable {
|
|||
/**
|
||||
* Default cache implementation: uses {@link WAH8DocIdSet}.
|
||||
*/
|
||||
protected DocIdSet cacheImpl(DocIdSetIterator iterator, AtomicReader reader) throws IOException {
|
||||
protected DocIdSet cacheImpl(DocIdSetIterator iterator, LeafReader reader) throws IOException {
|
||||
WAH8DocIdSet.Builder builder = new WAH8DocIdSet.Builder();
|
||||
builder.add(iterator);
|
||||
return builder.build();
|
||||
|
@ -98,8 +98,8 @@ public class CachingWrapperFilter extends Filter implements Accountable {
|
|||
int hitCount, missCount;
|
||||
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(AtomicReaderContext context, final Bits acceptDocs) throws IOException {
|
||||
final AtomicReader reader = context.reader();
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, final Bits acceptDocs) throws IOException {
|
||||
final LeafReader reader = context.reader();
|
||||
final Object key = reader.getCoreCacheKey();
|
||||
|
||||
DocIdSet docIdSet = cache.get(key);
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
|
||||
/**
|
||||
* <p>Expert: Collectors are primarily meant to be used to
|
||||
|
@ -71,6 +71,6 @@ public interface Collector {
|
|||
* @param context
|
||||
* next atomic reader context
|
||||
*/
|
||||
LeafCollector getLeafCollector(AtomicReaderContext context) throws IOException;
|
||||
LeafCollector getLeafCollector(LeafReaderContext context) throws IOException;
|
||||
|
||||
}
|
||||
|
|
|
@ -17,7 +17,7 @@ package org.apache.lucene.search;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -134,7 +134,7 @@ public class ConstantScoreQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
final DocIdSetIterator disi;
|
||||
if (filter != null) {
|
||||
assert query == null;
|
||||
|
@ -150,7 +150,7 @@ public class ConstantScoreQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
final DocIdSetIterator disi;
|
||||
if (filter != null) {
|
||||
assert query == null;
|
||||
|
@ -176,7 +176,7 @@ public class ConstantScoreQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
|
||||
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
|
||||
final Scorer cs = scorer(context, context.reader().getLiveDocs());
|
||||
final boolean exists = (cs != null && cs.advance(doc) == doc);
|
||||
|
||||
|
|
|
@ -23,7 +23,7 @@ import java.util.Iterator;
|
|||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -153,7 +153,7 @@ public class DisjunctionMaxQuery extends Query implements Iterable<Query> {
|
|||
|
||||
/** Create the scorer used to score our associated DisjunctionMaxQuery */
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
List<Scorer> scorers = new ArrayList<>();
|
||||
for (Weight w : weights) {
|
||||
// we will advance() subscorers
|
||||
|
@ -175,7 +175,7 @@ public class DisjunctionMaxQuery extends Query implements Iterable<Query> {
|
|||
|
||||
/** Explain the score we computed for doc */
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
|
||||
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
|
||||
if (disjuncts.size() == 1) return weights.get(0).explain(context,doc);
|
||||
ComplexExplanation result = new ComplexExplanation();
|
||||
float max = 0.0f, sum = 0.0f;
|
||||
|
|
|
@ -18,15 +18,14 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* A range filter built on top of a cached multi-valued term field (from {@link AtomicReader#getSortedSetDocValues}).
|
||||
* A range filter built on top of a cached multi-valued term field (from {@link org.apache.lucene.index.LeafReader#getSortedSetDocValues}).
|
||||
*
|
||||
* <p>Like {@link DocValuesRangeFilter}, this is just a specialized range query versus
|
||||
* using a TermRangeQuery with {@link DocTermOrdsRewriteMethod}: it will only do
|
||||
|
@ -50,17 +49,17 @@ public abstract class DocTermOrdsRangeFilter extends Filter {
|
|||
|
||||
/** This method is implemented for each data type */
|
||||
@Override
|
||||
public abstract DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException;
|
||||
public abstract DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException;
|
||||
|
||||
/**
|
||||
* Creates a BytesRef range filter using {@link AtomicReader#getSortedSetDocValues}. This works with all
|
||||
* Creates a BytesRef range filter using {@link org.apache.lucene.index.LeafReader#getSortedSetDocValues}. This works with all
|
||||
* fields containing zero or one term in the field. The range can be half-open by setting one
|
||||
* of the values to <code>null</code>.
|
||||
*/
|
||||
public static DocTermOrdsRangeFilter newBytesRefRange(String field, BytesRef lowerVal, BytesRef upperVal, boolean includeLower, boolean includeUpper) {
|
||||
return new DocTermOrdsRangeFilter(field, lowerVal, upperVal, includeLower, includeUpper) {
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
final SortedSetDocValues docTermOrds = DocValues.getSortedSet(context.reader(), field);
|
||||
final long lowerPoint = lowerVal == null ? -1 : docTermOrds.lookupTerm(lowerVal);
|
||||
final long upperPoint = upperVal == null ? -1 : docTermOrds.lookupTerm(upperVal);
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
|
@ -83,7 +83,7 @@ public final class DocTermOrdsRewriteMethod extends MultiTermQuery.RewriteMethod
|
|||
* results.
|
||||
*/
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(AtomicReaderContext context, final Bits acceptDocs) throws IOException {
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, final Bits acceptDocs) throws IOException {
|
||||
final SortedSetDocValues docTermOrds = DocValues.getSortedSet(context.reader(), query.field);
|
||||
// Cannot use FixedBitSet because we require long index (ord):
|
||||
final LongBitSet termSet = new LongBitSet(docTermOrds.getValueCount());
|
||||
|
|
|
@ -22,8 +22,7 @@ import org.apache.lucene.document.DoubleField; // for javadocs
|
|||
import org.apache.lucene.document.FloatField; // for javadocs
|
||||
import org.apache.lucene.document.IntField; // for javadocs
|
||||
import org.apache.lucene.document.LongField; // for javadocs
|
||||
import org.apache.lucene.index.AtomicReader; // for javadocs
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
|
@ -33,7 +32,7 @@ import org.apache.lucene.util.NumericUtils;
|
|||
|
||||
/**
|
||||
* A range filter built on top of numeric doc values field
|
||||
* (from {@link AtomicReader#getNumericDocValues(String)}).
|
||||
* (from {@link org.apache.lucene.index.LeafReader#getNumericDocValues(String)}).
|
||||
*
|
||||
* <p>{@code DocValuesRangeFilter} builds a single cache for the field the first time it is used.
|
||||
* Each subsequent {@code DocValuesRangeFilter} on the same field then reuses this cache,
|
||||
|
@ -50,7 +49,7 @@ import org.apache.lucene.util.NumericUtils;
|
|||
* LongField} or {@link DoubleField}. But
|
||||
* it has the problem that it only works with exact one value/document (see below).
|
||||
*
|
||||
* <p>As with all {@link AtomicReader#getNumericDocValues} based functionality,
|
||||
* <p>As with all {@link org.apache.lucene.index.LeafReader#getNumericDocValues} based functionality,
|
||||
* {@code DocValuesRangeFilter} is only valid for
|
||||
* fields which exact one term for each document (except for {@link #newStringRange}
|
||||
* where 0 terms are also allowed). Due to historical reasons, for numeric ranges
|
||||
|
@ -81,17 +80,17 @@ public abstract class DocValuesRangeFilter<T> extends Filter {
|
|||
|
||||
/** This method is implemented for each data type */
|
||||
@Override
|
||||
public abstract DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException;
|
||||
public abstract DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException;
|
||||
|
||||
/**
|
||||
* Creates a string range filter using {@link AtomicReader#getSortedDocValues(String)}. This works with all
|
||||
* Creates a string range filter using {@link org.apache.lucene.index.LeafReader#getSortedDocValues(String)}. This works with all
|
||||
* fields containing zero or one term in the field. The range can be half-open by setting one
|
||||
* of the values to <code>null</code>.
|
||||
*/
|
||||
public static DocValuesRangeFilter<String> newStringRange(String field, String lowerVal, String upperVal, boolean includeLower, boolean includeUpper) {
|
||||
return new DocValuesRangeFilter<String>(field, lowerVal, upperVal, includeLower, includeUpper) {
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
final SortedDocValues fcsi = DocValues.getSorted(context.reader(), field);
|
||||
final int lowerPoint = lowerVal == null ? -1 : fcsi.lookupTerm(new BytesRef(lowerVal));
|
||||
final int upperPoint = upperVal == null ? -1 : fcsi.lookupTerm(new BytesRef(upperVal));
|
||||
|
@ -140,7 +139,7 @@ public abstract class DocValuesRangeFilter<T> extends Filter {
|
|||
}
|
||||
|
||||
/**
|
||||
* Creates a BytesRef range filter using {@link AtomicReader#getSortedDocValues(String)}. This works with all
|
||||
* Creates a BytesRef range filter using {@link org.apache.lucene.index.LeafReader#getSortedDocValues(String)}. This works with all
|
||||
* fields containing zero or one term in the field. The range can be half-open by setting one
|
||||
* of the values to <code>null</code>.
|
||||
*/
|
||||
|
@ -148,7 +147,7 @@ public abstract class DocValuesRangeFilter<T> extends Filter {
|
|||
public static DocValuesRangeFilter<BytesRef> newBytesRefRange(String field, BytesRef lowerVal, BytesRef upperVal, boolean includeLower, boolean includeUpper) {
|
||||
return new DocValuesRangeFilter<BytesRef>(field, lowerVal, upperVal, includeLower, includeUpper) {
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
final SortedDocValues fcsi = DocValues.getSorted(context.reader(), field);
|
||||
final int lowerPoint = lowerVal == null ? -1 : fcsi.lookupTerm(lowerVal);
|
||||
final int upperPoint = upperVal == null ? -1 : fcsi.lookupTerm(upperVal);
|
||||
|
@ -197,14 +196,14 @@ public abstract class DocValuesRangeFilter<T> extends Filter {
|
|||
}
|
||||
|
||||
/**
|
||||
* Creates a numeric range filter using {@link AtomicReader#getSortedDocValues(String)}. This works with all
|
||||
* Creates a numeric range filter using {@link org.apache.lucene.index.LeafReader#getSortedDocValues(String)}. This works with all
|
||||
* int fields containing exactly one numeric term in the field. The range can be half-open by setting one
|
||||
* of the values to <code>null</code>.
|
||||
*/
|
||||
public static DocValuesRangeFilter<Integer> newIntRange(String field, Integer lowerVal, Integer upperVal, boolean includeLower, boolean includeUpper) {
|
||||
return new DocValuesRangeFilter<Integer>(field, lowerVal, upperVal, includeLower, includeUpper) {
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
final int inclusiveLowerPoint, inclusiveUpperPoint;
|
||||
if (lowerVal != null) {
|
||||
int i = lowerVal.intValue();
|
||||
|
@ -239,14 +238,14 @@ public abstract class DocValuesRangeFilter<T> extends Filter {
|
|||
}
|
||||
|
||||
/**
|
||||
* Creates a numeric range filter using {@link AtomicReader#getNumericDocValues(String)}. This works with all
|
||||
* Creates a numeric range filter using {@link org.apache.lucene.index.LeafReader#getNumericDocValues(String)}. This works with all
|
||||
* long fields containing exactly one numeric term in the field. The range can be half-open by setting one
|
||||
* of the values to <code>null</code>.
|
||||
*/
|
||||
public static DocValuesRangeFilter<Long> newLongRange(String field, Long lowerVal, Long upperVal, boolean includeLower, boolean includeUpper) {
|
||||
return new DocValuesRangeFilter<Long>(field, lowerVal, upperVal, includeLower, includeUpper) {
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
final long inclusiveLowerPoint, inclusiveUpperPoint;
|
||||
if (lowerVal != null) {
|
||||
long i = lowerVal.longValue();
|
||||
|
@ -281,14 +280,14 @@ public abstract class DocValuesRangeFilter<T> extends Filter {
|
|||
}
|
||||
|
||||
/**
|
||||
* Creates a numeric range filter using {@link AtomicReader#getNumericDocValues(String)}. This works with all
|
||||
* Creates a numeric range filter using {@link org.apache.lucene.index.LeafReader#getNumericDocValues(String)}. This works with all
|
||||
* float fields containing exactly one numeric term in the field. The range can be half-open by setting one
|
||||
* of the values to <code>null</code>.
|
||||
*/
|
||||
public static DocValuesRangeFilter<Float> newFloatRange(String field, Float lowerVal, Float upperVal, boolean includeLower, boolean includeUpper) {
|
||||
return new DocValuesRangeFilter<Float>(field, lowerVal, upperVal, includeLower, includeUpper) {
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
// we transform the floating point numbers to sortable integers
|
||||
// using NumericUtils to easier find the next bigger/lower value
|
||||
final float inclusiveLowerPoint, inclusiveUpperPoint;
|
||||
|
@ -327,14 +326,14 @@ public abstract class DocValuesRangeFilter<T> extends Filter {
|
|||
}
|
||||
|
||||
/**
|
||||
* Creates a numeric range filter using {@link AtomicReader#getNumericDocValues(String)}. This works with all
|
||||
* Creates a numeric range filter using {@link org.apache.lucene.index.LeafReader#getNumericDocValues(String)}. This works with all
|
||||
* double fields containing exactly one numeric term in the field. The range can be half-open by setting one
|
||||
* of the values to <code>null</code>.
|
||||
*/
|
||||
public static DocValuesRangeFilter<Double> newDoubleRange(String field, Double lowerVal, Double upperVal, boolean includeLower, boolean includeUpper) {
|
||||
return new DocValuesRangeFilter<Double>(field, lowerVal, upperVal, includeLower, includeUpper) {
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
// we transform the floating point numbers to sortable integers
|
||||
// using NumericUtils to easier find the next bigger/lower value
|
||||
final double inclusiveLowerPoint, inclusiveUpperPoint;
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
|
@ -83,7 +83,7 @@ public final class DocValuesRewriteMethod extends MultiTermQuery.RewriteMethod {
|
|||
* results.
|
||||
*/
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(AtomicReaderContext context, final Bits acceptDocs) throws IOException {
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, final Bits acceptDocs) throws IOException {
|
||||
final SortedDocValues fcsi = DocValues.getSorted(context.reader(), query.field);
|
||||
// Cannot use FixedBitSet because we require long index (ord):
|
||||
final LongBitSet termSet = new LongBitSet(fcsi.getValueCount());
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocsEnum; // javadoc @link
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
|
@ -104,7 +104,7 @@ public class DocValuesTermsFilter extends Filter {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
final SortedDocValues fcsi = DocValues.getSorted(context.reader(), field);
|
||||
final FixedBitSet bits = new FixedBitSet(fcsi.getValueCount());
|
||||
for (int i=0;i<terms.length;i++) {
|
||||
|
|
|
@ -19,8 +19,7 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
|
@ -78,7 +77,7 @@ import org.apache.lucene.util.BytesRefBuilder;
|
|||
* priority queue. The {@link FieldValueHitQueue}
|
||||
* calls this method when a new hit is competitive.
|
||||
*
|
||||
* <li> {@link #setNextReader(AtomicReaderContext)} Invoked
|
||||
* <li> {@link #setNextReader(org.apache.lucene.index.LeafReaderContext)} Invoked
|
||||
* when the search is switching to the next segment.
|
||||
* You may need to update internal state of the
|
||||
* comparator, for example retrieving new values from
|
||||
|
@ -170,7 +169,7 @@ public abstract class FieldComparator<T> {
|
|||
public abstract void copy(int slot, int doc) throws IOException;
|
||||
|
||||
/**
|
||||
* Set a new {@link AtomicReaderContext}. All subsequent docIDs are relative to
|
||||
* Set a new {@link org.apache.lucene.index.LeafReaderContext}. All subsequent docIDs are relative to
|
||||
* the current reader (you must add docBase if you need to
|
||||
* map it to a top-level docID).
|
||||
*
|
||||
|
@ -180,7 +179,7 @@ public abstract class FieldComparator<T> {
|
|||
* comparator across segments
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public abstract FieldComparator<T> setNextReader(AtomicReaderContext context) throws IOException;
|
||||
public abstract FieldComparator<T> setNextReader(LeafReaderContext context) throws IOException;
|
||||
|
||||
/** Sets the Scorer to use in case a document's score is
|
||||
* needed.
|
||||
|
@ -235,7 +234,7 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator<T> setNextReader(AtomicReaderContext context) throws IOException {
|
||||
public FieldComparator<T> setNextReader(LeafReaderContext context) throws IOException {
|
||||
currentReaderValues = getNumericDocValues(context, field);
|
||||
if (missingValue != null) {
|
||||
docsWithField = DocValues.getDocsWithField(context.reader(), field);
|
||||
|
@ -250,13 +249,13 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
|
||||
/** Retrieves the NumericDocValues for the field in this segment */
|
||||
protected NumericDocValues getNumericDocValues(AtomicReaderContext context, String field) throws IOException {
|
||||
protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
|
||||
return DocValues.getNumeric(context.reader(), field);
|
||||
}
|
||||
}
|
||||
|
||||
/** Parses field's values as double (using {@link
|
||||
* AtomicReader#getNumericDocValues} and sorts by ascending value */
|
||||
* org.apache.lucene.index.LeafReader#getNumericDocValues} and sorts by ascending value */
|
||||
public static class DoubleComparator extends NumericComparator<Double> {
|
||||
private final double[] values;
|
||||
private double bottom;
|
||||
|
@ -324,7 +323,7 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
|
||||
/** Parses field's values as float (using {@link
|
||||
* AtomicReader#getNumericDocValues(String)} and sorts by ascending value */
|
||||
* org.apache.lucene.index.LeafReader#getNumericDocValues(String)} and sorts by ascending value */
|
||||
public static class FloatComparator extends NumericComparator<Float> {
|
||||
private final float[] values;
|
||||
private float bottom;
|
||||
|
@ -393,7 +392,7 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
|
||||
/** Parses field's values as int (using {@link
|
||||
* AtomicReader#getNumericDocValues(String)} and sorts by ascending value */
|
||||
* org.apache.lucene.index.LeafReader#getNumericDocValues(String)} and sorts by ascending value */
|
||||
public static class IntComparator extends NumericComparator<Integer> {
|
||||
private final int[] values;
|
||||
private int bottom; // Value of bottom of queue
|
||||
|
@ -461,7 +460,7 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
|
||||
/** Parses field's values as long (using {@link
|
||||
* AtomicReader#getNumericDocValues(String)} and sorts by ascending value */
|
||||
* org.apache.lucene.index.LeafReader#getNumericDocValues(String)} and sorts by ascending value */
|
||||
public static class LongComparator extends NumericComparator<Long> {
|
||||
private final long[] values;
|
||||
private long bottom;
|
||||
|
@ -565,7 +564,7 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator<Float> setNextReader(AtomicReaderContext context) {
|
||||
public FieldComparator<Float> setNextReader(LeafReaderContext context) {
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -641,7 +640,7 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator<Integer> setNextReader(AtomicReaderContext context) {
|
||||
public FieldComparator<Integer> setNextReader(LeafReaderContext context) {
|
||||
// TODO: can we "map" our docIDs to the current
|
||||
// reader? saves having to then subtract on every
|
||||
// compare call
|
||||
|
@ -675,7 +674,7 @@ public abstract class FieldComparator<T> {
|
|||
* ordinals. This is functionally equivalent to {@link
|
||||
* org.apache.lucene.search.FieldComparator.TermValComparator}, but it first resolves the string
|
||||
* to their relative ordinal positions (using the index
|
||||
* returned by {@link AtomicReader#getSortedDocValues(String)}), and
|
||||
* returned by {@link org.apache.lucene.index.LeafReader#getSortedDocValues(String)}), and
|
||||
* does most comparisons using the ordinals. For medium
|
||||
* to large results, this comparator will be much faster
|
||||
* than {@link org.apache.lucene.search.FieldComparator.TermValComparator}. For very small
|
||||
|
@ -819,12 +818,12 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
|
||||
/** Retrieves the SortedDocValues for the field in this segment */
|
||||
protected SortedDocValues getSortedDocValues(AtomicReaderContext context, String field) throws IOException {
|
||||
protected SortedDocValues getSortedDocValues(LeafReaderContext context, String field) throws IOException {
|
||||
return DocValues.getSorted(context.reader(), field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator<BytesRef> setNextReader(AtomicReaderContext context) throws IOException {
|
||||
public FieldComparator<BytesRef> setNextReader(LeafReaderContext context) throws IOException {
|
||||
termsIndex = getSortedDocValues(context, field);
|
||||
currentReaderGen++;
|
||||
|
||||
|
@ -983,12 +982,12 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
|
||||
/** Retrieves the BinaryDocValues for the field in this segment */
|
||||
protected BinaryDocValues getBinaryDocValues(AtomicReaderContext context, String field) throws IOException {
|
||||
protected BinaryDocValues getBinaryDocValues(LeafReaderContext context, String field) throws IOException {
|
||||
return DocValues.getBinary(context.reader(), field);
|
||||
}
|
||||
|
||||
/** Retrieves the set of documents that have a value in this segment */
|
||||
protected Bits getDocsWithField(AtomicReaderContext context, String field) throws IOException {
|
||||
protected Bits getDocsWithField(LeafReaderContext context, String field) throws IOException {
|
||||
return DocValues.getDocsWithField(context.reader(), field);
|
||||
}
|
||||
|
||||
|
@ -1002,7 +1001,7 @@ public abstract class FieldComparator<T> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FieldComparator<BytesRef> setNextReader(AtomicReaderContext context) throws IOException {
|
||||
public FieldComparator<BytesRef> setNextReader(LeafReaderContext context) throws IOException {
|
||||
docTerms = getBinaryDocValues(context, field);
|
||||
docsWithField = getDocsWithField(context, field);
|
||||
if (docsWithField instanceof Bits.MatchAllBits) {
|
||||
|
|
|
@ -18,8 +18,7 @@ package org.apache.lucene.search;
|
|||
*/
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.Bits.MatchAllBits;
|
||||
|
@ -28,7 +27,7 @@ import org.apache.lucene.util.Bits.MatchNoBits;
|
|||
/**
|
||||
* A {@link Filter} that accepts all documents that have one or more values in a
|
||||
* given field. This {@link Filter} request {@link Bits} from
|
||||
* {@link AtomicReader#getDocsWithField}
|
||||
* {@link org.apache.lucene.index.LeafReader#getDocsWithField}
|
||||
*/
|
||||
public class FieldValueFilter extends Filter {
|
||||
private final String field;
|
||||
|
@ -76,7 +75,7 @@ public class FieldValueFilter extends Filter {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs)
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs)
|
||||
throws IOException {
|
||||
final Bits docsWithField = DocValues.getDocsWithField(
|
||||
context.reader(), field);
|
||||
|
|
|
@ -19,8 +19,7 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReader; // javadocs
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
/**
|
||||
|
@ -39,12 +38,12 @@ public abstract class Filter {
|
|||
* must refer to document IDs for that segment, not for
|
||||
* the top-level reader.
|
||||
*
|
||||
* @param context a {@link AtomicReaderContext} instance opened on the index currently
|
||||
* @param context a {@link org.apache.lucene.index.LeafReaderContext} instance opened on the index currently
|
||||
* searched on. Note, it is likely that the provided reader info does not
|
||||
* represent the whole underlying index i.e. if the index has more than
|
||||
* one segment the given reader only represents a single segment.
|
||||
* The provided context is always an atomic context, so you can call
|
||||
* {@link AtomicReader#fields()}
|
||||
* {@link org.apache.lucene.index.LeafReader#fields()}
|
||||
* on the context's reader, for example.
|
||||
*
|
||||
* @param acceptDocs
|
||||
|
@ -56,5 +55,5 @@ public abstract class Filter {
|
|||
* the filter doesn't accept any documents otherwise internal optimization might not apply
|
||||
* in the case an <i>empty</i> {@link DocIdSet} is returned.
|
||||
*/
|
||||
public abstract DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException;
|
||||
public abstract DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException;
|
||||
}
|
||||
|
|
|
@ -2,7 +2,7 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -36,7 +36,7 @@ public class FilterCollector implements Collector {
|
|||
}
|
||||
|
||||
@Override
|
||||
public LeafCollector getLeafCollector(AtomicReaderContext context) throws IOException {
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
|
||||
return in.getLeafCollector(context);
|
||||
}
|
||||
|
||||
|
|
|
@ -17,7 +17,7 @@ package org.apache.lucene.search;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -98,7 +98,7 @@ public class FilteredQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext ir, int i) throws IOException {
|
||||
public Explanation explain(LeafReaderContext ir, int i) throws IOException {
|
||||
Explanation inner = weight.explain (ir, i);
|
||||
Filter f = FilteredQuery.this.filter;
|
||||
DocIdSet docIdSet = f.getDocIdSet(ir, ir.reader().getLiveDocs());
|
||||
|
@ -124,7 +124,7 @@ public class FilteredQuery extends Query {
|
|||
|
||||
// return a filtering scorer
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
assert filter != null;
|
||||
|
||||
DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);
|
||||
|
@ -138,7 +138,7 @@ public class FilteredQuery extends Query {
|
|||
|
||||
// return a filtering top scorer
|
||||
@Override
|
||||
public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
assert filter != null;
|
||||
|
||||
DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);
|
||||
|
@ -477,14 +477,14 @@ public class FilteredQuery extends Query {
|
|||
* Returns a filtered {@link Scorer} based on this strategy.
|
||||
*
|
||||
* @param context
|
||||
* the {@link AtomicReaderContext} for which to return the {@link Scorer}.
|
||||
* the {@link org.apache.lucene.index.LeafReaderContext} for which to return the {@link Scorer}.
|
||||
* @param weight the {@link FilteredQuery} {@link Weight} to create the filtered scorer.
|
||||
* @param docIdSet the filter {@link DocIdSet} to apply
|
||||
* @return a filtered scorer
|
||||
*
|
||||
* @throws IOException if an {@link IOException} occurs
|
||||
*/
|
||||
public abstract Scorer filteredScorer(AtomicReaderContext context,
|
||||
public abstract Scorer filteredScorer(LeafReaderContext context,
|
||||
Weight weight, DocIdSet docIdSet) throws IOException;
|
||||
|
||||
/**
|
||||
|
@ -494,12 +494,12 @@ public class FilteredQuery extends Query {
|
|||
* wraps that into a BulkScorer.
|
||||
*
|
||||
* @param context
|
||||
* the {@link AtomicReaderContext} for which to return the {@link Scorer}.
|
||||
* the {@link org.apache.lucene.index.LeafReaderContext} for which to return the {@link Scorer}.
|
||||
* @param weight the {@link FilteredQuery} {@link Weight} to create the filtered scorer.
|
||||
* @param docIdSet the filter {@link DocIdSet} to apply
|
||||
* @return a filtered top scorer
|
||||
*/
|
||||
public BulkScorer filteredBulkScorer(AtomicReaderContext context,
|
||||
public BulkScorer filteredBulkScorer(LeafReaderContext context,
|
||||
Weight weight, boolean scoreDocsInOrder, DocIdSet docIdSet) throws IOException {
|
||||
Scorer scorer = filteredScorer(context, weight, docIdSet);
|
||||
if (scorer == null) {
|
||||
|
@ -522,7 +522,7 @@ public class FilteredQuery extends Query {
|
|||
public static class RandomAccessFilterStrategy extends FilterStrategy {
|
||||
|
||||
@Override
|
||||
public Scorer filteredScorer(AtomicReaderContext context, Weight weight, DocIdSet docIdSet) throws IOException {
|
||||
public Scorer filteredScorer(LeafReaderContext context, Weight weight, DocIdSet docIdSet) throws IOException {
|
||||
final DocIdSetIterator filterIter = docIdSet.iterator();
|
||||
if (filterIter == null) {
|
||||
// this means the filter does not accept any documents.
|
||||
|
@ -577,7 +577,7 @@ public class FilteredQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer filteredScorer(AtomicReaderContext context,
|
||||
public Scorer filteredScorer(LeafReaderContext context,
|
||||
Weight weight, DocIdSet docIdSet) throws IOException {
|
||||
final DocIdSetIterator filterIter = docIdSet.iterator();
|
||||
if (filterIter == null) {
|
||||
|
@ -613,7 +613,7 @@ public class FilteredQuery extends Query {
|
|||
*/
|
||||
private static final class QueryFirstFilterStrategy extends FilterStrategy {
|
||||
@Override
|
||||
public Scorer filteredScorer(final AtomicReaderContext context,
|
||||
public Scorer filteredScorer(final LeafReaderContext context,
|
||||
Weight weight,
|
||||
DocIdSet docIdSet) throws IOException {
|
||||
Bits filterAcceptDocs = docIdSet.bits();
|
||||
|
@ -628,7 +628,7 @@ public class FilteredQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public BulkScorer filteredBulkScorer(final AtomicReaderContext context,
|
||||
public BulkScorer filteredBulkScorer(final LeafReaderContext context,
|
||||
Weight weight,
|
||||
boolean scoreDocsInOrder, // ignored (we always top-score in order)
|
||||
DocIdSet docIdSet) throws IOException {
|
||||
|
|
|
@ -32,7 +32,7 @@ import java.util.concurrent.ExecutorService;
|
|||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.DirectoryReader; // javadocs
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.MultiFields;
|
||||
|
@ -81,7 +81,7 @@ public class IndexSearcher {
|
|||
// NOTE: these members might change in incompatible ways
|
||||
// in the next release
|
||||
protected final IndexReaderContext readerContext;
|
||||
protected final List<AtomicReaderContext> leafContexts;
|
||||
protected final List<LeafReaderContext> leafContexts;
|
||||
/** used with executor - each slice holds a set of leafs executed within one thread */
|
||||
protected final LeafSlice[] leafSlices;
|
||||
|
||||
|
@ -164,9 +164,9 @@ public class IndexSearcher {
|
|||
/**
|
||||
* Expert: Creates an array of leaf slices each holding a subset of the given leaves.
|
||||
* Each {@link LeafSlice} is executed in a single thread. By default there
|
||||
* will be one {@link LeafSlice} per leaf ({@link AtomicReaderContext}).
|
||||
* will be one {@link LeafSlice} per leaf ({@link org.apache.lucene.index.LeafReaderContext}).
|
||||
*/
|
||||
protected LeafSlice[] slices(List<AtomicReaderContext> leaves) {
|
||||
protected LeafSlice[] slices(List<LeafReaderContext> leaves) {
|
||||
LeafSlice[] slices = new LeafSlice[leaves.size()];
|
||||
for (int i = 0; i < slices.length; i++) {
|
||||
slices[i] = new LeafSlice(leaves.get(i));
|
||||
|
@ -472,7 +472,7 @@ public class IndexSearcher {
|
|||
* @throws BooleanQuery.TooManyClauses If a query would exceed
|
||||
* {@link BooleanQuery#getMaxClauseCount()} clauses.
|
||||
*/
|
||||
protected TopDocs search(List<AtomicReaderContext> leaves, Weight weight, ScoreDoc after, int nDocs) throws IOException {
|
||||
protected TopDocs search(List<LeafReaderContext> leaves, Weight weight, ScoreDoc after, int nDocs) throws IOException {
|
||||
// single thread
|
||||
int limit = reader.maxDoc();
|
||||
if (limit == 0) {
|
||||
|
@ -558,7 +558,7 @@ public class IndexSearcher {
|
|||
* whether or not the fields in the returned {@link FieldDoc} instances should
|
||||
* be set by specifying fillFields.
|
||||
*/
|
||||
protected TopFieldDocs search(List<AtomicReaderContext> leaves, Weight weight, FieldDoc after, int nDocs,
|
||||
protected TopFieldDocs search(List<LeafReaderContext> leaves, Weight weight, FieldDoc after, int nDocs,
|
||||
Sort sort, boolean fillFields, boolean doDocScores, boolean doMaxScore) throws IOException {
|
||||
// single thread
|
||||
int limit = reader.maxDoc();
|
||||
|
@ -593,13 +593,13 @@ public class IndexSearcher {
|
|||
* @throws BooleanQuery.TooManyClauses If a query would exceed
|
||||
* {@link BooleanQuery#getMaxClauseCount()} clauses.
|
||||
*/
|
||||
protected void search(List<AtomicReaderContext> leaves, Weight weight, Collector collector)
|
||||
protected void search(List<LeafReaderContext> leaves, Weight weight, Collector collector)
|
||||
throws IOException {
|
||||
|
||||
// TODO: should we make this
|
||||
// threaded...? the Collector could be sync'd?
|
||||
// always use single thread:
|
||||
for (AtomicReaderContext ctx : leaves) { // search each subreader
|
||||
for (LeafReaderContext ctx : leaves) { // search each subreader
|
||||
final LeafCollector leafCollector;
|
||||
try {
|
||||
leafCollector = collector.getLeafCollector(ctx);
|
||||
|
@ -659,7 +659,7 @@ public class IndexSearcher {
|
|||
*/
|
||||
protected Explanation explain(Weight weight, int doc) throws IOException {
|
||||
int n = ReaderUtil.subIndex(doc, leafContexts);
|
||||
final AtomicReaderContext ctx = leafContexts.get(n);
|
||||
final LeafReaderContext ctx = leafContexts.get(n);
|
||||
int deBasedDoc = doc - ctx.docBase;
|
||||
|
||||
return weight.explain(ctx, deBasedDoc);
|
||||
|
@ -778,7 +778,7 @@ public class IndexSearcher {
|
|||
weight, after, nDocs, sort, true, doDocScores || sort.needsScores(), doMaxScore);
|
||||
lock.lock();
|
||||
try {
|
||||
final AtomicReaderContext ctx = slice.leaves[0];
|
||||
final LeafReaderContext ctx = slice.leaves[0];
|
||||
final int base = ctx.docBase;
|
||||
final LeafCollector collector = hq.getLeafCollector(ctx);
|
||||
collector.setScorer(fakeScorer);
|
||||
|
@ -858,9 +858,9 @@ public class IndexSearcher {
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public static class LeafSlice {
|
||||
final AtomicReaderContext[] leaves;
|
||||
final LeafReaderContext[] leaves;
|
||||
|
||||
public LeafSlice(AtomicReaderContext... leaves) {
|
||||
public LeafSlice(LeafReaderContext... leaves) {
|
||||
this.leaves = leaves;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,8 +19,6 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
|
||||
/**
|
||||
* <p>Collector decouples the score from the collected doc:
|
||||
* the score computation is skipped entirely if it's not
|
||||
|
@ -90,7 +88,7 @@ public interface LeafCollector {
|
|||
* number.
|
||||
* <p>Note: The collection of the current segment can be terminated by throwing
|
||||
* a {@link CollectionTerminatedException}. In this case, the last docs of the
|
||||
* current {@link AtomicReaderContext} will be skipped and {@link IndexSearcher}
|
||||
* current {@link org.apache.lucene.index.LeafReaderContext} will be skipped and {@link IndexSearcher}
|
||||
* will swallow the exception and continue collection with the next leaf.
|
||||
* <p>
|
||||
* Note: This is called in an inner search loop. For good search performance,
|
||||
|
|
|
@ -17,7 +17,7 @@ package org.apache.lucene.search;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.util.ToStringUtils;
|
||||
|
@ -114,12 +114,12 @@ public class MatchAllDocsQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
return new MatchAllScorer(context.reader(), acceptDocs, this, queryWeight);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc) {
|
||||
public Explanation explain(LeafReaderContext context, int doc) {
|
||||
// explain query weight
|
||||
Explanation queryExpl = new ComplexExplanation
|
||||
(true, queryWeight, "MatchAllDocsQuery, product of:");
|
||||
|
|
|
@ -20,9 +20,7 @@ package org.apache.lucene.search;
|
|||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.search.Collector;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
|
||||
/**
|
||||
* A {@link Collector} which allows running a search with several
|
||||
|
@ -95,7 +93,7 @@ public class MultiCollector implements Collector {
|
|||
}
|
||||
|
||||
@Override
|
||||
public LeafCollector getLeafCollector(AtomicReaderContext context) throws IOException {
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
|
||||
final LeafCollector[] leafCollectors = new LeafCollector[collectors.length];
|
||||
for (int i = 0; i < collectors.length; ++i) {
|
||||
leafCollectors[i] = collectors[i].getLeafCollector(context);
|
||||
|
|
|
@ -20,9 +20,9 @@ package org.apache.lucene.search;
|
|||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexReaderContext;
|
||||
|
@ -179,9 +179,9 @@ public class MultiPhraseQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
assert !termArrays.isEmpty();
|
||||
final AtomicReader reader = context.reader();
|
||||
final LeafReader reader = context.reader();
|
||||
final Bits liveDocs = acceptDocs;
|
||||
|
||||
PhraseQuery.PostingsAndFreq[] postingsFreqs = new PhraseQuery.PostingsAndFreq[termArrays.size()];
|
||||
|
@ -256,7 +256,7 @@ public class MultiPhraseQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
|
||||
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
|
||||
Scorer scorer = scorer(context, context.reader().getLiveDocs());
|
||||
if (scorer != null) {
|
||||
int newDoc = scorer.advance(doc);
|
||||
|
@ -473,7 +473,7 @@ class UnionDocsAndPositionsEnum extends DocsAndPositionsEnum {
|
|||
private IntQueue _posList;
|
||||
private long cost;
|
||||
|
||||
public UnionDocsAndPositionsEnum(Bits liveDocs, AtomicReaderContext context, Term[] terms, Map<Term,TermContext> termContexts, TermsEnum termsEnum) throws IOException {
|
||||
public UnionDocsAndPositionsEnum(Bits liveDocs, LeafReaderContext context, Term[] terms, Map<Term,TermContext> termContexts, TermsEnum termsEnum) throws IOException {
|
||||
List<DocsAndPositionsEnum> docsEnums = new LinkedList<>();
|
||||
for (int i = 0; i < terms.length; i++) {
|
||||
final Term term = terms[i];
|
||||
|
|
|
@ -19,10 +19,10 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
|
@ -83,8 +83,8 @@ public class MultiTermQueryWrapperFilter<Q extends MultiTermQuery> extends Filte
|
|||
* results.
|
||||
*/
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
final AtomicReader reader = context.reader();
|
||||
public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
final LeafReader reader = context.reader();
|
||||
final Fields fields = reader.fields();
|
||||
if (fields == null) {
|
||||
// reader has no fields
|
||||
|
|
|
@ -22,11 +22,11 @@ import java.util.ArrayList;
|
|||
import java.util.Arrays;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.IndexReaderContext;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.TermContext;
|
||||
|
@ -245,9 +245,9 @@ public class PhraseQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
assert !terms.isEmpty();
|
||||
final AtomicReader reader = context.reader();
|
||||
final LeafReader reader = context.reader();
|
||||
final Bits liveDocs = acceptDocs;
|
||||
PostingsAndFreq[] postingsFreqs = new PostingsAndFreq[terms.size()];
|
||||
|
||||
|
@ -292,12 +292,12 @@ public class PhraseQuery extends Query {
|
|||
}
|
||||
|
||||
// only called from assert
|
||||
private boolean termNotInReader(AtomicReader reader, Term term) throws IOException {
|
||||
private boolean termNotInReader(LeafReader reader, Term term) throws IOException {
|
||||
return reader.docFreq(term) == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
|
||||
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
|
||||
Scorer scorer = scorer(context, context.reader().getLiveDocs());
|
||||
if (scorer != null) {
|
||||
int newDoc = scorer.advance(doc);
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
|
||||
/**
|
||||
* A {@link Collector} implementation which wraps another
|
||||
|
@ -33,7 +33,7 @@ public class PositiveScoresOnlyCollector extends FilterCollector {
|
|||
}
|
||||
|
||||
@Override
|
||||
public LeafCollector getLeafCollector(AtomicReaderContext context)
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context)
|
||||
throws IOException {
|
||||
return new FilterLeafCollector(super.getLeafCollector(context)) {
|
||||
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.util.Arrays;
|
|||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
|
||||
/** A {@link Rescorer} that uses a provided Query to assign
|
||||
* scores to the first-pass hits.
|
||||
|
@ -58,7 +58,7 @@ public abstract class QueryRescorer extends Rescorer {
|
|||
}
|
||||
});
|
||||
|
||||
List<AtomicReaderContext> leaves = searcher.getIndexReader().leaves();
|
||||
List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
|
||||
|
||||
Weight weight = searcher.createNormalizedWeight(query);
|
||||
|
||||
|
@ -72,7 +72,7 @@ public abstract class QueryRescorer extends Rescorer {
|
|||
while (hitUpto < hits.length) {
|
||||
ScoreDoc hit = hits[hitUpto];
|
||||
int docID = hit.doc;
|
||||
AtomicReaderContext readerContext = null;
|
||||
LeafReaderContext readerContext = null;
|
||||
while (docID >= endDoc) {
|
||||
readerUpto++;
|
||||
readerContext = leaves.get(readerUpto);
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
/**
|
||||
|
@ -49,9 +49,9 @@ public class QueryWrapperFilter extends Filter {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DocIdSet getDocIdSet(final AtomicReaderContext context, final Bits acceptDocs) throws IOException {
|
||||
public DocIdSet getDocIdSet(final LeafReaderContext context, final Bits acceptDocs) throws IOException {
|
||||
// get a private context that is used to rewrite, createWeight and score eventually
|
||||
final AtomicReaderContext privateContext = context.reader().getContext();
|
||||
final LeafReaderContext privateContext = context.reader().getContext();
|
||||
final Weight weight = new IndexSearcher(privateContext).createNormalizedWeight(query);
|
||||
return new DocIdSet() {
|
||||
@Override
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
|
||||
/**
|
||||
* Base {@link Collector} implementation that is used to collect all contexts.
|
||||
|
@ -29,13 +29,13 @@ import org.apache.lucene.index.AtomicReaderContext;
|
|||
public abstract class SimpleCollector implements Collector, LeafCollector {
|
||||
|
||||
@Override
|
||||
public final LeafCollector getLeafCollector(AtomicReaderContext context) throws IOException {
|
||||
public final LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
|
||||
doSetNextReader(context);
|
||||
return this;
|
||||
}
|
||||
|
||||
/** This method is called before collecting <code>context</code>. */
|
||||
protected void doSetNextReader(AtomicReaderContext context) throws IOException {}
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {}
|
||||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) throws IOException {
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.util.Arrays;
|
|||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
|
||||
/**
|
||||
* A {@link Rescorer} that re-sorts according to a provided
|
||||
|
@ -51,7 +51,7 @@ public class SortRescorer extends Rescorer {
|
|||
}
|
||||
});
|
||||
|
||||
List<AtomicReaderContext> leaves = searcher.getIndexReader().leaves();
|
||||
List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
|
||||
|
||||
TopFieldCollector collector = TopFieldCollector.create(sort, topN, true, true, true, false);
|
||||
|
||||
|
@ -66,7 +66,7 @@ public class SortRescorer extends Rescorer {
|
|||
while (hitUpto < hits.length) {
|
||||
ScoreDoc hit = hits[hitUpto];
|
||||
int docID = hit.doc;
|
||||
AtomicReaderContext readerContext = null;
|
||||
LeafReaderContext readerContext = null;
|
||||
while (docID >= endDoc) {
|
||||
readerUpto++;
|
||||
readerContext = leaves.get(readerUpto);
|
||||
|
|
|
@ -19,12 +19,10 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.index.SortedNumericDocValues;
|
||||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.SortField;
|
||||
|
||||
/**
|
||||
* SortField for {@link SortedNumericDocValues}.
|
||||
|
@ -139,28 +137,28 @@ public class SortedNumericSortField extends SortField {
|
|||
case INT:
|
||||
return new FieldComparator.IntComparator(numHits, getField(), (Integer) missingValue) {
|
||||
@Override
|
||||
protected NumericDocValues getNumericDocValues(AtomicReaderContext context, String field) throws IOException {
|
||||
protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
|
||||
return SortedNumericSelector.wrap(DocValues.getSortedNumeric(context.reader(), field), selector, type);
|
||||
}
|
||||
};
|
||||
case FLOAT:
|
||||
return new FieldComparator.FloatComparator(numHits, getField(), (Float) missingValue) {
|
||||
@Override
|
||||
protected NumericDocValues getNumericDocValues(AtomicReaderContext context, String field) throws IOException {
|
||||
protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
|
||||
return SortedNumericSelector.wrap(DocValues.getSortedNumeric(context.reader(), field), selector, type);
|
||||
}
|
||||
};
|
||||
case LONG:
|
||||
return new FieldComparator.LongComparator(numHits, getField(), (Long) missingValue) {
|
||||
@Override
|
||||
protected NumericDocValues getNumericDocValues(AtomicReaderContext context, String field) throws IOException {
|
||||
protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
|
||||
return SortedNumericSelector.wrap(DocValues.getSortedNumeric(context.reader(), field), selector, type);
|
||||
}
|
||||
};
|
||||
case DOUBLE:
|
||||
return new FieldComparator.DoubleComparator(numHits, getField(), (Double) missingValue) {
|
||||
@Override
|
||||
protected NumericDocValues getNumericDocValues(AtomicReaderContext context, String field) throws IOException {
|
||||
protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) throws IOException {
|
||||
return SortedNumericSelector.wrap(DocValues.getSortedNumeric(context.reader(), field), selector, type);
|
||||
}
|
||||
};
|
||||
|
|
|
@ -19,12 +19,10 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.SortedSetDocValues;
|
||||
import org.apache.lucene.search.FieldComparator;
|
||||
import org.apache.lucene.search.SortField;
|
||||
|
||||
/**
|
||||
* SortField for {@link SortedSetDocValues}.
|
||||
|
@ -124,7 +122,7 @@ public class SortedSetSortField extends SortField {
|
|||
public FieldComparator<?> getComparator(int numHits, int sortPos) throws IOException {
|
||||
return new FieldComparator.TermOrdValComparator(numHits, getField(), missingValue == STRING_LAST) {
|
||||
@Override
|
||||
protected SortedDocValues getSortedDocValues(AtomicReaderContext context, String field) throws IOException {
|
||||
protected SortedDocValues getSortedDocValues(LeafReaderContext context, String field) throws IOException {
|
||||
SortedSetDocValues sortedSet = DocValues.getSortedSet(context.reader(), field);
|
||||
return SortedSetSelector.wrap(sortedSet, selector);
|
||||
}
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexReaderContext;
|
||||
|
@ -46,7 +46,7 @@ abstract class TermCollectingRewrite<Q extends Query> extends MultiTermQuery.Rew
|
|||
|
||||
final void collectTerms(IndexReader reader, MultiTermQuery query, TermCollector collector) throws IOException {
|
||||
IndexReaderContext topReaderContext = reader.getContext();
|
||||
for (AtomicReaderContext context : topReaderContext.leaves()) {
|
||||
for (LeafReaderContext context : topReaderContext.leaves()) {
|
||||
final Fields fields = context.reader().fields();
|
||||
if (fields == null) {
|
||||
// reader has no fields
|
||||
|
@ -77,10 +77,10 @@ abstract class TermCollectingRewrite<Q extends Query> extends MultiTermQuery.Rew
|
|||
|
||||
static abstract class TermCollector {
|
||||
|
||||
protected AtomicReaderContext readerContext;
|
||||
protected LeafReaderContext readerContext;
|
||||
protected IndexReaderContext topReaderContext;
|
||||
|
||||
public void setReaderContext(IndexReaderContext topReaderContext, AtomicReaderContext readerContext) {
|
||||
public void setReaderContext(IndexReaderContext topReaderContext, LeafReaderContext readerContext) {
|
||||
this.readerContext = readerContext;
|
||||
this.topReaderContext = topReaderContext;
|
||||
}
|
||||
|
|
|
@ -20,9 +20,9 @@ package org.apache.lucene.search;
|
|||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.IndexReaderContext;
|
||||
import org.apache.lucene.index.ReaderUtil;
|
||||
import org.apache.lucene.index.Term;
|
||||
|
@ -75,7 +75,7 @@ public class TermQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
assert termStates.topReaderContext == ReaderUtil.getTopLevelContext(context) : "The top-reader used to create Weight (" + termStates.topReaderContext + ") is not the same as the current reader's top-reader (" + ReaderUtil.getTopLevelContext(context);
|
||||
final TermsEnum termsEnum = getTermsEnum(context);
|
||||
if (termsEnum == null) {
|
||||
|
@ -90,7 +90,7 @@ public class TermQuery extends Query {
|
|||
* Returns a {@link TermsEnum} positioned at this weights Term or null if
|
||||
* the term does not exist in the given context
|
||||
*/
|
||||
private TermsEnum getTermsEnum(AtomicReaderContext context) throws IOException {
|
||||
private TermsEnum getTermsEnum(LeafReaderContext context) throws IOException {
|
||||
final TermState state = termStates.get(context.ord);
|
||||
if (state == null) { // term is not present in that reader
|
||||
assert termNotInReader(context.reader(), term) : "no termstate found but term exists in reader term=" + term;
|
||||
|
@ -102,14 +102,14 @@ public class TermQuery extends Query {
|
|||
return termsEnum;
|
||||
}
|
||||
|
||||
private boolean termNotInReader(AtomicReader reader, Term term) throws IOException {
|
||||
private boolean termNotInReader(LeafReader reader, Term term) throws IOException {
|
||||
// only called from assert
|
||||
//System.out.println("TQ.termNotInReader reader=" + reader + " term=" + field + ":" + bytes.utf8ToString());
|
||||
return reader.docFreq(term) == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
|
||||
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
|
||||
Scorer scorer = scorer(context, context.reader().getLiveDocs());
|
||||
if (scorer != null) {
|
||||
int newDoc = scorer.advance(doc);
|
||||
|
|
|
@ -17,7 +17,7 @@ package org.apache.lucene.search;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
|
||||
|
@ -132,7 +132,7 @@ public class TimeLimitingCollector implements Collector {
|
|||
}
|
||||
|
||||
@Override
|
||||
public LeafCollector getLeafCollector(AtomicReaderContext context) throws IOException {
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
|
||||
this.docBase = context.docBase;
|
||||
if (Long.MIN_VALUE == t0) {
|
||||
setBaseline();
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.search.FieldValueHitQueue.Entry;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
|
||||
|
@ -92,7 +92,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
this.docBase = context.docBase;
|
||||
queue.setComparator(0, comparator.setNextReader(context));
|
||||
comparator = queue.firstComparator;
|
||||
|
@ -446,7 +446,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
docBase = context.docBase;
|
||||
for (int i = 0; i < comparators.length; i++) {
|
||||
queue.setComparator(i, comparators[i].setNextReader(context));
|
||||
|
@ -1001,7 +1001,7 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
docBase = context.docBase;
|
||||
afterDoc = after.doc - docBase;
|
||||
for (int i = 0; i < comparators.length; i++) {
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
|
||||
/**
|
||||
* A {@link Collector} implementation that collects the top-scoring hits,
|
||||
|
@ -113,7 +113,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
super.doSetNextReader(context);
|
||||
afterDoc = after.doc - context.docBase;
|
||||
}
|
||||
|
@ -208,7 +208,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
super.doSetNextReader(context);
|
||||
afterDoc = after.doc - context.docBase;
|
||||
}
|
||||
|
@ -300,7 +300,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(AtomicReaderContext context) throws IOException {
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
docBase = context.docBase;
|
||||
}
|
||||
|
||||
|
|
|
@ -19,8 +19,7 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReader; // javadocs
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.IndexReaderContext; // javadocs
|
||||
import org.apache.lucene.search.similarities.Similarity;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -32,10 +31,10 @@ import org.apache.lucene.util.Bits;
|
|||
* {@link Query}, so that a {@link Query} instance can be reused. <br>
|
||||
* {@link IndexSearcher} dependent state of the query should reside in the
|
||||
* {@link Weight}. <br>
|
||||
* {@link AtomicReader} dependent state should reside in the {@link Scorer}.
|
||||
* {@link org.apache.lucene.index.LeafReader} dependent state should reside in the {@link Scorer}.
|
||||
* <p>
|
||||
* Since {@link Weight} creates {@link Scorer} instances for a given
|
||||
* {@link AtomicReaderContext} ({@link #scorer(AtomicReaderContext, Bits)})
|
||||
* {@link org.apache.lucene.index.LeafReaderContext} ({@link #scorer(org.apache.lucene.index.LeafReaderContext, Bits)})
|
||||
* callers must maintain the relationship between the searcher's top-level
|
||||
* {@link IndexReaderContext} and the context used to create a {@link Scorer}.
|
||||
* <p>
|
||||
|
@ -50,7 +49,7 @@ import org.apache.lucene.util.Bits;
|
|||
* <li>The query normalization factor is passed to {@link #normalize(float, float)}. At
|
||||
* this point the weighting is complete.
|
||||
* <li>A <code>Scorer</code> is constructed by
|
||||
* {@link #scorer(AtomicReaderContext, Bits)}.
|
||||
* {@link #scorer(org.apache.lucene.index.LeafReaderContext, Bits)}.
|
||||
* </ol>
|
||||
*
|
||||
* @since 2.9
|
||||
|
@ -65,7 +64,7 @@ public abstract class Weight {
|
|||
* @return an Explanation for the score
|
||||
* @throws IOException if an {@link IOException} occurs
|
||||
*/
|
||||
public abstract Explanation explain(AtomicReaderContext context, int doc) throws IOException;
|
||||
public abstract Explanation explain(LeafReaderContext context, int doc) throws IOException;
|
||||
|
||||
/** The query that this concerns. */
|
||||
public abstract Query getQuery();
|
||||
|
@ -89,7 +88,7 @@ public abstract class Weight {
|
|||
* query.
|
||||
*
|
||||
* @param context
|
||||
* the {@link AtomicReaderContext} for which to return the {@link Scorer}.
|
||||
* the {@link org.apache.lucene.index.LeafReaderContext} for which to return the {@link Scorer}.
|
||||
* @param acceptDocs
|
||||
* Bits that represent the allowable docs to match (typically deleted docs
|
||||
* but possibly filtering other documents)
|
||||
|
@ -97,7 +96,7 @@ public abstract class Weight {
|
|||
* @return a {@link Scorer} which scores documents in/out-of order.
|
||||
* @throws IOException if there is a low-level I/O error
|
||||
*/
|
||||
public abstract Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException;
|
||||
public abstract Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException;
|
||||
|
||||
/**
|
||||
* Optional method, to return a {@link BulkScorer} to
|
||||
|
@ -108,7 +107,7 @@ public abstract class Weight {
|
|||
* collects the resulting hits.
|
||||
*
|
||||
* @param context
|
||||
* the {@link AtomicReaderContext} for which to return the {@link Scorer}.
|
||||
* the {@link org.apache.lucene.index.LeafReaderContext} for which to return the {@link Scorer}.
|
||||
* @param scoreDocsInOrder
|
||||
* specifies whether in-order scoring of documents is required. Note
|
||||
* that if set to false (i.e., out-of-order scoring is required),
|
||||
|
@ -126,7 +125,7 @@ public abstract class Weight {
|
|||
* passes them to a collector.
|
||||
* @throws IOException if there is a low-level I/O error
|
||||
*/
|
||||
public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
|
||||
Scorer scorer = scorer(context, acceptDocs);
|
||||
if (scorer == null) {
|
||||
|
@ -199,7 +198,7 @@ public abstract class Weight {
|
|||
* Returns true iff this implementation scores docs only out of order. This
|
||||
* method is used in conjunction with {@link Collector}'s
|
||||
* {@link LeafCollector#acceptsDocsOutOfOrder() acceptsDocsOutOfOrder} and
|
||||
* {@link #bulkScorer(AtomicReaderContext, boolean, Bits)} to
|
||||
* {@link #bulkScorer(org.apache.lucene.index.LeafReaderContext, boolean, Bits)} to
|
||||
* create a matching {@link Scorer} instance for a given {@link Collector}, or
|
||||
* vice versa.
|
||||
* <p>
|
||||
|
|
|
@ -436,21 +436,21 @@ on the built-in available scoring models and extending or changing Similarity.
|
|||
that scores via a {@link org.apache.lucene.search.similarities.Similarity Similarity} will just defer to the Similarity's implementation:
|
||||
{@link org.apache.lucene.search.similarities.Similarity.SimWeight#normalize SimWeight#normalize(float,float)}.</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.search.Weight#scorer(org.apache.lucene.index.AtomicReaderContext, org.apache.lucene.util.Bits)
|
||||
scorer(AtomicReaderContext context, Bits acceptDocs)} —
|
||||
{@link org.apache.lucene.search.Weight#scorer(org.apache.lucene.index.LeafReaderContext, org.apache.lucene.util.Bits)
|
||||
scorer(LeafReaderContext context, Bits acceptDocs)} —
|
||||
Construct a new {@link org.apache.lucene.search.Scorer Scorer} for this Weight. See <a href="#scorerClass">The Scorer Class</a>
|
||||
below for help defining a Scorer. As the name implies, the Scorer is responsible for doing the actual scoring of documents
|
||||
given the Query.
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.search.Weight#bulkScorer(org.apache.lucene.index.AtomicReaderContext, boolean, org.apache.lucene.util.Bits)
|
||||
scorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs)} —
|
||||
{@link org.apache.lucene.search.Weight#bulkScorer(org.apache.lucene.index.LeafReaderContext, boolean, org.apache.lucene.util.Bits)
|
||||
scorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs)} —
|
||||
Construct a new {@link org.apache.lucene.search.BulkScorer BulkScorer} for this Weight. See <a href="#bulkScorerClass">The BulkScorer Class</a>
|
||||
below for help defining a BulkScorer. This is an optional method, and most queries do not implement it.
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.search.Weight#explain(org.apache.lucene.index.AtomicReaderContext, int)
|
||||
explain(AtomicReaderContext context, int doc)} — Provide a means for explaining why a given document was
|
||||
{@link org.apache.lucene.search.Weight#explain(org.apache.lucene.index.LeafReaderContext, int)
|
||||
explain(LeafReaderContext context, int doc)} — Provide a means for explaining why a given document was
|
||||
scored the way it was.
|
||||
Typically a weight such as TermWeight
|
||||
that scores via a {@link org.apache.lucene.search.similarities.Similarity Similarity} will make use of the Similarity's implementation:
|
||||
|
|
|
@ -17,7 +17,7 @@ package org.apache.lucene.search.payloads;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.search.ComplexExplanation;
|
||||
import org.apache.lucene.search.Explanation;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
|
@ -148,13 +148,13 @@ public class PayloadNearQuery extends SpanNearQuery {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
return new PayloadNearSpanScorer(query.getSpans(context, acceptDocs, termContexts), this,
|
||||
similarity, similarity.simScorer(stats, context));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
|
||||
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
|
||||
PayloadNearSpanScorer scorer = (PayloadNearSpanScorer) scorer(context, context.reader().getLiveDocs());
|
||||
if (scorer != null) {
|
||||
int newDoc = scorer.advance(doc);
|
||||
|
|
|
@ -26,7 +26,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexReaderContext;
|
||||
import org.apache.lucene.index.Term;
|
||||
|
@ -184,8 +184,8 @@ public class PayloadSpanUtil {
|
|||
for (Term term : terms) {
|
||||
termContexts.put(term, TermContext.build(context, term));
|
||||
}
|
||||
for (AtomicReaderContext atomicReaderContext : context.leaves()) {
|
||||
final Spans spans = query.getSpans(atomicReaderContext, atomicReaderContext.reader().getLiveDocs(), termContexts);
|
||||
for (LeafReaderContext leafReaderContext : context.leaves()) {
|
||||
final Spans spans = query.getSpans(leafReaderContext, leafReaderContext.reader().getLiveDocs(), termContexts);
|
||||
while (spans.next() == true) {
|
||||
if (spans.isPayloadAvailable()) {
|
||||
Collection<byte[]> payload = spans.getPayload();
|
||||
|
|
|
@ -17,7 +17,7 @@ package org.apache.lucene.search.payloads;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
|
@ -79,7 +79,7 @@ public class PayloadTermQuery extends SpanTermQuery {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
|
||||
return new PayloadTermSpanScorer((TermSpans) query.getSpans(context, acceptDocs, termContexts),
|
||||
this, similarity.simScorer(stats, context));
|
||||
}
|
||||
|
@ -175,7 +175,7 @@ public class PayloadTermQuery extends SpanTermQuery {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
|
||||
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
|
||||
PayloadTermSpanScorer scorer = (PayloadTermSpanScorer) scorer(context, context.reader().getLiveDocs());
|
||||
if (scorer != null) {
|
||||
int newDoc = scorer.advance(doc);
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.search.similarities;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.FieldInvertState;
|
||||
import org.apache.lucene.index.NumericDocValues;
|
||||
import org.apache.lucene.search.CollectionStatistics;
|
||||
|
@ -212,7 +212,7 @@ public class BM25Similarity extends Similarity {
|
|||
}
|
||||
|
||||
@Override
|
||||
public final SimScorer simScorer(SimWeight stats, AtomicReaderContext context) throws IOException {
|
||||
public final SimScorer simScorer(SimWeight stats, LeafReaderContext context) throws IOException {
|
||||
BM25Stats bm25stats = (BM25Stats) stats;
|
||||
return new BM25DocScorer(bm25stats, context.reader().getNormValues(bm25stats.field));
|
||||
}
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.search.similarities;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.FieldInvertState;
|
||||
import org.apache.lucene.search.CollectionStatistics;
|
||||
import org.apache.lucene.search.Explanation;
|
||||
|
@ -57,7 +57,7 @@ public class MultiSimilarity extends Similarity {
|
|||
}
|
||||
|
||||
@Override
|
||||
public SimScorer simScorer(SimWeight stats, AtomicReaderContext context) throws IOException {
|
||||
public SimScorer simScorer(SimWeight stats, LeafReaderContext context) throws IOException {
|
||||
SimScorer subScorers[] = new SimScorer[sims.length];
|
||||
for (int i = 0; i < subScorers.length; i++) {
|
||||
subScorers[i] = sims[i].simScorer(((MultiStats)stats).subStats[i], context);
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue