You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2012/01/31 00:34:14 UTC

svn commit: r1238085 [3/10] - in /lucene/dev/trunk: ./ dev-tools/idea/lucene/contrib/ lucene/ lucene/contrib/ lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/ lucene/contrib/highlighter/src/test/org/apache/lucene/search/highlight...

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ParallelReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ParallelReader.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ParallelReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ParallelReader.java Mon Jan 30 23:34:03 2012
@@ -22,10 +22,9 @@ import java.util.*;
 
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.ReaderUtil;
 
 
-/** An IndexReader which reads multiple, parallel indexes.  Each index added
+/** An AtomicIndexReader which reads multiple, parallel indexes.  Each index added
  * must have the same number of documents, but typically each contains
  * different fields.  Each document contains the union of the fields of all
  * documents with the same document number.  When searching, matches for a
@@ -42,15 +41,14 @@ import org.apache.lucene.util.ReaderUtil
  * same order to the other indexes. <em>Failure to do so will result in
  * undefined behavior</em>.
  */
-public class ParallelReader extends IndexReader {
-  private List<IndexReader> readers = new ArrayList<IndexReader>();
+public class ParallelReader extends AtomicReader {
+  private List<AtomicReader> readers = new ArrayList<AtomicReader>();
   private List<Boolean> decrefOnClose = new ArrayList<Boolean>(); // remember which subreaders to decRef on close
   boolean incRefReaders = false;
-  private SortedMap<String,IndexReader> fieldToReader = new TreeMap<String,IndexReader>();
-  private Map<IndexReader,Collection<String>> readerToFields = new HashMap<IndexReader,Collection<String>>();
-  private List<IndexReader> storedFieldReaders = new ArrayList<IndexReader>();
+  private SortedMap<String,AtomicReader> fieldToReader = new TreeMap<String,AtomicReader>();
+  private Map<AtomicReader,Collection<String>> readerToFields = new HashMap<AtomicReader,Collection<String>>();
+  private List<AtomicReader> storedFieldReaders = new ArrayList<AtomicReader>();
   private Map<String, DocValues> normsCache = new HashMap<String,DocValues>();
-  private final ReaderContext topLevelReaderContext = new AtomicReaderContext(this);
   private int maxDoc;
   private int numDocs;
   private boolean hasDeletions;
@@ -77,7 +75,7 @@ public class ParallelReader extends Inde
   @Override
   public String toString() {
     final StringBuilder buffer = new StringBuilder("ParallelReader(");
-    final Iterator<IndexReader> iter = readers.iterator();
+    final Iterator<AtomicReader> iter = readers.iterator();
     if (iter.hasNext()) {
       buffer.append(iter.next());
     }
@@ -88,25 +86,25 @@ public class ParallelReader extends Inde
     return buffer.toString();
   }
   
- /** Add an IndexReader.
+ /** Add an AtomicIndexReader.
   * @throws IOException if there is a low-level IO error
   */
-  public void add(IndexReader reader) throws IOException {
+  public void add(AtomicReader reader) throws IOException {
     ensureOpen();
     add(reader, false);
   }
 
- /** Add an IndexReader whose stored fields will not be returned.  This can
+ /** Add an AtomicIndexReader whose stored fields will not be returned.  This can
   * accelerate search when stored fields are only needed from a subset of
   * the IndexReaders.
   *
   * @throws IllegalArgumentException if not all indexes contain the same number
   *     of documents
   * @throws IllegalArgumentException if not all indexes have the same value
-  *     of {@link IndexReader#maxDoc()}
+  *     of {@link AtomicReader#maxDoc()}
   * @throws IOException if there is a low-level IO error
   */
-  public void add(IndexReader reader, boolean ignoreStoredFields)
+  public void add(AtomicReader reader, boolean ignoreStoredFields)
     throws IOException {
 
     ensureOpen();
@@ -123,13 +121,13 @@ public class ParallelReader extends Inde
       throw new IllegalArgumentException
         ("All readers must have same numDocs: "+numDocs+"!="+reader.numDocs());
 
-    final FieldInfos readerFieldInfos = ReaderUtil.getMergedFieldInfos(reader);
+    final FieldInfos readerFieldInfos = MultiFields.getMergedFieldInfos(reader);
     for(FieldInfo fieldInfo : readerFieldInfos) {   // update fieldToReader map
       // NOTE: first reader having a given field "wins":
       if (fieldToReader.get(fieldInfo.name) == null) {
         fieldInfos.add(fieldInfo);
         fieldToReader.put(fieldInfo.name, reader);
-        this.fields.addField(fieldInfo.name, MultiFields.getFields(reader).terms(fieldInfo.name));
+        this.fields.addField(fieldInfo.name, reader.terms(fieldInfo.name));
       }
     }
 
@@ -205,7 +203,7 @@ public class ParallelReader extends Inde
   @Override
   public Bits getLiveDocs() {
     ensureOpen();
-    return MultiFields.getLiveDocs(readers.get(0));
+    return readers.get(0).getLiveDocs();
   }
 
   @Override
@@ -214,88 +212,6 @@ public class ParallelReader extends Inde
     return fields;
   }
   
-  /**
-   * Tries to reopen the subreaders.
-   * <br>
-   * If one or more subreaders could be re-opened (i. e. subReader.reopen() 
-   * returned a new instance != subReader), then a new ParallelReader instance 
-   * is returned, otherwise null is returned.
-   * <p>
-   * A re-opened instance might share one or more subreaders with the old 
-   * instance. Index modification operations result in undefined behavior
-   * when performed before the old instance is closed.
-   * (see {@link IndexReader#openIfChanged}).
-   * <p>
-   * If subreaders are shared, then the reference count of those
-   * readers is increased to ensure that the subreaders remain open
-   * until the last referring reader is closed.
-   * 
-   * @throws CorruptIndexException if the index is corrupt
-   * @throws IOException if there is a low-level IO error 
-   */
-  @Override
-  protected synchronized IndexReader doOpenIfChanged() throws CorruptIndexException, IOException {
-    ensureOpen();
-    
-    boolean reopened = false;
-    List<IndexReader> newReaders = new ArrayList<IndexReader>();
-    
-    boolean success = false;
-    
-    try {
-      for (final IndexReader oldReader : readers) {
-        IndexReader newReader = null;
-        newReader = IndexReader.openIfChanged(oldReader);
-        if (newReader != null) {
-          reopened = true;
-        } else {
-          newReader = oldReader;
-        }
-        newReaders.add(newReader);
-      }
-      success = true;
-    } finally {
-      if (!success && reopened) {
-        for (int i = 0; i < newReaders.size(); i++) {
-          IndexReader r = newReaders.get(i);
-          if (r != readers.get(i)) {
-            try {
-              r.close();
-            } catch (IOException ignore) {
-              // keep going - we want to clean up as much as possible
-            }
-          }
-        }
-      }
-    }
-
-    if (reopened) {
-      List<Boolean> newDecrefOnClose = new ArrayList<Boolean>();
-      // TODO: maybe add a special reopen-ctor for norm-copying?
-      ParallelReader pr = new ParallelReader();
-      for (int i = 0; i < readers.size(); i++) {
-        IndexReader oldReader = readers.get(i);
-        IndexReader newReader = newReaders.get(i);
-        if (newReader == oldReader) {
-          newDecrefOnClose.add(Boolean.TRUE);
-          newReader.incRef();
-        } else {
-          // this is a new subreader instance, so on close() we don't
-          // decRef but close it 
-          newDecrefOnClose.add(Boolean.FALSE);
-        }
-        pr.add(newReader, !storedFieldReaders.contains(oldReader));
-      }
-      pr.decrefOnClose = newDecrefOnClose;
-      pr.incRefReaders = incRefReaders;
-      return pr;
-    } else {
-      // No subreader was refreshed
-      return null;
-    }
-  }
-
-
   @Override
   public int numDocs() {
     // Don't call ensureOpen() here (it could affect performance)
@@ -317,7 +233,7 @@ public class ParallelReader extends Inde
   @Override
   public void document(int docID, StoredFieldVisitor visitor) throws CorruptIndexException, IOException {
     ensureOpen();
-    for (final IndexReader reader: storedFieldReaders) {
+    for (final AtomicReader reader: storedFieldReaders) {
       reader.document(docID, visitor);
     }
   }
@@ -327,7 +243,7 @@ public class ParallelReader extends Inde
   public Fields getTermVectors(int docID) throws IOException {
     ensureOpen();
     ParallelFields fields = new ParallelFields();
-    for (Map.Entry<String,IndexReader> ent : fieldToReader.entrySet()) {
+    for (Map.Entry<String,AtomicReader> ent : fieldToReader.entrySet()) {
       String fieldName = ent.getKey();
       Terms vector = ent.getValue().getTermVector(docID, fieldName);
       if (vector != null) {
@@ -341,44 +257,13 @@ public class ParallelReader extends Inde
   @Override
   public boolean hasNorms(String field) throws IOException {
     ensureOpen();
-    IndexReader reader = fieldToReader.get(field);
+    AtomicReader reader = fieldToReader.get(field);
     return reader==null ? false : reader.hasNorms(field);
   }
 
-  @Override
-  public int docFreq(String field, BytesRef term) throws IOException {
-    ensureOpen();
-    IndexReader reader = fieldToReader.get(field);
-    return reader == null? 0 : reader.docFreq(field, term);
-  }
-
-  /**
-   * Checks recursively if all subreaders are up to date. 
-   */
-  @Override
-  public boolean isCurrent() throws CorruptIndexException, IOException {
-    ensureOpen();
-    for (final IndexReader reader : readers) {
-      if (!reader.isCurrent()) {
-        return false;
-      }
-    }
-    
-    // all subreaders are up to date
-    return true;
-  }
-
-  /** Not implemented.
-   * @throws UnsupportedOperationException
-   */
-  @Override
-  public long getVersion() {
-    throw new UnsupportedOperationException("ParallelReader does not support this method.");
-  }
-
   // for testing
-  IndexReader[] getSubReaders() {
-    return readers.toArray(new IndexReader[readers.size()]);
+  AtomicReader[] getSubReaders() {
+    return readers.toArray(new AtomicReader[readers.size()]);
   }
 
   @Override
@@ -392,17 +277,11 @@ public class ParallelReader extends Inde
     }
   }
 
-  @Override
-  public ReaderContext getTopReaderContext() {
-    ensureOpen();
-    return topLevelReaderContext;
-  }
-
   // TODO: I suspect this is completely untested!!!!!
   @Override
   public DocValues docValues(String field) throws IOException {
-    IndexReader reader = fieldToReader.get(field);
-    return reader == null ? null : MultiDocValues.getDocValues(reader, field);
+    AtomicReader reader = fieldToReader.get(field);
+    return reader == null ? null : reader.docValues(field);
   }
   
   // TODO: I suspect this is completely untested!!!!!
@@ -410,8 +289,8 @@ public class ParallelReader extends Inde
   public synchronized DocValues normValues(String field) throws IOException {
     DocValues values = normsCache.get(field);
     if (values == null) {
-      IndexReader reader = fieldToReader.get(field);
-      values = reader == null ? null : MultiDocValues.getNormDocValues(reader, field);
+      AtomicReader reader = fieldToReader.get(field);
+      values = reader == null ? null : reader.normValues(field);
       normsCache.put(field, values);
     } 
     return values;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java Mon Jan 30 23:34:03 2012
@@ -62,7 +62,7 @@ public class PersistentSnapshotDeletionP
    * keeps a lock on the snapshots directory).
    */
   public static Map<String, String> readSnapshotsInfo(Directory dir) throws IOException {
-    IndexReader r = IndexReader.open(dir);
+    IndexReader r = DirectoryReader.open(dir);
     Map<String, String> snapshots = new HashMap<String, String>();
     try {
       int numDocs = r.numDocs();

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Mon Jan 30 23:34:03 2012
@@ -76,7 +76,7 @@ final class SegmentMerger {
     try {
       new ReaderUtil.Gather(reader) {
         @Override
-        protected void add(int base, IndexReader r) {
+        protected void add(int base, AtomicReader r) {
           mergeState.readers.add(new MergeState.IndexReaderAndLiveDocs(r, r.getLiveDocs()));
         }
       }.run();
@@ -201,7 +201,7 @@ final class SegmentMerger {
     Map<FieldInfo,TypePromoter> normValuesTypes = new HashMap<FieldInfo,TypePromoter>();
 
     for (MergeState.IndexReaderAndLiveDocs readerAndLiveDocs : mergeState.readers) {
-      final IndexReader reader = readerAndLiveDocs.reader;
+      final AtomicReader reader = readerAndLiveDocs.reader;
       FieldInfos readerFieldInfos = reader.getFieldInfos();
       for (FieldInfo fi : readerFieldInfos) {
         FieldInfo merged = mergeState.fieldInfos.add(fi);
@@ -323,7 +323,12 @@ final class SegmentMerger {
       docBase += docCount;
 
       if (mergeState.payloadProcessorProvider != null) {
-        mergeState.dirPayloadProcessor[i] = mergeState.payloadProcessorProvider.getDirProcessor(reader.reader.directory());
+        // TODO: the PayloadProcessorProvider should take AtomicReader as parameter
+        // and find out by itself if it can provide a processor:
+        if (!(reader.reader instanceof SegmentReader))
+          throw new UnsupportedOperationException("Payload processing currently requires exclusively SegmentReaders to be merged.");
+        final Directory dir = ((SegmentReader) reader.reader).directory();
+        mergeState.dirPayloadProcessor[i] = mergeState.payloadProcessorProvider.getDirProcessor(dir);
       }
 
       i++;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java Mon Jan 30 23:34:03 2012
@@ -30,11 +30,9 @@ import org.apache.lucene.util.Bits;
 /**
  * @lucene.experimental
  */
-public final class SegmentReader extends IndexReader {
+public final class SegmentReader extends AtomicReader {
 
   private final SegmentInfo si;
-  private final ReaderContext readerContext = new AtomicReaderContext(this);
-  
   private final Bits liveDocs;
 
   // Normally set to si.docCount - si.delDocCount, unless we
@@ -186,12 +184,6 @@ public final class SegmentReader extends
     return si.toString(si.dir, si.docCount - numDocs - si.getDelCount());
   }
   
-  @Override
-  public ReaderContext getTopReaderContext() {
-    ensureOpen();
-    return readerContext;
-  }
-
   /**
    * Return the name of the segment this reader is reading.
    */
@@ -207,7 +199,6 @@ public final class SegmentReader extends
   }
 
   /** Returns the directory this index resides in. */
-  @Override
   public Directory directory() {
     // Don't ensureOpen here -- in certain cases, when a
     // cloned/reopened reader needs to commit, it may call
@@ -228,7 +219,6 @@ public final class SegmentReader extends
     return this;
   }
   
-  @Override
   public int getTermInfosIndexDivisor() {
     return core.termsIndexDivisor;
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/BooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/BooleanQuery.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/BooleanQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/BooleanQuery.java Mon Jan 30 23:34:03 2012
@@ -20,8 +20,8 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.util.*;
 
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermsEnum;
@@ -240,7 +240,7 @@ public class BooleanQuery extends Query 
       for (Iterator<Weight> wIter = weights.iterator(); wIter.hasNext();) {
         Weight w = wIter.next();
         BooleanClause c = cIter.next();
-        if (w.scorer(context, true, true, context.reader.getLiveDocs()) == null) {
+        if (w.scorer(context, true, true, context.reader().getLiveDocs()) == null) {
           if (c.isRequired()) {
             fail = true;
             Explanation r = new Explanation(0.0f, "no match on required clause (" + c.getQuery().toString() + ")");

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/BooleanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/BooleanScorer.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/BooleanScorer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/BooleanScorer.java Mon Jan 30 23:34:03 2012
@@ -22,7 +22,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery.BooleanWeight;
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CachingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CachingCollector.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CachingCollector.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CachingCollector.java Mon Jan 30 23:34:03 2012
@@ -17,7 +17,7 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.util.RamUsageEstimator;
 
 import java.io.IOException;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java Mon Jan 30 23:34:03 2012
@@ -22,8 +22,9 @@ import java.util.Collections;
 import java.util.Map;
 import java.util.WeakHashMap;
 
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DirectoryReader; // javadocs
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.Bits;
 
@@ -53,13 +54,13 @@ public class CachingWrapperFilter extend
 
   /** Wraps another filter's result and caches it. If
    * {@code recacheDeletes} is {@code true}, then new deletes (for example
-   * after {@link IndexReader#openIfChanged}) will cause the filter
+   * after {@link DirectoryReader#openIfChanged}) will cause the filter
    * {@link DocIdSet} to be recached.
    *
    * <p>If your index changes seldom, it is recommended to use {@code recacheDeletes=true},
    * as recaching will only occur when the index is reopened.
    * For near-real-time indexes or indexes that are often
-   * reopened with (e.g., {@link IndexReader#openIfChanged} is used), you should
+   * reopened with (e.g., {@link DirectoryReader#openIfChanged} is used), you should
    * pass {@code recacheDeletes=false}. This will cache the filter results omitting
    * deletions and will AND them in while scoring.
    * @param filter Filter to cache results of
@@ -76,7 +77,7 @@ public class CachingWrapperFilter extend
    *  returns <code>true</code>, else it copies the {@link DocIdSetIterator} into
    *  a {@link FixedBitSet}.
    */
-  protected DocIdSet docIdSetToCache(DocIdSet docIdSet, IndexReader reader) throws IOException {
+  protected DocIdSet docIdSetToCache(DocIdSet docIdSet, AtomicReader reader) throws IOException {
     if (docIdSet == null) {
       // this is better than returning null, as the nonnull result can be cached
       return DocIdSet.EMPTY_DOCIDSET;
@@ -102,7 +103,7 @@ public class CachingWrapperFilter extend
 
   @Override
   public DocIdSet getDocIdSet(AtomicReaderContext context, final Bits acceptDocs) throws IOException {
-    final IndexReader reader = context.reader;
+    final AtomicReader reader = context.reader();
 
     // Only cache if incoming acceptDocs is == live docs;
     // if Lucene passes in more interesting acceptDocs in

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/Collector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/Collector.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/Collector.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/Collector.java Mon Jan 30 23:34:03 2012
@@ -19,8 +19,8 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
-import org.apache.lucene.index.IndexReader.ReaderContext;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.IndexReaderContext;
 
 /**
  * <p>Expert: Collectors are primarily meant to be used to
@@ -145,9 +145,9 @@ public abstract class Collector {
 
   /**
    * Called before collecting from each {@link AtomicReaderContext}. All doc ids in
-   * {@link #collect(int)} will correspond to {@link ReaderContext#reader}.
+   * {@link #collect(int)} will correspond to {@link IndexReaderContext#reader}.
    * 
-   * Add {@link AtomicReaderContext#docBase} to the current  {@link ReaderContext#reader}'s
+   * Add {@link AtomicReaderContext#docBase} to the current  {@link IndexReaderContext#reader}'s
    * internal document id to re-base ids in {@link #collect(int)}.
    * 
    * @param context

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java Mon Jan 30 23:34:03 2012
@@ -17,8 +17,8 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
@@ -149,7 +149,7 @@ public class ConstantScoreQuery extends 
 
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      final Scorer cs = scorer(context, true, false, context.reader.getLiveDocs());
+      final Scorer cs = scorer(context, true, false, context.reader().getLiveDocs());
       final boolean exists = (cs != null && cs.advance(doc) == doc);
 
       final ComplexExplanation result = new ComplexExplanation();

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java Mon Jan 30 23:34:03 2012
@@ -22,8 +22,8 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.Set;
 
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.util.Bits;
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCache.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCache.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCache.java Mon Jan 30 23:34:03 2012
@@ -24,7 +24,7 @@ import java.text.DecimalFormat;
 import org.apache.lucene.analysis.NumericTokenStream; // for javadocs
 import org.apache.lucene.document.NumericField; // for javadocs
 import org.apache.lucene.index.DocTermOrds;
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -63,7 +63,7 @@ public interface FieldCache {
   }
 
   /** Interface to parse bytes from document fields.
-   * @see FieldCache#getBytes(IndexReader, String, FieldCache.ByteParser, boolean)
+   * @see FieldCache#getBytes(AtomicReader, String, FieldCache.ByteParser, boolean)
    */
   public interface ByteParser extends Parser {
     /** Return a single Byte representation of this field's value. */
@@ -71,7 +71,7 @@ public interface FieldCache {
   }
 
   /** Interface to parse shorts from document fields.
-   * @see FieldCache#getShorts(IndexReader, String, FieldCache.ShortParser, boolean)
+   * @see FieldCache#getShorts(AtomicReader, String, FieldCache.ShortParser, boolean)
    */
   public interface ShortParser extends Parser {
     /** Return a short representation of this field's value. */
@@ -79,7 +79,7 @@ public interface FieldCache {
   }
 
   /** Interface to parse ints from document fields.
-   * @see FieldCache#getInts(IndexReader, String, FieldCache.IntParser, boolean)
+   * @see FieldCache#getInts(AtomicReader, String, FieldCache.IntParser, boolean)
    */
   public interface IntParser extends Parser {
     /** Return an integer representation of this field's value. */
@@ -87,7 +87,7 @@ public interface FieldCache {
   }
 
   /** Interface to parse floats from document fields.
-   * @see FieldCache#getFloats(IndexReader, String, FieldCache.FloatParser, boolean)
+   * @see FieldCache#getFloats(AtomicReader, String, FieldCache.FloatParser, boolean)
    */
   public interface FloatParser extends Parser {
     /** Return an float representation of this field's value. */
@@ -95,7 +95,7 @@ public interface FieldCache {
   }
 
   /** Interface to parse long from document fields.
-   * @see FieldCache#getLongs(IndexReader, String, FieldCache.LongParser, boolean)
+   * @see FieldCache#getLongs(AtomicReader, String, FieldCache.LongParser, boolean)
    */
   public interface LongParser extends Parser {
     /** Return an long representation of this field's value. */
@@ -103,7 +103,7 @@ public interface FieldCache {
   }
 
   /** Interface to parse doubles from document fields.
-   * @see FieldCache#getDoubles(IndexReader, String, FieldCache.DoubleParser, boolean)
+   * @see FieldCache#getDoubles(AtomicReader, String, FieldCache.DoubleParser, boolean)
    */
   public interface DoubleParser extends Parser {
     /** Return an long representation of this field's value. */
@@ -303,7 +303,7 @@ public interface FieldCache {
    * <code>reader.maxDoc()</code>, with turned on bits for each docid that 
    * does have a value for this field.
    */
-  public Bits getDocsWithField(IndexReader reader, String field) 
+  public Bits getDocsWithField(AtomicReader reader, String field) 
   throws IOException;
 
   /** Checks the internal cache for an appropriate entry, and if none is
@@ -317,7 +317,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public byte[] getBytes (IndexReader reader, String field, boolean setDocsWithField)
+  public byte[] getBytes (AtomicReader reader, String field, boolean setDocsWithField)
   throws IOException;
 
   /** Checks the internal cache for an appropriate entry, and if none is found,
@@ -332,7 +332,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public byte[] getBytes (IndexReader reader, String field, ByteParser parser, boolean setDocsWithField)
+  public byte[] getBytes (AtomicReader reader, String field, ByteParser parser, boolean setDocsWithField)
   throws IOException;
 
   /** Checks the internal cache for an appropriate entry, and if none is
@@ -346,7 +346,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public short[] getShorts (IndexReader reader, String field, boolean setDocsWithField)
+  public short[] getShorts (AtomicReader reader, String field, boolean setDocsWithField)
   throws IOException;
 
   /** Checks the internal cache for an appropriate entry, and if none is found,
@@ -361,7 +361,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public short[] getShorts (IndexReader reader, String field, ShortParser parser, boolean setDocsWithField)
+  public short[] getShorts (AtomicReader reader, String field, ShortParser parser, boolean setDocsWithField)
   throws IOException;
   
   /** Checks the internal cache for an appropriate entry, and if none is
@@ -375,7 +375,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public int[] getInts (IndexReader reader, String field, boolean setDocsWithField)
+  public int[] getInts (AtomicReader reader, String field, boolean setDocsWithField)
   throws IOException;
 
   /** Checks the internal cache for an appropriate entry, and if none is found,
@@ -390,7 +390,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public int[] getInts (IndexReader reader, String field, IntParser parser, boolean setDocsWithField)
+  public int[] getInts (AtomicReader reader, String field, IntParser parser, boolean setDocsWithField)
   throws IOException;
 
   /** Checks the internal cache for an appropriate entry, and if
@@ -404,7 +404,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public float[] getFloats (IndexReader reader, String field, boolean setDocsWithField)
+  public float[] getFloats (AtomicReader reader, String field, boolean setDocsWithField)
   throws IOException;
 
   /** Checks the internal cache for an appropriate entry, and if
@@ -419,7 +419,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public float[] getFloats (IndexReader reader, String field,
+  public float[] getFloats (AtomicReader reader, String field,
                             FloatParser parser, boolean setDocsWithField) throws IOException;
 
   /**
@@ -435,7 +435,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws java.io.IOException If any error occurs.
    */
-  public long[] getLongs(IndexReader reader, String field, boolean setDocsWithField)
+  public long[] getLongs(AtomicReader reader, String field, boolean setDocsWithField)
           throws IOException;
 
   /**
@@ -452,7 +452,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException If any error occurs.
    */
-  public long[] getLongs(IndexReader reader, String field, LongParser parser, boolean setDocsWithField)
+  public long[] getLongs(AtomicReader reader, String field, LongParser parser, boolean setDocsWithField)
           throws IOException;
 
   /**
@@ -468,7 +468,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException If any error occurs.
    */
-  public double[] getDoubles(IndexReader reader, String field, boolean setDocsWithField)
+  public double[] getDoubles(AtomicReader reader, String field, boolean setDocsWithField)
           throws IOException;
 
   /**
@@ -485,7 +485,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException If any error occurs.
    */
-  public double[] getDoubles(IndexReader reader, String field, DoubleParser parser, boolean setDocsWithField)
+  public double[] getDoubles(AtomicReader reader, String field, DoubleParser parser, boolean setDocsWithField)
           throws IOException;
 
   /** Returned by {@link #getTerms} */
@@ -513,15 +513,15 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public DocTerms getTerms (IndexReader reader, String field)
+  public DocTerms getTerms (AtomicReader reader, String field)
   throws IOException;
 
-  /** Expert: just like {@link #getTerms(IndexReader,String)},
+  /** Expert: just like {@link #getTerms(AtomicReader,String)},
    *  but you can specify whether more RAM should be consumed in exchange for
    *  faster lookups (default is "true").  Note that the
    *  first call for a given reader and field "wins",
    *  subsequent calls will share the same cache entry. */
-  public DocTerms getTerms (IndexReader reader, String field, boolean fasterButMoreRAM)
+  public DocTerms getTerms (AtomicReader reader, String field, boolean fasterButMoreRAM)
   throws IOException;
 
   /** Returned by {@link #getTermsIndex} */
@@ -589,16 +589,16 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public DocTermsIndex getTermsIndex (IndexReader reader, String field)
+  public DocTermsIndex getTermsIndex (AtomicReader reader, String field)
   throws IOException;
 
   /** Expert: just like {@link
-   *  #getTermsIndex(IndexReader,String)}, but you can specify
+   *  #getTermsIndex(AtomicReader,String)}, but you can specify
    *  whether more RAM should be consumed in exchange for
    *  faster lookups (default is "true").  Note that the
    *  first call for a given reader and field "wins",
    *  subsequent calls will share the same cache entry. */
-  public DocTermsIndex getTermsIndex (IndexReader reader, String field, boolean fasterButMoreRAM)
+  public DocTermsIndex getTermsIndex (AtomicReader reader, String field, boolean fasterButMoreRAM)
   throws IOException;
 
   /**
@@ -611,7 +611,7 @@ public interface FieldCache {
    * @return a {@link DocTermOrds} instance
    * @throws IOException  If any error occurs.
    */
-  public DocTermOrds getDocTermOrds(IndexReader reader, String field) throws IOException;
+  public DocTermOrds getDocTermOrds(AtomicReader reader, String field) throws IOException;
 
   /**
    * EXPERT: A unique Identifier/Description for each item in the FieldCache. 
@@ -677,7 +677,7 @@ public interface FieldCache {
    * currently in the FieldCache.
    * <p>
    * NOTE: These CacheEntry objects maintain a strong reference to the 
-   * Cached Values.  Maintaining references to a CacheEntry the IndexReader 
+   * Cached Values.  Maintaining references to a CacheEntry the AtomicIndexReader 
    * associated with it has garbage collected will prevent the Value itself
    * from being garbage collected when the Cache drops the WeakReference.
    * </p>
@@ -705,7 +705,7 @@ public interface FieldCache {
    * top-level reader, it usually will have no effect as
    * Lucene now caches at the segment reader level.
    */
-  public abstract void purge(IndexReader r);
+  public abstract void purge(AtomicReader r);
 
   /**
    * If non-null, FieldCacheImpl will warn whenever

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheImpl.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheImpl.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheImpl.java Mon Jan 30 23:34:03 2012
@@ -29,6 +29,7 @@ import java.util.WeakHashMap;
 import org.apache.lucene.index.DocTermOrds;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.OrdTermState;
 import org.apache.lucene.index.SegmentReader;
@@ -48,8 +49,6 @@ import org.apache.lucene.util.packed.Pac
  * Expert: The default cache implementation, storing all values in memory.
  * A WeakHashMap is used for storage.
  *
- * <p>Created: May 19, 2004 4:40:36 PM
- *
  * @since   lucene 1.4
  */
 class FieldCacheImpl implements FieldCache {
@@ -76,7 +75,7 @@ class FieldCacheImpl implements FieldCac
     init();
   }
 
-  public synchronized void purge(IndexReader r) {
+  public synchronized void purge(AtomicReader r) {
     for(Cache c : caches.values()) {
       c.purge(r);
     }
@@ -158,21 +157,20 @@ class FieldCacheImpl implements FieldCac
   final IndexReader.ReaderClosedListener purgeReader = new IndexReader.ReaderClosedListener() {
     @Override
     public void onClose(IndexReader owner) {
-      FieldCacheImpl.this.purge(owner);
+      assert owner instanceof AtomicReader;
+      FieldCacheImpl.this.purge((AtomicReader) owner);
     }
   };
   
-  private void initReader(IndexReader reader) {
+  private void initReader(AtomicReader reader) {
     if (reader instanceof SegmentReader) {
       ((SegmentReader) reader).addCoreClosedListener(purgeCore);
-    } else if (reader.getSequentialSubReaders() != null) {
-      throw new UnsupportedOperationException("Please use SlowMultiReaderWrapper, if you really need a top level FieldCache");
     } else {
       // we have a slow reader of some sort, try to register a purge event
       // rather than relying on gc:
       Object key = reader.getCoreCacheKey();
-      if (key instanceof IndexReader) {
-        ((IndexReader)key).addReaderClosedListener(purgeReader); 
+      if (key instanceof AtomicReader) {
+        ((AtomicReader)key).addReaderClosedListener(purgeReader); 
       } else {
         // last chance
         reader.addReaderClosedListener(purgeReader); 				
@@ -191,11 +189,11 @@ class FieldCacheImpl implements FieldCac
 
     final Map<Object,Map<Entry,Object>> readerCache = new WeakHashMap<Object,Map<Entry,Object>>();
     
-    protected abstract Object createValue(IndexReader reader, Entry key, boolean setDocsWithField)
+    protected abstract Object createValue(AtomicReader reader, Entry key, boolean setDocsWithField)
         throws IOException;
 
     /** Remove this reader from the cache, if present. */
-    public void purge(IndexReader r) {
+    public void purge(AtomicReader r) {
       Object readerKey = r.getCoreCacheKey();
       synchronized(readerCache) {
         readerCache.remove(readerKey);
@@ -204,7 +202,7 @@ class FieldCacheImpl implements FieldCac
 
     /** Sets the key to the value for the provided reader;
      *  if the key is already set then this doesn't change it. */
-    public void put(IndexReader reader, Entry key, Object value) {
+    public void put(AtomicReader reader, Entry key, Object value) {
       final Object readerKey = reader.getCoreCacheKey();
       synchronized (readerCache) {
         Map<Entry,Object> innerCache = readerCache.get(readerKey);
@@ -223,7 +221,7 @@ class FieldCacheImpl implements FieldCac
       }
     }
 
-    public Object get(IndexReader reader, Entry key, boolean setDocsWithField) throws IOException {
+    public Object get(AtomicReader reader, Entry key, boolean setDocsWithField) throws IOException {
       Map<Entry,Object> innerCache;
       Object value;
       final Object readerKey = reader.getCoreCacheKey();
@@ -321,12 +319,12 @@ class FieldCacheImpl implements FieldCac
   }
 
   // inherit javadocs
-  public byte[] getBytes (IndexReader reader, String field, boolean setDocsWithField) throws IOException {
+  public byte[] getBytes (AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
     return getBytes(reader, field, null, setDocsWithField);
   }
 
   // inherit javadocs
-  public byte[] getBytes(IndexReader reader, String field, ByteParser parser, boolean setDocsWithField)
+  public byte[] getBytes(AtomicReader reader, String field, ByteParser parser, boolean setDocsWithField)
       throws IOException {
     return (byte[]) caches.get(Byte.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
   }
@@ -336,7 +334,7 @@ class FieldCacheImpl implements FieldCac
       super(wrapper);
     }
     @Override
-    protected Object createValue(IndexReader reader, Entry entryKey, boolean setDocsWithField)
+    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField)
         throws IOException {
       String field = entryKey.field;
       ByteParser parser = (ByteParser) entryKey.custom;
@@ -393,12 +391,12 @@ class FieldCacheImpl implements FieldCac
   }
   
   // inherit javadocs
-  public short[] getShorts (IndexReader reader, String field, boolean setDocsWithField) throws IOException {
+  public short[] getShorts (AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
     return getShorts(reader, field, null, setDocsWithField);
   }
 
   // inherit javadocs
-  public short[] getShorts(IndexReader reader, String field, ShortParser parser, boolean setDocsWithField)
+  public short[] getShorts(AtomicReader reader, String field, ShortParser parser, boolean setDocsWithField)
       throws IOException {
     return (short[]) caches.get(Short.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
   }
@@ -409,7 +407,7 @@ class FieldCacheImpl implements FieldCac
     }
 
     @Override
-    protected Object createValue(IndexReader reader, Entry entryKey, boolean setDocsWithField)
+    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField)
         throws IOException {
       String field = entryKey.field;
       ShortParser parser = (ShortParser) entryKey.custom;
@@ -466,7 +464,7 @@ class FieldCacheImpl implements FieldCac
   }
 
   // null Bits means no docs matched
-  void setDocsWithField(IndexReader reader, String field, Bits docsWithField) {
+  void setDocsWithField(AtomicReader reader, String field, Bits docsWithField) {
     final int maxDoc = reader.maxDoc();
     final Bits bits;
     if (docsWithField == null) {
@@ -487,12 +485,12 @@ class FieldCacheImpl implements FieldCac
   }
   
   // inherit javadocs
-  public int[] getInts (IndexReader reader, String field, boolean setDocsWithField) throws IOException {
+  public int[] getInts (AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
     return getInts(reader, field, null, setDocsWithField);
   }
 
   // inherit javadocs
-  public int[] getInts(IndexReader reader, String field, IntParser parser, boolean setDocsWithField)
+  public int[] getInts(AtomicReader reader, String field, IntParser parser, boolean setDocsWithField)
       throws IOException {
     return (int[]) caches.get(Integer.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
   }
@@ -503,7 +501,7 @@ class FieldCacheImpl implements FieldCac
     }
 
     @Override
-    protected Object createValue(IndexReader reader, Entry entryKey, boolean setDocsWithField)
+    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField)
         throws IOException {
       String field = entryKey.field;
       IntParser parser = (IntParser) entryKey.custom;
@@ -574,7 +572,7 @@ class FieldCacheImpl implements FieldCac
     }
   }
   
-  public Bits getDocsWithField(IndexReader reader, String field)
+  public Bits getDocsWithField(AtomicReader reader, String field)
       throws IOException {
     return (Bits) caches.get(DocsWithFieldCache.class).get(reader, new Entry(field, null), false);
   }
@@ -585,7 +583,7 @@ class FieldCacheImpl implements FieldCac
     }
     
     @Override
-      protected Object createValue(IndexReader reader, Entry entryKey, boolean setDocsWithField /* ignored */)
+      protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField /* ignored */)
     throws IOException {
       final String field = entryKey.field;      
       FixedBitSet res = null;
@@ -635,13 +633,13 @@ class FieldCacheImpl implements FieldCac
   }
 
   // inherit javadocs
-  public float[] getFloats (IndexReader reader, String field, boolean setDocsWithField)
+  public float[] getFloats (AtomicReader reader, String field, boolean setDocsWithField)
     throws IOException {
     return getFloats(reader, field, null, setDocsWithField);
   }
 
   // inherit javadocs
-  public float[] getFloats(IndexReader reader, String field, FloatParser parser, boolean setDocsWithField)
+  public float[] getFloats(AtomicReader reader, String field, FloatParser parser, boolean setDocsWithField)
     throws IOException {
 
     return (float[]) caches.get(Float.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
@@ -653,7 +651,7 @@ class FieldCacheImpl implements FieldCac
     }
 
     @Override
-    protected Object createValue(IndexReader reader, Entry entryKey, boolean setDocsWithField)
+    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField)
         throws IOException {
       String field = entryKey.field;
       FloatParser parser = (FloatParser) entryKey.custom;
@@ -725,12 +723,12 @@ class FieldCacheImpl implements FieldCac
   }
 
 
-  public long[] getLongs(IndexReader reader, String field, boolean setDocsWithField) throws IOException {
+  public long[] getLongs(AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
     return getLongs(reader, field, null, setDocsWithField);
   }
   
   // inherit javadocs
-  public long[] getLongs(IndexReader reader, String field, FieldCache.LongParser parser, boolean setDocsWithField)
+  public long[] getLongs(AtomicReader reader, String field, FieldCache.LongParser parser, boolean setDocsWithField)
       throws IOException {
     return (long[]) caches.get(Long.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
   }
@@ -741,7 +739,7 @@ class FieldCacheImpl implements FieldCac
     }
 
     @Override
-    protected Object createValue(IndexReader reader, Entry entryKey, boolean setDocsWithField)
+    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField)
         throws IOException {
       String field = entryKey.field;
       FieldCache.LongParser parser = (FieldCache.LongParser) entryKey.custom;
@@ -813,13 +811,13 @@ class FieldCacheImpl implements FieldCac
   }
 
   // inherit javadocs
-  public double[] getDoubles(IndexReader reader, String field, boolean setDocsWithField)
+  public double[] getDoubles(AtomicReader reader, String field, boolean setDocsWithField)
     throws IOException {
     return getDoubles(reader, field, null, setDocsWithField);
   }
 
   // inherit javadocs
-  public double[] getDoubles(IndexReader reader, String field, FieldCache.DoubleParser parser, boolean setDocsWithField)
+  public double[] getDoubles(AtomicReader reader, String field, FieldCache.DoubleParser parser, boolean setDocsWithField)
       throws IOException {
     return (double[]) caches.get(Double.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
   }
@@ -830,7 +828,7 @@ class FieldCacheImpl implements FieldCac
     }
 
     @Override
-    protected Object createValue(IndexReader reader, Entry entryKey, boolean setDocsWithField)
+    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField)
         throws IOException {
       String field = entryKey.field;
       FieldCache.DoubleParser parser = (FieldCache.DoubleParser) entryKey.custom;
@@ -1075,11 +1073,11 @@ class FieldCacheImpl implements FieldCac
 
   private static boolean DEFAULT_FASTER_BUT_MORE_RAM = true;
 
-  public DocTermsIndex getTermsIndex(IndexReader reader, String field) throws IOException {
+  public DocTermsIndex getTermsIndex(AtomicReader reader, String field) throws IOException {
     return getTermsIndex(reader, field, DEFAULT_FASTER_BUT_MORE_RAM);
   }
 
-  public DocTermsIndex getTermsIndex(IndexReader reader, String field, boolean fasterButMoreRAM) throws IOException {
+  public DocTermsIndex getTermsIndex(AtomicReader reader, String field, boolean fasterButMoreRAM) throws IOException {
     return (DocTermsIndex) caches.get(DocTermsIndex.class).get(reader, new Entry(field, Boolean.valueOf(fasterButMoreRAM)), false);
   }
 
@@ -1089,7 +1087,7 @@ class FieldCacheImpl implements FieldCac
     }
 
     @Override
-    protected Object createValue(IndexReader reader, Entry entryKey, boolean setDocsWithField /* ignored */)
+    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField /* ignored */)
         throws IOException {
 
       Terms terms = reader.terms(entryKey.field);
@@ -1220,11 +1218,11 @@ class FieldCacheImpl implements FieldCac
 
   // TODO: this if DocTermsIndex was already created, we
   // should share it...
-  public DocTerms getTerms(IndexReader reader, String field) throws IOException {
+  public DocTerms getTerms(AtomicReader reader, String field) throws IOException {
     return getTerms(reader, field, DEFAULT_FASTER_BUT_MORE_RAM);
   }
 
-  public DocTerms getTerms(IndexReader reader, String field, boolean fasterButMoreRAM) throws IOException {
+  public DocTerms getTerms(AtomicReader reader, String field, boolean fasterButMoreRAM) throws IOException {
     return (DocTerms) caches.get(DocTerms.class).get(reader, new Entry(field, Boolean.valueOf(fasterButMoreRAM)), false);
   }
 
@@ -1234,7 +1232,7 @@ class FieldCacheImpl implements FieldCac
     }
 
     @Override
-    protected Object createValue(IndexReader reader, Entry entryKey, boolean setDocsWithField /* ignored */)
+    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField /* ignored */)
         throws IOException {
 
       Terms terms = reader.terms(entryKey.field);
@@ -1308,7 +1306,7 @@ class FieldCacheImpl implements FieldCac
     }
   }
 
-  public DocTermOrds getDocTermOrds(IndexReader reader, String field) throws IOException {
+  public DocTermOrds getDocTermOrds(AtomicReader reader, String field) throws IOException {
     return (DocTermOrds) caches.get(DocTermOrds.class).get(reader, new Entry(field, null), false);
   }
 
@@ -1318,7 +1316,7 @@ class FieldCacheImpl implements FieldCac
     }
 
     @Override
-    protected Object createValue(IndexReader reader, Entry entryKey, boolean setDocsWithField /* ignored */)
+    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField /* ignored */)
         throws IOException {
       return new DocTermOrds(reader, entryKey.field);
     }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java Mon Jan 30 23:34:03 2012
@@ -18,8 +18,8 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.AtomicReader; // for javadocs
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -84,7 +84,7 @@ public abstract class FieldCacheRangeFil
     return new FieldCacheRangeFilter<String>(field, null, lowerVal, upperVal, includeLower, includeUpper) {
       @Override
       public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
-        final FieldCache.DocTermsIndex fcsi = FieldCache.DEFAULT.getTermsIndex(context.reader, field);
+        final FieldCache.DocTermsIndex fcsi = FieldCache.DEFAULT.getTermsIndex(context.reader(), field);
         final BytesRef spare = new BytesRef();
         final int lowerPoint = fcsi.binarySearchLookup(lowerVal == null ? null : new BytesRef(lowerVal), spare);
         final int upperPoint = fcsi.binarySearchLookup(upperVal == null ? null : new BytesRef(upperVal), spare);
@@ -122,7 +122,7 @@ public abstract class FieldCacheRangeFil
         
         assert inclusiveLowerPoint > 0 && inclusiveUpperPoint > 0;
         
-        return new FieldCacheDocIdSet(context.reader.maxDoc(), acceptDocs) {
+        return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
           @Override
           protected final boolean matchDoc(int doc) {
             final int docOrd = fcsi.getOrd(doc);
@@ -134,7 +134,7 @@ public abstract class FieldCacheRangeFil
   }
   
   /**
-   * Creates a numeric range filter using {@link FieldCache#getBytes(IndexReader,String,boolean)}. This works with all
+   * Creates a numeric range filter using {@link FieldCache#getBytes(AtomicReader,String,boolean)}. This works with all
    * byte 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>.
    */
@@ -143,7 +143,7 @@ public abstract class FieldCacheRangeFil
   }
   
   /**
-   * Creates a numeric range filter using {@link FieldCache#getBytes(IndexReader,String,FieldCache.ByteParser,boolean)}. This works with all
+   * Creates a numeric range filter using {@link FieldCache#getBytes(AtomicReader,String,FieldCache.ByteParser,boolean)}. This works with all
    * byte 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>.
    */
@@ -172,8 +172,8 @@ public abstract class FieldCacheRangeFil
         if (inclusiveLowerPoint > inclusiveUpperPoint)
           return DocIdSet.EMPTY_DOCIDSET;
         
-        final byte[] values = FieldCache.DEFAULT.getBytes(context.reader, field, (FieldCache.ByteParser) parser, false);
-        return new FieldCacheDocIdSet(context.reader.maxDoc(), acceptDocs) {
+        final byte[] values = FieldCache.DEFAULT.getBytes(context.reader(), field, (FieldCache.ByteParser) parser, false);
+        return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
           @Override
           protected boolean matchDoc(int doc) {
             return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
@@ -184,7 +184,7 @@ public abstract class FieldCacheRangeFil
   }
   
   /**
-   * Creates a numeric range filter using {@link FieldCache#getShorts(IndexReader,String,boolean)}. This works with all
+   * Creates a numeric range filter using {@link FieldCache#getShorts(AtomicReader,String,boolean)}. This works with all
    * short 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>.
    */
@@ -193,7 +193,7 @@ public abstract class FieldCacheRangeFil
   }
   
   /**
-   * Creates a numeric range filter using {@link FieldCache#getShorts(IndexReader,String,FieldCache.ShortParser,boolean)}. This works with all
+   * Creates a numeric range filter using {@link FieldCache#getShorts(AtomicReader,String,FieldCache.ShortParser,boolean)}. This works with all
    * short 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>.
    */
@@ -222,8 +222,8 @@ public abstract class FieldCacheRangeFil
         if (inclusiveLowerPoint > inclusiveUpperPoint)
           return DocIdSet.EMPTY_DOCIDSET;
         
-        final short[] values = FieldCache.DEFAULT.getShorts(context.reader, field, (FieldCache.ShortParser) parser, false);
-        return new FieldCacheDocIdSet(context.reader.maxDoc(), acceptDocs) {
+        final short[] values = FieldCache.DEFAULT.getShorts(context.reader(), field, (FieldCache.ShortParser) parser, false);
+        return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
           @Override
           protected boolean matchDoc(int doc) {
             return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
@@ -234,7 +234,7 @@ public abstract class FieldCacheRangeFil
   }
   
   /**
-   * Creates a numeric range filter using {@link FieldCache#getInts(IndexReader,String,boolean)}. This works with all
+   * Creates a numeric range filter using {@link FieldCache#getInts(AtomicReader,String,boolean)}. 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>.
    */
@@ -243,7 +243,7 @@ public abstract class FieldCacheRangeFil
   }
   
   /**
-   * Creates a numeric range filter using {@link FieldCache#getInts(IndexReader,String,FieldCache.IntParser,boolean)}. This works with all
+   * Creates a numeric range filter using {@link FieldCache#getInts(AtomicReader,String,FieldCache.IntParser,boolean)}. 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>.
    */
@@ -272,8 +272,8 @@ public abstract class FieldCacheRangeFil
         if (inclusiveLowerPoint > inclusiveUpperPoint)
           return DocIdSet.EMPTY_DOCIDSET;
         
-        final int[] values = FieldCache.DEFAULT.getInts(context.reader, field, (FieldCache.IntParser) parser, false);
-        return new FieldCacheDocIdSet(context.reader.maxDoc(), acceptDocs) {
+        final int[] values = FieldCache.DEFAULT.getInts(context.reader(), field, (FieldCache.IntParser) parser, false);
+        return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
           @Override
           protected boolean matchDoc(int doc) {
             return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
@@ -284,7 +284,7 @@ public abstract class FieldCacheRangeFil
   }
   
   /**
-   * Creates a numeric range filter using {@link FieldCache#getLongs(IndexReader,String,boolean)}. This works with all
+   * Creates a numeric range filter using {@link FieldCache#getLongs(AtomicReader,String,boolean)}. 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>.
    */
@@ -293,7 +293,7 @@ public abstract class FieldCacheRangeFil
   }
   
   /**
-   * Creates a numeric range filter using {@link FieldCache#getLongs(IndexReader,String,FieldCache.LongParser,boolean)}. This works with all
+   * Creates a numeric range filter using {@link FieldCache#getLongs(AtomicReader,String,FieldCache.LongParser,boolean)}. 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>.
    */
@@ -322,8 +322,8 @@ public abstract class FieldCacheRangeFil
         if (inclusiveLowerPoint > inclusiveUpperPoint)
           return DocIdSet.EMPTY_DOCIDSET;
         
-        final long[] values = FieldCache.DEFAULT.getLongs(context.reader, field, (FieldCache.LongParser) parser, false);
-        return new FieldCacheDocIdSet(context.reader.maxDoc(), acceptDocs) {
+        final long[] values = FieldCache.DEFAULT.getLongs(context.reader(), field, (FieldCache.LongParser) parser, false);
+        return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
           @Override
           protected boolean matchDoc(int doc) {
             return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
@@ -334,7 +334,7 @@ public abstract class FieldCacheRangeFil
   }
   
   /**
-   * Creates a numeric range filter using {@link FieldCache#getFloats(IndexReader,String,boolean)}. This works with all
+   * Creates a numeric range filter using {@link FieldCache#getFloats(AtomicReader,String,boolean)}. 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>.
    */
@@ -343,7 +343,7 @@ public abstract class FieldCacheRangeFil
   }
   
   /**
-   * Creates a numeric range filter using {@link FieldCache#getFloats(IndexReader,String,FieldCache.FloatParser,boolean)}. This works with all
+   * Creates a numeric range filter using {@link FieldCache#getFloats(AtomicReader,String,FieldCache.FloatParser,boolean)}. 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>.
    */
@@ -376,8 +376,8 @@ public abstract class FieldCacheRangeFil
         if (inclusiveLowerPoint > inclusiveUpperPoint)
           return DocIdSet.EMPTY_DOCIDSET;
         
-        final float[] values = FieldCache.DEFAULT.getFloats(context.reader, field, (FieldCache.FloatParser) parser, false);
-        return new FieldCacheDocIdSet(context.reader.maxDoc(), acceptDocs) {
+        final float[] values = FieldCache.DEFAULT.getFloats(context.reader(), field, (FieldCache.FloatParser) parser, false);
+        return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
           @Override
           protected boolean matchDoc(int doc) {
             return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
@@ -388,7 +388,7 @@ public abstract class FieldCacheRangeFil
   }
   
   /**
-   * Creates a numeric range filter using {@link FieldCache#getDoubles(IndexReader,String,boolean)}. This works with all
+   * Creates a numeric range filter using {@link FieldCache#getDoubles(AtomicReader,String,boolean)}. 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>.
    */
@@ -397,7 +397,7 @@ public abstract class FieldCacheRangeFil
   }
   
   /**
-   * Creates a numeric range filter using {@link FieldCache#getDoubles(IndexReader,String,FieldCache.DoubleParser,boolean)}. This works with all
+   * Creates a numeric range filter using {@link FieldCache#getDoubles(AtomicReader,String,FieldCache.DoubleParser,boolean)}. 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>.
    */
@@ -430,9 +430,9 @@ public abstract class FieldCacheRangeFil
         if (inclusiveLowerPoint > inclusiveUpperPoint)
           return DocIdSet.EMPTY_DOCIDSET;
         
-        final double[] values = FieldCache.DEFAULT.getDoubles(context.reader, field, (FieldCache.DoubleParser) parser, false);
+        final double[] values = FieldCache.DEFAULT.getDoubles(context.reader(), field, (FieldCache.DoubleParser) parser, false);
         // ignore deleted docs if range doesn't contain 0
-        return new FieldCacheDocIdSet(context.reader.maxDoc(), acceptDocs) {
+        return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
           @Override
           protected boolean matchDoc(int doc) {
             return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java Mon Jan 30 23:34:03 2012
@@ -19,9 +19,9 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DocsEnum; // javadoc @link
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -118,7 +118,7 @@ public class FieldCacheTermsFilter exten
 
   @Override
   public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
-    final FieldCache.DocTermsIndex fcsi = getFieldCache().getTermsIndex(context.reader, field);
+    final FieldCache.DocTermsIndex fcsi = getFieldCache().getTermsIndex(context.reader(), field);
     final FixedBitSet bits = new FixedBitSet(fcsi.numOrd());
     final BytesRef spare = new BytesRef();
     for (int i=0;i<terms.length;i++) {
@@ -127,7 +127,7 @@ public class FieldCacheTermsFilter exten
         bits.set(termNumber);
       }
     }
-    return new FieldCacheDocIdSet(context.reader.maxDoc(), acceptDocs) {
+    return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
       @Override
       protected final boolean matchDoc(int doc) {
         return bits.get(fcsi.getOrd(doc));

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldComparator.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldComparator.java Mon Jan 30 23:34:03 2012
@@ -20,9 +20,9 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.util.Comparator;
 
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.AtomicReader; // javadocs
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.FieldCache.ByteParser;
 import org.apache.lucene.search.FieldCache.DocTerms;
 import org.apache.lucene.search.FieldCache.DocTermsIndex;
@@ -72,7 +72,7 @@ import org.apache.lucene.util.packed.Pac
  *       priority queue.  The {@link FieldValueHitQueue}
  *       calls this method when a new hit is competitive.
  *
- *  <li> {@link #setNextReader(IndexReader.AtomicReaderContext)} Invoked
+ *  <li> {@link #setNextReader(AtomicReaderContext)} 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
@@ -203,7 +203,7 @@ public abstract class FieldComparator<T>
     @Override
     public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
       if (missingValue != null) {
-        docsWithField = FieldCache.DEFAULT.getDocsWithField(context.reader, field);
+        docsWithField = FieldCache.DEFAULT.getDocsWithField(context.reader(), field);
         // optimization to remove unneeded checks on the bit interface:
         if (docsWithField instanceof Bits.MatchAllBits) {
           docsWithField = null;
@@ -261,7 +261,7 @@ public abstract class FieldComparator<T>
     public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
       // NOTE: must do this before calling super otherwise
       // we compute the docsWithField Bits twice!
-      currentReaderValues = FieldCache.DEFAULT.getBytes(context.reader, field, parser, missingValue != null);
+      currentReaderValues = FieldCache.DEFAULT.getBytes(context.reader(), field, parser, missingValue != null);
       return super.setNextReader(context);
     }
     
@@ -338,7 +338,7 @@ public abstract class FieldComparator<T>
     public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
       // NOTE: must do this before calling super otherwise
       // we compute the docsWithField Bits twice!
-      currentReaderValues = FieldCache.DEFAULT.getDoubles(context.reader, field, parser, missingValue != null);
+      currentReaderValues = FieldCache.DEFAULT.getDoubles(context.reader(), field, parser, missingValue != null);
       return super.setNextReader(context);
     }
     
@@ -397,7 +397,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
-      final DocValues docValues = context.reader.docValues(field);
+      final DocValues docValues = context.reader().docValues(field);
       if (docValues != null) {
         currentReaderValues = docValues.getSource(); 
       } else {
@@ -481,7 +481,7 @@ public abstract class FieldComparator<T>
     public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
       // NOTE: must do this before calling super otherwise
       // we compute the docsWithField Bits twice!
-      currentReaderValues = FieldCache.DEFAULT.getFloats(context.reader, field, parser, missingValue != null);
+      currentReaderValues = FieldCache.DEFAULT.getFloats(context.reader(), field, parser, missingValue != null);
       return super.setNextReader(context);
     }
     
@@ -543,7 +543,7 @@ public abstract class FieldComparator<T>
     public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
       // NOTE: must do this before calling super otherwise
       // we compute the docsWithField Bits twice!
-      currentReaderValues = FieldCache.DEFAULT.getShorts(context.reader, field, parser, missingValue != null);
+      currentReaderValues = FieldCache.DEFAULT.getShorts(context.reader(), field, parser, missingValue != null);
       return super.setNextReader(context);
     }
 
@@ -627,7 +627,7 @@ public abstract class FieldComparator<T>
     public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
       // NOTE: must do this before calling super otherwise
       // we compute the docsWithField Bits twice!
-      currentReaderValues = FieldCache.DEFAULT.getInts(context.reader, field, parser, missingValue != null);
+      currentReaderValues = FieldCache.DEFAULT.getInts(context.reader(), field, parser, missingValue != null);
       return super.setNextReader(context);
     }
     
@@ -690,7 +690,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
-      DocValues docValues = context.reader.docValues(field);
+      DocValues docValues = context.reader().docValues(field);
       if (docValues != null) {
         currentReaderValues = docValues.getSource();
       } else {
@@ -775,7 +775,7 @@ public abstract class FieldComparator<T>
     public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
       // NOTE: must do this before calling super otherwise
       // we compute the docsWithField Bits twice!
-      currentReaderValues = FieldCache.DEFAULT.getLongs(context.reader, field, parser, missingValue != null);
+      currentReaderValues = FieldCache.DEFAULT.getLongs(context.reader(), field, parser, missingValue != null);
       return super.setNextReader(context);
     }
     
@@ -1288,7 +1288,7 @@ public abstract class FieldComparator<T>
     @Override
     public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
       final int docBase = context.docBase;
-      termsIndex = FieldCache.DEFAULT.getTermsIndex(context.reader, field);
+      termsIndex = FieldCache.DEFAULT.getTermsIndex(context.reader(), field);
       final PackedInts.Reader docToOrd = termsIndex.getDocToOrd();
       FieldComparator perSegComp = null;
       if (docToOrd.hasArray()) {
@@ -1706,19 +1706,19 @@ public abstract class FieldComparator<T>
     public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
       final int docBase = context.docBase;
 
-      final DocValues dv = context.reader.docValues(field);
+      final DocValues dv = context.reader().docValues(field);
       if (dv == null) {
         // This may mean entire segment had no docs with
         // this DV field; use default field value (empty
         // byte[]) in this case:
-        termsIndex = DocValues.getDefaultSortedSource(DocValues.Type.BYTES_VAR_SORTED, context.reader.maxDoc());
+        termsIndex = DocValues.getDefaultSortedSource(DocValues.Type.BYTES_VAR_SORTED, context.reader().maxDoc());
       } else {
         termsIndex = dv.getSource().asSortedSource();
         if (termsIndex == null) {
           // This means segment has doc values, but they are
           // not able to provide a sorted source; consider
           // this a hard error:
-          throw new IllegalStateException("DocValues exist for field \"" + field + "\", but not as a sorted source: type=" + dv.getSource().type() + " reader=" + context.reader);
+          throw new IllegalStateException("DocValues exist for field \"" + field + "\", but not as a sorted source: type=" + dv.getSource().type() + " reader=" + context.reader());
         }
       }
 
@@ -1853,7 +1853,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
-      docTerms = FieldCache.DEFAULT.getTerms(context.reader, field);
+      docTerms = FieldCache.DEFAULT.getTerms(context.reader(), field);
       return this;
     }
     
@@ -1885,7 +1885,7 @@ public abstract class FieldComparator<T>
    *  comparisons are done using BytesRef.compareTo, which is
    *  slow for medium to large result sets but possibly
    *  very fast for very small results sets.  The BytesRef
-   *  values are obtained using {@link IndexReader#docValues}. */
+   *  values are obtained using {@link AtomicReader#docValues}. */
   public static final class TermValDocValuesComparator extends FieldComparator<BytesRef> {
 
     private BytesRef[] values;
@@ -1922,7 +1922,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
-      final DocValues dv = context.reader.docValues(field);
+      final DocValues dv = context.reader().docValues(field);
       if (dv != null) {
         docTerms = dv.getSource();
       } else {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldValueFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldValueFilter.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldValueFilter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldValueFilter.java Mon Jan 30 23:34:03 2012
@@ -18,7 +18,7 @@ package org.apache.lucene.search;
  */
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.Bits.MatchAllBits;
 import org.apache.lucene.util.Bits.MatchNoBits;
@@ -77,12 +77,12 @@ public class FieldValueFilter extends Fi
   public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs)
       throws IOException {
     final Bits docsWithField = FieldCache.DEFAULT.getDocsWithField(
-        context.reader, field);
+        context.reader(), field);
     if (negate) {
       if (docsWithField instanceof MatchAllBits) {
         return null;
       }
-      return new FieldCacheDocIdSet(context.reader.maxDoc(), acceptDocs) {
+      return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
         @Override
         protected final boolean matchDoc(int doc) {
           return !docsWithField.get(doc);
@@ -97,7 +97,7 @@ public class FieldValueFilter extends Fi
         // :-)
         return BitsFilteredDocIdSet.wrap((DocIdSet) docsWithField, acceptDocs);
       }
-      return new FieldCacheDocIdSet(context.reader.maxDoc(), acceptDocs) {
+      return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
         @Override
         protected final boolean matchDoc(int doc) {
           return docsWithField.get(doc);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/Filter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/Filter.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/Filter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/Filter.java Mon Jan 30 23:34:03 2012
@@ -19,8 +19,9 @@ 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.IndexReader; // javadocs
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.util.Bits;
 
 /** 
@@ -44,7 +45,7 @@ public abstract class Filter {
    *         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 IndexReader#fields()}
+   *         {@link AtomicReader#fields()}
    *         on the context's reader, for example.
    *
    * @param acceptDocs

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredQuery.java Mon Jan 30 23:34:03 2012
@@ -17,8 +17,8 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
@@ -100,7 +100,7 @@ public class FilteredQuery extends Query
       public Explanation explain (AtomicReaderContext ir, int i) throws IOException {
         Explanation inner = weight.explain (ir, i);
         Filter f = FilteredQuery.this.filter;
-        DocIdSet docIdSet = f.getDocIdSet(ir, ir.reader.getLiveDocs());
+        DocIdSet docIdSet = f.getDocIdSet(ir, ir.reader().getLiveDocs());
         DocIdSetIterator docIdSetIterator = docIdSet == null ? DocIdSet.EMPTY_DOCIDSET.iterator() : docIdSet.iterator();
         if (docIdSetIterator == null) {
           docIdSetIterator = DocIdSet.EMPTY_DOCIDSET.iterator();

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Mon Jan 30 23:34:03 2012
@@ -31,11 +31,12 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.lucene.document.Document;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
-import org.apache.lucene.index.IndexReader.ReaderContext;
+import org.apache.lucene.index.DirectoryReader; // javadocs
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
@@ -56,10 +57,11 @@ import org.apache.lucene.util.ThreadInte
  * multiple searches instead of creating a new one
  * per-search.  If your index has changed and you wish to
  * see the changes reflected in searching, you should
- * use {@link IndexReader#openIfChanged} to obtain a new reader and
+ * use {@link DirectoryReader#openIfChanged(DirectoryReader)}
+ * to obtain a new reader and
  * then create a new IndexSearcher from that.  Also, for
  * low-latency turnaround it's best to use a near-real-time
- * reader ({@link IndexReader#open(IndexWriter,boolean)}).
+ * reader ({@link DirectoryReader#open(IndexWriter,boolean)}).
  * Once you have a new {@link IndexReader}, it's relatively
  * cheap to create a new IndexSearcher from it.
  * 
@@ -76,7 +78,7 @@ public class IndexSearcher {
   
   // NOTE: these members might change in incompatible ways
   // in the next release
-  protected final ReaderContext readerContext;
+  protected final IndexReaderContext readerContext;
   protected final AtomicReaderContext[] leafContexts;
   // used with executor - each slice holds a set of leafs executed within one thread
   protected final LeafSlice[] leafSlices;
@@ -122,7 +124,7 @@ public class IndexSearcher {
   }
 
   /**
-   * Creates a searcher searching the provided top-level {@link ReaderContext}.
+   * Creates a searcher searching the provided top-level {@link IndexReaderContext}.
    * <p>
    * Given a non-<code>null</code> {@link ExecutorService} this method runs
    * searches for each segment separately, using the provided ExecutorService.
@@ -133,13 +135,13 @@ public class IndexSearcher {
    * silently close file descriptors (see <a
    * href="https://issues.apache.org/jira/browse/LUCENE-2239">LUCENE-2239</a>).
    * 
-   * @see ReaderContext
+   * @see IndexReaderContext
    * @see IndexReader#getTopReaderContext()
    * @lucene.experimental
    */
-  public IndexSearcher(ReaderContext context, ExecutorService executor) {
-    assert context.isTopLevel: "IndexSearcher's ReaderContext must be topLevel for reader" + context.reader;
-    reader = context.reader;
+  public IndexSearcher(IndexReaderContext context, ExecutorService executor) {
+    assert context.isTopLevel: "IndexSearcher's ReaderContext must be topLevel for reader" + context.reader();
+    reader = context.reader();
     this.executor = executor;
     this.readerContext = context;
     leafContexts = ReaderUtil.leaves(context);
@@ -147,13 +149,13 @@ public class IndexSearcher {
   }
 
   /**
-   * Creates a searcher searching the provided top-level {@link ReaderContext}.
+   * Creates a searcher searching the provided top-level {@link IndexReaderContext}.
    *
-   * @see ReaderContext
+   * @see IndexReaderContext
    * @see IndexReader#getTopReaderContext()
    * @lucene.experimental
    */
-  public IndexSearcher(ReaderContext context) {
+  public IndexSearcher(IndexReaderContext context) {
     this(context, null);
   }
   
@@ -402,7 +404,7 @@ public class IndexSearcher {
    * <p>NOTE: this does not compute scores by default.  If you
    * need scores, create a {@link TopFieldCollector}
    * instance by calling {@link TopFieldCollector#create} and
-   * then pass that to {@link #search(IndexReader.AtomicReaderContext[], Weight,
+   * then pass that to {@link #search(AtomicReaderContext[], Weight,
    * Collector)}.</p>
    */
   protected TopFieldDocs search(Weight weight, int nDocs,
@@ -451,7 +453,7 @@ public class IndexSearcher {
    * <p>NOTE: this does not compute scores by default.  If you
    * need scores, create a {@link TopFieldCollector}
    * instance by calling {@link TopFieldCollector#create} and
-   * then pass that to {@link #search(IndexReader.AtomicReaderContext[], Weight, 
+   * then pass that to {@link #search(AtomicReaderContext[], Weight, 
    * Collector)}.</p>
    */
   protected TopFieldDocs search(AtomicReaderContext[] leaves, Weight weight, int nDocs,
@@ -501,7 +503,7 @@ public class IndexSearcher {
     // always use single thread:
     for (int i = 0; i < leaves.length; i++) { // search each subreader
       collector.setNextReader(leaves[i]);
-      Scorer scorer = weight.scorer(leaves[i], !collector.acceptsDocsOutOfOrder(), true, leaves[i].reader.getLiveDocs());
+      Scorer scorer = weight.scorer(leaves[i], !collector.acceptsDocsOutOfOrder(), true, leaves[i].reader().getLiveDocs());
       if (scorer != null) {
         scorer.score(collector);
       }
@@ -589,11 +591,11 @@ public class IndexSearcher {
   }
   
   /**
-   * Returns this searchers the top-level {@link ReaderContext}.
+   * Returns this searchers the top-level {@link IndexReaderContext}.
    * @see IndexReader#getTopReaderContext()
    */
   /* sugar for #getReader().getTopReaderContext() */
-  public ReaderContext getTopReaderContext() {
+  public IndexReaderContext getTopReaderContext() {
     return readerContext;
   }
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java Mon Jan 30 23:34:03 2012
@@ -17,8 +17,8 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.Bits;
@@ -106,7 +106,7 @@ public class MatchAllDocsQuery extends Q
     @Override
     public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
         boolean topScorer, Bits acceptDocs) throws IOException {
-      return new MatchAllScorer(context.reader, acceptDocs, this, queryWeight);
+      return new MatchAllScorer(context.reader(), acceptDocs, this, queryWeight);
     }
 
     @Override