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 {