LUCENE-2858: fix queries module

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene2858@1237320 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2012-01-29 16:00:32 +00:00
parent 6d4484cb4a
commit 45c377007f
51 changed files with 117 additions and 111 deletions

View File

@ -22,8 +22,8 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Iterator;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.search.BitsFilteredDocIdSet;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.DocIdSet;
@ -52,7 +52,7 @@ public class BooleanFilter extends Filter implements Iterable<FilterClause> {
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
FixedBitSet res = null;
final IndexReader reader = context.reader;
final AtomicIndexReader reader = context.reader();
boolean hasShouldClauses = false;
for (final FilterClause fc : clauses) {

View File

@ -17,8 +17,8 @@ package org.apache.lucene.queries;
* limitations under the License.
*/
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.search.BitsFilteredDocIdSet;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
@ -129,7 +129,7 @@ public class ChainedFilter extends Filter {
private OpenBitSetDISI initialResult(AtomicReaderContext context, int logic, int[] index)
throws IOException {
IndexReader reader = context.reader;
AtomicIndexReader reader = context.reader();
OpenBitSetDISI result;
/**
* First AND operation takes place against a completely false

View File

@ -19,7 +19,7 @@ package org.apache.lucene.queries;
import java.io.IOException;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.index.IndexReader; // for javadocs
import org.apache.lucene.search.Explanation;
import org.apache.lucene.search.FieldCache; // for javadocs

View File

@ -22,7 +22,7 @@ import java.util.Set;
import java.util.Arrays;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.ComplexExplanation;
import org.apache.lucene.search.Explanation;

View File

@ -18,7 +18,7 @@ package org.apache.lucene.queries;
*/
import org.apache.lucene.index.*;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.Filter;
import org.apache.lucene.util.Bits;
@ -55,7 +55,7 @@ public class TermsFilter extends Filter {
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
IndexReader reader = context.reader;
AtomicIndexReader reader = context.reader();
FixedBitSet result = new FixedBitSet(reader.maxDoc());
Fields fields = reader.fields();

View File

@ -19,7 +19,7 @@ package org.apache.lucene.queries.function;
import org.apache.lucene.search.*;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ToStringUtils;

View File

@ -18,7 +18,7 @@ package org.apache.lucene.queries.function;
*/
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.search.*;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.util.Bits;
@ -96,7 +96,7 @@ public class FunctionQuery extends Query {
@Override
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
return ((AllScorer)scorer(context, true, true, context.reader.getLiveDocs())).explain(doc);
return ((AllScorer)scorer(context, true, true, context.reader().getLiveDocs())).explain(doc);
}
}
@ -113,7 +113,7 @@ public class FunctionQuery extends Query {
super(w);
this.weight = w;
this.qWeight = qWeight;
this.reader = context.reader;
this.reader = context.reader();
this.maxDoc = reader.maxDoc();
this.liveDocs = acceptDocs;
vals = func.getValues(weight.context, context);

View File

@ -18,7 +18,7 @@ package org.apache.lucene.queries.function;
*/
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.FieldComparatorSource;
import org.apache.lucene.search.Scorer;

View File

@ -22,7 +22,7 @@ import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.ValueSourceScorer;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.UnicodeUtil;
@ -43,7 +43,7 @@ public abstract class StringIndexDocValues extends FunctionValues {
public StringIndexDocValues(ValueSource vs, AtomicReaderContext context, String field) throws IOException {
try {
termsIndex = FieldCache.DEFAULT.getTermsIndex(context.reader, field);
termsIndex = FieldCache.DEFAULT.getTermsIndex(context.reader(), field);
} catch (RuntimeException e) {
throw new StringIndexException(field, e);
}

View File

@ -19,7 +19,7 @@ package org.apache.lucene.queries.function.valuesource;
import java.io.IOException;
import java.util.Map;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.search.FieldCache;
@ -51,7 +51,7 @@ public class ByteFieldSource extends FieldCacheSource {
@Override
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
final byte[] arr = cache.getBytes(readerContext.reader, field, parser, false);
final byte[] arr = cache.getBytes(readerContext.reader(), field, parser, false);
return new FunctionValues() {
@Override

View File

@ -17,7 +17,7 @@ package org.apache.lucene.queries.function.valuesource;
* limitations under the License.
*/
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.docvalues.StringIndexDocValues;
import org.apache.lucene.queries.function.ValueSource; //javadoc
@ -35,7 +35,7 @@ public class BytesRefFieldSource extends FieldCacheSource {
}
@Override
public FunctionValues getValues(Map context, IndexReader.AtomicReaderContext readerContext) throws IOException {
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
return new StringIndexDocValues(this, readerContext, field) {
@Override

View File

@ -17,7 +17,7 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.docvalues.FloatDocValues;

View File

@ -16,7 +16,7 @@ package org.apache.lucene.queries.function.valuesource;
* limitations under the License.
*/
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.IndexSearcher;

View File

@ -17,7 +17,7 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;

View File

@ -17,7 +17,7 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.docvalues.DoubleDocValues;

View File

@ -20,7 +20,7 @@ package org.apache.lucene.queries.function.valuesource;
import java.io.IOException;
import java.util.Map;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSourceScorer;
@ -57,8 +57,8 @@ public class DoubleFieldSource extends FieldCacheSource {
@Override
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
final double[] arr = cache.getDoubles(readerContext.reader, field, parser, true);
final Bits valid = cache.getDocsWithField(readerContext.reader, field);
final double[] arr = cache.getDoubles(readerContext.reader(), field, parser, true);
final Bits valid = cache.getDocsWithField(readerContext.reader(), field);
return new DoubleDocValues(this) {
@Override
public double doubleVal(int doc) {

View File

@ -17,7 +17,7 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.docvalues.FloatDocValues;

View File

@ -20,7 +20,7 @@ package org.apache.lucene.queries.function.valuesource;
import java.io.IOException;
import java.util.Map;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.docvalues.FloatDocValues;
import org.apache.lucene.search.FieldCache;
@ -55,8 +55,8 @@ public class FloatFieldSource extends FieldCacheSource {
@Override
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
final float[] arr = cache.getFloats(readerContext.reader, field, parser, true);
final Bits valid = cache.getDocsWithField(readerContext.reader, field);
final float[] arr = cache.getFloats(readerContext.reader(), field, parser, true);
final Bits valid = cache.getDocsWithField(readerContext.reader(), field);
return new FloatDocValues(this) {
@Override

View File

@ -18,7 +18,7 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.*;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.similarities.Similarity;

View File

@ -18,7 +18,7 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.Explanation;

View File

@ -20,7 +20,7 @@ package org.apache.lucene.queries.function.valuesource;
import java.io.IOException;
import java.util.Map;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSourceScorer;
@ -57,8 +57,8 @@ public class IntFieldSource extends FieldCacheSource {
@Override
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
final int[] arr = cache.getInts(readerContext.reader, field, parser, true);
final Bits valid = cache.getDocsWithField(readerContext.reader, field);
final int[] arr = cache.getInts(readerContext.reader(), field, parser, true);
final Bits valid = cache.getDocsWithField(readerContext.reader(), field);
return new IntDocValues(this) {
final MutableValueInt val = new MutableValueInt();

View File

@ -21,7 +21,7 @@ import java.io.IOException;
import java.util.Map;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.docvalues.IntDocValues;
import org.apache.lucene.search.FieldCache.DocTerms;
@ -52,8 +52,8 @@ public class JoinDocFreqValueSource extends FieldCacheSource {
@Override
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException
{
final DocTerms terms = cache.getTerms(readerContext.reader, field, true );
final IndexReader top = ReaderUtil.getTopLevelContext(readerContext).reader;
final DocTerms terms = cache.getTerms(readerContext.reader(), field, true );
final IndexReader top = ReaderUtil.getTopLevelContext(readerContext).reader();
return new IntDocValues(this) {
BytesRef ref = new BytesRef();

View File

@ -17,7 +17,7 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.docvalues.FloatDocValues;

View File

@ -16,7 +16,7 @@ package org.apache.lucene.queries.function.valuesource;
* limitations under the License.
*/
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.docvalues.StrDocValues;

View File

@ -20,7 +20,7 @@ package org.apache.lucene.queries.function.valuesource;
import java.io.IOException;
import java.util.Map;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSourceScorer;
@ -66,8 +66,8 @@ public class LongFieldSource extends FieldCacheSource {
@Override
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
final long[] arr = cache.getLongs(readerContext.reader, field, parser, true);
final Bits valid = cache.getDocsWithField(readerContext.reader, field);
final long[] arr = cache.getLongs(readerContext.reader(), field, parser, true);
final Bits valid = cache.getDocsWithField(readerContext.reader(), field);
return new LongDocValues(this) {
@Override

View File

@ -16,7 +16,7 @@
*/
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.IndexSearcher;

View File

@ -17,7 +17,7 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.docvalues.BoolDocValues;

View File

@ -16,7 +16,7 @@ package org.apache.lucene.queries.function.valuesource;
* limitations under the License.
*/
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.docvalues.FloatDocValues;

View File

@ -16,7 +16,7 @@ package org.apache.lucene.queries.function.valuesource;
* limitations under the License.
*/
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.IndexSearcher;

View File

@ -18,7 +18,7 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.docvalues.FloatDocValues;
@ -57,7 +57,7 @@ public class NormValueSource extends ValueSource {
throw new UnsupportedOperationException("requires a TFIDFSimilarity (such as DefaultSimilarity)");
}
final TFIDFSimilarity similarity = (TFIDFSimilarity) sim;
DocValues dv = readerContext.reader.normValues(field);
DocValues dv = readerContext.reader().normValues(field);
if (dv == null) {
return new ConstDoubleDocValues(0.0, this);

View File

@ -16,7 +16,7 @@
*/
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.util.ReaderUtil;
@ -37,7 +37,7 @@ public class NumDocsValueSource extends ValueSource {
@Override
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
// Searcher has no numdocs so we must use the reader instead
return new ConstIntDocValues(ReaderUtil.getTopLevelContext(readerContext).reader.numDocs(), this);
return new ConstIntDocValues(ReaderUtil.getTopLevelContext(readerContext).reader().numDocs(), this);
}
@Override

View File

@ -21,7 +21,7 @@ import java.util.Map;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
@ -43,7 +43,7 @@ public class NumericIndexDocValueSource extends ValueSource {
@Override
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
final Source source = readerContext.reader.docValues(field)
final Source source = readerContext.reader().docValues(field)
.getSource();
Type type = source.type();
switch (type) {

View File

@ -17,9 +17,11 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.AtomicIndexReader;
import org.apache.lucene.index.CompositeIndexReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.SlowMultiReaderWrapper;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.index.SlowCompositeReaderWrapper;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.docvalues.IntDocValues;
@ -66,8 +68,11 @@ public class OrdFieldSource extends ValueSource {
@Override
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
final int off = readerContext.docBase;
final IndexReader topReader = ReaderUtil.getTopLevelContext(readerContext).reader;
final FieldCache.DocTermsIndex sindex = FieldCache.DEFAULT.getTermsIndex(new SlowMultiReaderWrapper(topReader), field);
final IndexReader topReader = ReaderUtil.getTopLevelContext(readerContext).reader();
final AtomicIndexReader r = topReader instanceof CompositeIndexReader
? new SlowCompositeReaderWrapper((CompositeIndexReader)topReader)
: (AtomicIndexReader) topReader;
final FieldCache.DocTermsIndex sindex = FieldCache.DEFAULT.getTermsIndex(r, field);
return new IntDocValues(this) {
protected String toTerm(String readableValue) {
return readableValue;

View File

@ -17,7 +17,7 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.docvalues.FloatDocValues;
@ -96,7 +96,7 @@ class QueryDocValues extends FloatDocValues {
super(vs);
this.readerContext = readerContext;
this.acceptDocs = readerContext.reader.getLiveDocs();
this.acceptDocs = readerContext.reader().getLiveDocs();
this.defVal = vs.defVal;
this.q = vs.q;
this.fcontext = fcontext;

View File

@ -17,7 +17,7 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.docvalues.FloatDocValues;

View File

@ -17,7 +17,7 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.docvalues.FloatDocValues;

View File

@ -17,9 +17,11 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.AtomicIndexReader;
import org.apache.lucene.index.CompositeIndexReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.SlowMultiReaderWrapper;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.index.SlowCompositeReaderWrapper;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.docvalues.IntDocValues;
@ -65,10 +67,13 @@ public class ReverseOrdFieldSource extends ValueSource {
// TODO: this is trappy? perhaps this query instead should make you pass a slow reader yourself?
@Override
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
final IndexReader topReader = ReaderUtil.getTopLevelContext(readerContext).reader;
final IndexReader topReader = ReaderUtil.getTopLevelContext(readerContext).reader();
final AtomicIndexReader r = topReader instanceof CompositeIndexReader
? new SlowCompositeReaderWrapper((CompositeIndexReader)topReader)
: (AtomicIndexReader) topReader;
final int off = readerContext.docBase;
final FieldCache.DocTermsIndex sindex = FieldCache.DEFAULT.getTermsIndex(new SlowMultiReaderWrapper(topReader), field);
final FieldCache.DocTermsIndex sindex = FieldCache.DEFAULT.getTermsIndex(r, field);
final int end = sindex.numOrd();
return new IntDocValues(this) {

View File

@ -17,7 +17,7 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.docvalues.FloatDocValues;
@ -66,7 +66,7 @@ public class ScaleFloatFunction extends ValueSource {
float maxVal = Float.NEGATIVE_INFINITY;
for (AtomicReaderContext leaf : leaves) {
int maxDoc = leaf.reader.maxDoc();
int maxDoc = leaf.reader().maxDoc();
FunctionValues vals = source.getValues(context, leaf);
for (int i=0; i<maxDoc; i++) {

View File

@ -19,7 +19,7 @@ package org.apache.lucene.queries.function.valuesource;
import java.io.IOException;
import java.util.Map;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.search.FieldCache;
@ -48,7 +48,7 @@ public class ShortFieldSource extends FieldCacheSource {
@Override
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
final short[] arr = cache.getShorts(readerContext.reader, field, parser, false);
final short[] arr = cache.getShorts(readerContext.reader(), field, parser, false);
return new FunctionValues() {
@Override

View File

@ -17,7 +17,7 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.docvalues.BoolDocValues;

View File

@ -17,7 +17,7 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.docvalues.FloatDocValues;

View File

@ -17,8 +17,8 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Terms;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
@ -50,15 +50,15 @@ public class SumTotalTermFreqValueSource extends ValueSource {
}
@Override
public FunctionValues getValues(Map context, IndexReader.AtomicReaderContext readerContext) throws IOException {
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
return (FunctionValues)context.get(this);
}
@Override
public void createWeight(Map context, IndexSearcher searcher) throws IOException {
long sumTotalTermFreq = 0;
for (IndexReader.AtomicReaderContext readerContext : searcher.getTopReaderContext().leaves()) {
Fields fields = readerContext.reader.fields();
for (AtomicReaderContext readerContext : searcher.getTopReaderContext().leaves()) {
Fields fields = readerContext.reader().fields();
if (fields == null) continue;
Terms terms = fields.terms(indexedField);
if (terms == null) continue;

View File

@ -18,7 +18,7 @@ package org.apache.lucene.queries.function.valuesource;
*/
import org.apache.lucene.index.*;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.docvalues.FloatDocValues;
import org.apache.lucene.search.DocIdSetIterator;
@ -42,7 +42,7 @@ public class TFValueSource extends TermFreqValueSource {
@Override
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
Fields fields = readerContext.reader.fields();
Fields fields = readerContext.reader().fields();
final Terms terms = fields.terms(field);
final Similarity sim = ((IndexSearcher)context.get("searcher")).getSimilarityProvider().get(field);
if (!(sim instanceof TFIDFSimilarity)) {

View File

@ -18,7 +18,7 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.*;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.docvalues.IntDocValues;
import org.apache.lucene.search.DocIdSetIterator;
@ -39,7 +39,7 @@ public class TermFreqValueSource extends DocFreqValueSource {
@Override
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
Fields fields = readerContext.reader.fields();
Fields fields = readerContext.reader().fields();
final Terms terms = fields.terms(field);
return new IntDocValues(this) {

View File

@ -17,7 +17,7 @@
package org.apache.lucene.queries.function.valuesource;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.docvalues.LongDocValues;
@ -54,15 +54,15 @@ public class TotalTermFreqValueSource extends ValueSource {
}
@Override
public FunctionValues getValues(Map context, IndexReader.AtomicReaderContext readerContext) throws IOException {
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
return (FunctionValues)context.get(this);
}
@Override
public void createWeight(Map context, IndexSearcher searcher) throws IOException {
long totalTermFreq = 0;
for (IndexReader.AtomicReaderContext readerContext : searcher.getTopReaderContext().leaves()) {
totalTermFreq += readerContext.reader.totalTermFreq(indexedField, indexedBytes);
for (AtomicReaderContext readerContext : searcher.getTopReaderContext().leaves()) {
totalTermFreq += readerContext.reader().totalTermFreq(indexedField, indexedBytes);
}
final long ttf = Math.max(-1, totalTermFreq); // we may have added up -1s if not supported
context.put(this, new LongDocValues(this) {

View File

@ -16,7 +16,7 @@ package org.apache.lucene.queries.function.valuesource;
* limitations under the License.
*/
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.IndexSearcher;

View File

@ -25,6 +25,7 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
@ -34,7 +35,6 @@ import org.apache.lucene.search.similarities.TFIDFSimilarity;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.PriorityQueue;
import org.apache.lucene.util.ReaderUtil;
import org.apache.lucene.util.UnicodeUtil;
@ -570,7 +570,7 @@ public final class MoreLikeThis {
public Query like(int docNum) throws IOException {
if (fieldNames == null) {
// gather list of valid fields from lucene
Collection<String> fields = ReaderUtil.getIndexedFields(ir);
Collection<String> fields = MultiFields.getIndexedFields(ir);
fieldNames = fields.toArray(new String[fields.size()]);
}

View File

@ -21,17 +21,16 @@ import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.SlowMultiReaderWrapper;
import org.apache.lucene.index.SlowCompositeReaderWrapper;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.TermRangeFilter;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.QueryWrapperFilter;
import org.apache.lucene.store.Directory;
@ -43,7 +42,7 @@ import java.io.IOException;
public class BooleanFilterTest extends LuceneTestCase {
private Directory directory;
private IndexReader reader;
private AtomicIndexReader reader;
@Override
public void setUp() throws Exception {
@ -57,7 +56,7 @@ public class BooleanFilterTest extends LuceneTestCase {
addDoc(writer, "guest", "020", "20050101", "Y");
addDoc(writer, "admin", "020", "20050101", "Maybe");
addDoc(writer, "admin guest", "030", "20050101", "N");
reader = new SlowMultiReaderWrapper(writer.getReader());
reader = new SlowCompositeReaderWrapper(writer.getReader());
writer.close();
}
@ -97,7 +96,7 @@ public class BooleanFilterTest extends LuceneTestCase {
return new Filter() {
@Override
public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) {
return new FixedBitSet(context.reader.maxDoc());
return new FixedBitSet(context.reader().maxDoc());
}
};
}

View File

@ -22,10 +22,10 @@ import java.util.HashSet;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.index.MultiReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.SlowMultiReaderWrapper;
import org.apache.lucene.index.SlowCompositeReaderWrapper;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Filter;
import org.apache.lucene.store.Directory;
@ -62,27 +62,26 @@ public class TermsFilterTest extends LuceneTestCase {
doc.add(newField(fieldName, "" + term, StringField.TYPE_STORED));
w.addDocument(doc);
}
IndexReader reader = new SlowMultiReaderWrapper(w.getReader());
assertTrue(reader.getTopReaderContext().isAtomic);
IndexReader reader = new SlowCompositeReaderWrapper(w.getReader());
assertTrue(reader.getTopReaderContext() instanceof AtomicReaderContext);
AtomicReaderContext context = (AtomicReaderContext) reader.getTopReaderContext();
assertTrue(context.isAtomic);
w.close();
TermsFilter tf = new TermsFilter();
tf.addTerm(new Term(fieldName, "19"));
FixedBitSet bits = (FixedBitSet) tf.getDocIdSet(context, context.reader.getLiveDocs());
FixedBitSet bits = (FixedBitSet) tf.getDocIdSet(context, context.reader().getLiveDocs());
assertEquals("Must match nothing", 0, bits.cardinality());
tf.addTerm(new Term(fieldName, "20"));
bits = (FixedBitSet) tf.getDocIdSet(context, context.reader.getLiveDocs());
bits = (FixedBitSet) tf.getDocIdSet(context, context.reader().getLiveDocs());
assertEquals("Must match 1", 1, bits.cardinality());
tf.addTerm(new Term(fieldName, "10"));
bits = (FixedBitSet) tf.getDocIdSet(context, context.reader.getLiveDocs());
bits = (FixedBitSet) tf.getDocIdSet(context, context.reader().getLiveDocs());
assertEquals("Must match 2", 2, bits.cardinality());
tf.addTerm(new Term(fieldName, "00"));
bits = (FixedBitSet) tf.getDocIdSet(context, context.reader.getLiveDocs());
bits = (FixedBitSet) tf.getDocIdSet(context, context.reader().getLiveDocs());
assertEquals("Must match 2", 2, bits.cardinality());
reader.close();
@ -112,8 +111,8 @@ public class TermsFilterTest extends LuceneTestCase {
tf.addTerm(new Term(fieldName, "content1"));
MultiReader multi = new MultiReader(reader1, reader2);
for (IndexReader.AtomicReaderContext context : ReaderUtil.leaves(multi.getTopReaderContext())) {
FixedBitSet bits = (FixedBitSet) tf.getDocIdSet(context, context.reader.getLiveDocs());
for (AtomicReaderContext context : ReaderUtil.leaves(multi.getTopReaderContext())) {
FixedBitSet bits = (FixedBitSet) tf.getDocIdSet(context, context.reader().getLiveDocs());
assertTrue("Must be >= 0", bits.cardinality() >= 0);
}
multi.close();

View File

@ -28,7 +28,7 @@ import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.AtomicIndexReader.AtomicReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Term;
@ -174,11 +174,11 @@ public class TestCustomScoreQuery extends FunctionTestSetup {
@Override
protected CustomScoreProvider getCustomScoreProvider(AtomicReaderContext context) throws IOException {
final int[] values = FieldCache.DEFAULT.getInts(context.reader, INT_FIELD, false);
final int[] values = FieldCache.DEFAULT.getInts(context.reader(), INT_FIELD, false);
return new CustomScoreProvider(context) {
@Override
public float customScore(int doc, float subScore, float valSrcScore) throws IOException {
assertTrue(doc <= context.reader.maxDoc());
assertTrue(doc <= context.reader().maxDoc());
return values[doc];
}
};

View File

@ -19,11 +19,9 @@ package org.apache.lucene.queries.function;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.queries.function.valuesource.OrdFieldSource;
import org.apache.lucene.queries.function.valuesource.ReverseOrdFieldSource;
import org.apache.lucene.search.*;
import org.apache.lucene.util.ReaderUtil;
import org.junit.BeforeClass;
import org.junit.Test;