You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rj...@apache.org on 2014/09/24 03:20:45 UTC
svn commit: r1627188 [3/13] - in /lucene/dev/branches/branch_5x: ./
dev-tools/ lucene/ lucene/analysis/
lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/
lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/std40/...
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java Wed Sep 24 01:20:36 2014
@@ -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();
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MultiFields.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MultiFields.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MultiFields.java Wed Sep 24 01:20:36 2014
@@ -60,7 +60,7 @@ public final class MultiFields extends F
* 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 F
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 F
* 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 F
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 F
*/
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();
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java Wed Sep 24 01:20:36 2014
@@ -96,7 +96,7 @@ public class ParallelCompositeReader ext
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 ext
// 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 ext
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)");
}
}
Copied: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java (from r1627177, lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java?p2=lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java&p1=lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java&r1=1627177&r2=1627188&rev=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java Wed Sep 24 01:20:36 2014
@@ -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 extend
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 extend
// 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 extend
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 extend
}
// 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 extend
@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 extend
@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 extend
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 extend
@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 extend
@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 extend
@Override
public void checkIntegrity() throws IOException {
ensureOpen();
- for (AtomicReader reader : completeReaderSet) {
+ for (LeafReader reader : completeReaderSet) {
reader.checkIntegrity();
}
}
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/ReaderUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/ReaderUtil.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/ReaderUtil.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/ReaderUtil.java Wed Sep 24 01:20:36 2014
@@ -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
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java Wed Sep 24 01:20:36 2014
@@ -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;
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java Wed Sep 24 01:20:36 2014
@@ -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);
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java Wed Sep 24 01:20:36 2014
@@ -50,7 +50,7 @@ import org.apache.lucene.util.RamUsageEs
* 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)
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SimpleMergedSegmentWarmer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SimpleMergedSegmentWarmer.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SimpleMergedSegmentWarmer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SimpleMergedSegmentWarmer.java Wed Sep 24 01:20:36 2014
@@ -38,7 +38,7 @@ public class SimpleMergedSegmentWarmer e
}
@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;
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java Wed Sep 24 01:20:36 2014
@@ -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.MultiRead
* 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 SlowCompositeReaderWr
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 SlowCompositeReaderWr
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 SlowCompositeReaderWr
@Override
public void checkIntegrity() throws IOException {
ensureOpen();
- for (AtomicReaderContext ctx : in.leaves()) {
+ for (LeafReaderContext ctx : in.leaves()) {
ctx.reader().checkIntegrity();
}
}
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java Wed Sep 24 01:20:36 2014
@@ -37,7 +37,7 @@ final class StandardDirectoryReader exte
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 exte
}
/** 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 exte
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 exte
@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();
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/TermContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/TermContext.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/TermContext.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/TermContext.java Wed Sep 24 01:20:36 2014
@@ -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) {
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/package.html?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/package.html (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/package.html Wed Sep 24 01:20:36 2014
@@ -254,7 +254,7 @@ its {@link org.apache.lucene.search.simi
</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>
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java Wed Sep 24 01:20:36 2014
@@ -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
}
@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
}
@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
}
@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(),
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java Wed Sep 24 01:20:36 2014
@@ -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
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java Wed Sep 24 01:20:36 2014
@@ -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 e
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 e
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 e
}
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) {
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java Wed Sep 24 01:20:36 2014
@@ -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 extend
* 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 extend
/**
* 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 extend
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);
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Collector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Collector.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Collector.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Collector.java Wed Sep 24 01:20:36 2014
@@ -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;
}
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java Wed Sep 24 01:20:36 2014
@@ -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
}
@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
}
@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
}
@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);
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java Wed Sep 24 01:20:36 2014
@@ -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
/** 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
/** 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;
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRangeFilter.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRangeFilter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRangeFilter.java Wed Sep 24 01:20:36 2014
@@ -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 DocTermOrdsRangeFi
/** 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);
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRewriteMethod.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRewriteMethod.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRewriteMethod.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRewriteMethod.java Wed Sep 24 01:20:36 2014
@@ -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 DocTermOrdsRewriteMet
* 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());
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeFilter.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeFilter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeFilter.java Wed Sep 24 01:20:36 2014
@@ -22,8 +22,7 @@ import org.apache.lucene.document.Double
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.NumericUti
/**
* 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.NumericUti
* 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 DocValuesRangeFilt
/** 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 DocValuesRangeFilt
}
/**
- * 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 DocValuesRangeFilt
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 DocValuesRangeFilt
}
/**
- * 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 DocValuesRangeFilt
}
/**
- * 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 DocValuesRangeFilt
}
/**
- * 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 DocValuesRangeFilt
}
/**
- * 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;
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java Wed Sep 24 01:20:36 2014
@@ -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 DocValuesRewriteMetho
* 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());
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocValuesTermsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocValuesTermsFilter.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocValuesTermsFilter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocValuesTermsFilter.java Wed Sep 24 01:20:36 2014
@@ -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 extend
}
@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++) {
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java Wed Sep 24 01:20:36 2014
@@ -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.BytesRefBu
* 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) {
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldValueFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldValueFilter.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldValueFilter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FieldValueFilter.java Wed Sep 24 01:20:36 2014
@@ -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.Match
/**
* 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 Fi
}
@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);
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Filter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Filter.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Filter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Filter.java Wed Sep 24 01:20:36 2014
@@ -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;
}
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilterCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilterCollector.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilterCollector.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilterCollector.java Wed Sep 24 01:20:36 2014
@@ -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
}
@Override
- public LeafCollector getLeafCollector(AtomicReaderContext context) throws IOException {
+ public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
return in.getLeafCollector(context);
}
Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1627188&r1=1627187&r2=1627188&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java Wed Sep 24 01:20:36 2014
@@ -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 {