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 [2/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/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DirectoryReader.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DirectoryReader.java Mon Jan 30 23:34:03 2012
@@ -26,20 +26,126 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.lucene.search.SearcherManager; // javadocs
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.IOUtils;
 
-/** 
- * An IndexReader which reads indexes with multiple segments.
- */
-final class DirectoryReader extends BaseMultiReader<SegmentReader> {
+/** DirectoryReader is an implementation of {@link CompositeReader}
+ that can read indexes in a {@link Directory}. 
+
+ <p>DirectoryReader instances are usually constructed with a call to
+ one of the static <code>open()</code> methods, e.g. {@link
+ #open(Directory)}.
+
+ <p> For efficiency, in this API documents are often referred to via
+ <i>document numbers</i>, non-negative integers which each name a unique
+ document in the index.  These document numbers are ephemeral -- they may change
+ as documents are added to and deleted from an index.  Clients should thus not
+ rely on a given document having the same number between sessions.
+
+ <p>
+ <a name="thread-safety"></a><p><b>NOTE</b>: {@link
+ IndexReader} instances are completely thread
+ safe, meaning multiple threads can call any of its methods,
+ concurrently.  If your application requires external
+ synchronization, you should <b>not</b> synchronize on the
+ <code>IndexReader</code> instance; use your own
+ (non-Lucene) objects instead.
+*/
+public final class DirectoryReader extends BaseMultiReader<SegmentReader> {
+  static int DEFAULT_TERMS_INDEX_DIVISOR = 1;
+
   protected final Directory directory;
   private final IndexWriter writer;
   private final SegmentInfos segmentInfos;
   private final int termInfosIndexDivisor;
   private final boolean applyAllDeletes;
   
+  /** Returns a IndexReader reading the index in the given
+   *  Directory
+   * @param directory the index directory
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
+   */
+  public static DirectoryReader open(final Directory directory) throws CorruptIndexException, IOException {
+    return open(directory, null, DEFAULT_TERMS_INDEX_DIVISOR);
+  }
+  
+  /** Expert: Returns a IndexReader reading the index in the given
+   *  Directory with the given termInfosIndexDivisor.
+   * @param directory the index directory
+   * @param termInfosIndexDivisor Subsamples which indexed
+   *  terms are loaded into RAM. This has the same effect as {@link
+   *  IndexWriterConfig#setTermIndexInterval} except that setting
+   *  must be done at indexing time while this setting can be
+   *  set per reader.  When set to N, then one in every
+   *  N*termIndexInterval terms in the index is loaded into
+   *  memory.  By setting this to a value > 1 you can reduce
+   *  memory usage, at the expense of higher latency when
+   *  loading a TermInfo.  The default value is 1.  Set this
+   *  to -1 to skip loading the terms index entirely.
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
+   */
+  public static DirectoryReader open(final Directory directory, int termInfosIndexDivisor) throws CorruptIndexException, IOException {
+    return open(directory, null, termInfosIndexDivisor);
+  }
+  
+  /**
+   * Open a near real time IndexReader from the {@link org.apache.lucene.index.IndexWriter}.
+   *
+   * @param writer The IndexWriter to open from
+   * @param applyAllDeletes If true, all buffered deletes will
+   * be applied (made visible) in the returned reader.  If
+   * false, the deletes are not applied but remain buffered
+   * (in IndexWriter) so that they will be applied in the
+   * future.  Applying deletes can be costly, so if your app
+   * can tolerate deleted documents being returned you might
+   * gain some performance by passing false.
+   * @return The new IndexReader
+   * @throws CorruptIndexException
+   * @throws IOException if there is a low-level IO error
+   *
+   * @see #openIfChanged(DirectoryReader,IndexWriter,boolean)
+   *
+   * @lucene.experimental
+   */
+  public static DirectoryReader open(final IndexWriter writer, boolean applyAllDeletes) throws CorruptIndexException, IOException {
+    return writer.getReader(applyAllDeletes);
+  }
+
+  /** Expert: returns an IndexReader reading the index in the given
+   *  {@link IndexCommit}.
+   * @param commit the commit point to open
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
+   */
+  public static DirectoryReader open(final IndexCommit commit) throws CorruptIndexException, IOException {
+    return open(commit.getDirectory(), commit, DEFAULT_TERMS_INDEX_DIVISOR);
+  }
+
+
+  /** Expert: returns an IndexReader reading the index in the given
+   *  {@link IndexCommit} and termInfosIndexDivisor.
+   * @param commit the commit point to open
+   * @param termInfosIndexDivisor Subsamples which indexed
+   *  terms are loaded into RAM. This has the same effect as {@link
+   *  IndexWriterConfig#setTermIndexInterval} except that setting
+   *  must be done at indexing time while this setting can be
+   *  set per reader.  When set to N, then one in every
+   *  N*termIndexInterval terms in the index is loaded into
+   *  memory.  By setting this to a value > 1 you can reduce
+   *  memory usage, at the expense of higher latency when
+   *  loading a TermInfo.  The default value is 1.  Set this
+   *  to -1 to skip loading the terms index entirely.
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
+   */
+  public static DirectoryReader open(final IndexCommit commit, int termInfosIndexDivisor) throws CorruptIndexException, IOException {
+    return open(commit.getDirectory(), commit, termInfosIndexDivisor);
+  }
+
   DirectoryReader(SegmentReader[] readers, Directory directory, IndexWriter writer,
     SegmentInfos sis, int termInfosIndexDivisor, boolean applyAllDeletes) throws IOException {
     super(readers);
@@ -50,9 +156,9 @@ final class DirectoryReader extends Base
     this.applyAllDeletes = applyAllDeletes;
   }
 
-  static IndexReader open(final Directory directory, final IndexCommit commit,
+  private static DirectoryReader open(final Directory directory, final IndexCommit commit,
                           final int termInfosIndexDivisor) throws CorruptIndexException, IOException {
-    return (IndexReader) new SegmentInfos.FindSegmentsFile(directory) {
+    return (DirectoryReader) new SegmentInfos.FindSegmentsFile(directory) {
       @Override
       protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
         SegmentInfos sis = new SegmentInfos();
@@ -116,7 +222,7 @@ final class DirectoryReader extends Base
   }
 
   /** This constructor is only used for {@link #doOpenIfChanged()} */
-  static DirectoryReader open(Directory directory, IndexWriter writer, SegmentInfos infos, SegmentReader[] oldReaders,
+  private static DirectoryReader open(Directory directory, IndexWriter writer, SegmentInfos infos, SegmentReader[] oldReaders,
     int termInfosIndexDivisor) throws IOException {
     // we put the old SegmentReaders in a map, that allows us
     // to lookup a reader using its segment name
@@ -202,6 +308,116 @@ final class DirectoryReader extends Base
         infos, termInfosIndexDivisor, false);
   }
 
+  /**
+   * If the index has changed since the provided reader was
+   * opened, open and return a new reader; else, return
+   * null.  The new reader, if not null, will be the same
+   * type of reader as the previous one, ie an NRT reader
+   * will open a new NRT reader, a MultiReader will open a
+   * new MultiReader,  etc.
+   *
+   * <p>This method is typically far less costly than opening a
+   * fully new <code>DirectoryReader</code> as it shares
+   * resources (for example sub-readers) with the provided
+   * <code>DirectoryReader</code>, when possible.
+   *
+   * <p>The provided reader is not closed (you are responsible
+   * for doing so); if a new reader is returned you also
+   * must eventually close it.  Be sure to never close a
+   * reader while other threads are still using it; see
+   * {@link SearcherManager} to simplify managing this.
+   *
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
+   * @return null if there are no changes; else, a new
+   * DirectoryReader instance which you must eventually close
+   */  
+  public static DirectoryReader openIfChanged(DirectoryReader oldReader) throws IOException {
+    final DirectoryReader newReader = oldReader.doOpenIfChanged();
+    assert newReader != oldReader;
+    return newReader;
+  }
+
+  /**
+   * If the IndexCommit differs from what the
+   * provided reader is searching, open and return a new
+   * reader; else, return null.
+   *
+   * @see #openIfChanged(DirectoryReader)
+   */
+  public static DirectoryReader openIfChanged(DirectoryReader oldReader, IndexCommit commit) throws IOException {
+    final DirectoryReader newReader = oldReader.doOpenIfChanged(commit);
+    assert newReader != oldReader;
+    return newReader;
+  }
+
+  /**
+   * Expert: If there changes (committed or not) in the
+   * {@link IndexWriter} versus what the provided reader is
+   * searching, then open and return a new
+   * IndexReader searching both committed and uncommitted
+   * changes from the writer; else, return null (though, the
+   * current implementation never returns null).
+   *
+   * <p>This provides "near real-time" searching, in that
+   * changes made during an {@link IndexWriter} session can be
+   * quickly made available for searching without closing
+   * the writer nor calling {@link IndexWriter#commit}.
+   *
+   * <p>It's <i>near</i> real-time because there is no hard
+   * guarantee on how quickly you can get a new reader after
+   * making changes with IndexWriter.  You'll have to
+   * experiment in your situation to determine if it's
+   * fast enough.  As this is a new and experimental
+   * feature, please report back on your findings so we can
+   * learn, improve and iterate.</p>
+   *
+   * <p>The very first time this method is called, this
+   * writer instance will make every effort to pool the
+   * readers that it opens for doing merges, applying
+   * deletes, etc.  This means additional resources (RAM,
+   * file descriptors, CPU time) will be consumed.</p>
+   *
+   * <p>For lower latency on reopening a reader, you should
+   * call {@link IndexWriterConfig#setMergedSegmentWarmer} to
+   * pre-warm a newly merged segment before it's committed
+   * to the index.  This is important for minimizing
+   * index-to-search delay after a large merge.  </p>
+   *
+   * <p>If an addIndexes* call is running in another thread,
+   * then this reader will only search those segments from
+   * the foreign index that have been successfully copied
+   * over, so far.</p>
+   *
+   * <p><b>NOTE</b>: Once the writer is closed, any
+   * outstanding readers may continue to be used.  However,
+   * if you attempt to reopen any of those readers, you'll
+   * hit an {@link org.apache.lucene.store.AlreadyClosedException}.</p>
+   *
+   * @return DirectoryReader that covers entire index plus all
+   * changes made so far by this IndexWriter instance, or
+   * null if there are no new changes
+   *
+   * @param writer The IndexWriter to open from
+   *
+   * @param applyAllDeletes If true, all buffered deletes will
+   * be applied (made visible) in the returned reader.  If
+   * false, the deletes are not applied but remain buffered
+   * (in IndexWriter) so that they will be applied in the
+   * future.  Applying deletes can be costly, so if your app
+   * can tolerate deleted documents being returned you might
+   * gain some performance by passing false.
+   *
+   * @throws IOException
+   *
+   * @lucene.experimental
+   */
+  public static DirectoryReader openIfChanged(DirectoryReader oldReader, IndexWriter writer, boolean applyAllDeletes) throws IOException {
+    final DirectoryReader newReader = oldReader.doOpenIfChanged(writer, applyAllDeletes);
+    assert newReader != oldReader;
+    return newReader;
+  }
+
   /** {@inheritDoc} */
   @Override
   public String toString() {
@@ -223,13 +439,11 @@ final class DirectoryReader extends Base
     return buffer.toString();
   }
 
-  @Override
-  protected final IndexReader doOpenIfChanged() throws CorruptIndexException, IOException {
+  protected final DirectoryReader doOpenIfChanged() throws CorruptIndexException, IOException {
     return doOpenIfChanged(null);
   }
 
-  @Override
-  protected final IndexReader doOpenIfChanged(final IndexCommit commit) throws CorruptIndexException, IOException {
+  protected final DirectoryReader doOpenIfChanged(final IndexCommit commit) throws CorruptIndexException, IOException {
     ensureOpen();
 
     // If we were obtained by writer.getReader(), re-ask the
@@ -241,18 +455,16 @@ final class DirectoryReader extends Base
     }
   }
 
-  @Override
-  protected final IndexReader doOpenIfChanged(IndexWriter writer, boolean applyAllDeletes) throws CorruptIndexException, IOException {
+  protected final DirectoryReader doOpenIfChanged(IndexWriter writer, boolean applyAllDeletes) throws CorruptIndexException, IOException {
     ensureOpen();
     if (writer == this.writer && applyAllDeletes == this.applyAllDeletes) {
       return doOpenFromWriter(null);
     } else {
-      // fail by calling supers impl throwing UOE
-      return super.doOpenIfChanged(writer, applyAllDeletes);
+      return writer.getReader(applyAllDeletes);
     }
   }
 
-  private final IndexReader doOpenFromWriter(IndexCommit commit) throws CorruptIndexException, IOException {
+  private final DirectoryReader doOpenFromWriter(IndexCommit commit) throws CorruptIndexException, IOException {
     if (commit != null) {
       throw new IllegalArgumentException("a reader obtained from IndexWriter.getReader() cannot currently accept a commit");
     }
@@ -261,7 +473,7 @@ final class DirectoryReader extends Base
       return null;
     }
 
-    IndexReader reader = writer.getReader(applyAllDeletes);
+    DirectoryReader reader = writer.getReader(applyAllDeletes);
 
     // If in fact no changes took place, return null:
     if (reader.getVersion() == segmentInfos.getVersion()) {
@@ -272,7 +484,7 @@ final class DirectoryReader extends Base
     return reader;
   }
 
-  private synchronized IndexReader doOpenNoWriter(IndexCommit commit) throws CorruptIndexException, IOException {
+  private synchronized DirectoryReader doOpenNoWriter(IndexCommit commit) throws CorruptIndexException, IOException {
 
     if (commit == null) {
       if (isCurrent()) {
@@ -287,7 +499,7 @@ final class DirectoryReader extends Base
       }
     }
 
-    return (IndexReader) new SegmentInfos.FindSegmentsFile(directory) {
+    return (DirectoryReader) new SegmentInfos.FindSegmentsFile(directory) {
       @Override
       protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
         final SegmentInfos infos = new SegmentInfos();
@@ -301,20 +513,25 @@ final class DirectoryReader extends Base
     return DirectoryReader.open(directory, writer, infos, subReaders, termInfosIndexDivisor);
   }
 
-  /** Version number when this IndexReader was opened. */
-  @Override
+  /**
+   * Version number when this IndexReader was opened. Not
+   * implemented in the IndexReader base class.
+   *
+   * <p>This method
+   * returns the version recorded in the commit that the
+   * reader opened.  This version is advanced every time
+   * a change is made with {@link IndexWriter}.</p>
+   */
   public long getVersion() {
     ensureOpen();
     return segmentInfos.getVersion();
   }
 
-  @Override
   public Map<String,String> getCommitUserData() {
     ensureOpen();
     return segmentInfos.getUserData();
   }
 
-  @Override
   public boolean isCurrent() throws CorruptIndexException, IOException {
     ensureOpen();
     if (writer == null || writer.isClosed()) {
@@ -348,7 +565,6 @@ final class DirectoryReader extends Base
   }
 
   /** 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
@@ -356,7 +572,6 @@ final class DirectoryReader extends Base
     return directory;
   }
 
-  @Override
   public int getTermInfosIndexDivisor() {
     ensureOpen();
     return termInfosIndexDivisor;
@@ -367,13 +582,26 @@ final class DirectoryReader extends Base
    * <p/>
    * @lucene.experimental
    */
-  @Override
   public IndexCommit getIndexCommit() throws IOException {
     ensureOpen();
     return new ReaderCommit(segmentInfos, directory);
   }
 
-  /** @see org.apache.lucene.index.IndexReader#listCommits */
+  /** Returns all commit points that exist in the Directory.
+   *  Normally, because the default is {@link
+   *  KeepOnlyLastCommitDeletionPolicy}, there would be only
+   *  one commit point.  But if you're using a custom {@link
+   *  IndexDeletionPolicy} then there could be many commits.
+   *  Once you have a given commit, you can open a reader on
+   *  it by calling {@link IndexReader#open(IndexCommit)}
+   *  There must be at least one commit in
+   *  the Directory, else this method throws {@link
+   *  IndexNotFoundException}.  Note that if a commit is in
+   *  progress while this method is running, that commit
+   *  may or may not be returned.
+   *  
+   *  @return a sorted list of {@link IndexCommit}s, from oldest 
+   *  to latest. */
   public static List<IndexCommit> listCommits(Directory dir) throws IOException {
     final String[] files = dir.listAll();
 
@@ -420,6 +648,53 @@ final class DirectoryReader extends Base
     return commits;
   }  
   
+  /**
+   * Reads version number from segments files. The version number is
+   * initialized with a timestamp and then increased by one for each change of
+   * the index.
+   * 
+   * @param directory where the index resides.
+   * @return version number.
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
+   */
+  public static long getCurrentVersion(Directory directory) throws CorruptIndexException, IOException {
+    return SegmentInfos.readCurrentVersion(directory);
+  }
+    
+  /**
+   * Reads commitUserData, previously passed to {@link
+   * IndexWriter#commit(Map)}, from current index
+   * segments file.  This will return null if {@link
+   * IndexWriter#commit(Map)} has never been called for
+   * this index.
+   * 
+   * @param directory where the index resides.
+   * @return commit userData.
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
+   *
+   * @see #getCommitUserData()
+   */
+  public static Map<String, String> getCommitUserData(Directory directory) throws CorruptIndexException, IOException {
+    return SegmentInfos.readCurrentUserData(directory);
+  }
+
+  /**
+   * Returns <code>true</code> if an index exists at the specified directory.
+   * @param  directory the directory to check for an index
+   * @return <code>true</code> if an index exists; <code>false</code> otherwise
+   * @throws IOException if there is a problem with accessing the index
+   */
+  public static boolean indexExists(Directory directory) throws IOException {
+    try {
+      new SegmentInfos().read(directory);
+      return true;
+    } catch (IOException ioe) {
+      return false;
+    }
+  }
+
   private static final class ReaderCommit extends IndexCommit {
     private String segmentsFileName;
     Collection<String> files;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocTermOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocTermOrds.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocTermOrds.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocTermOrds.java Mon Jan 30 23:34:03 2012
@@ -68,8 +68,6 @@ import java.util.Comparator;
  *
  * The RAM consumption of this class can be high!
  *
- * <p>NOTE: the provided reader must be an atomic reader
- *
  * @lucene.experimental
  */
 
@@ -149,19 +147,19 @@ public class DocTermOrds {
   }
 
   /** Inverts all terms */
-  public DocTermOrds(IndexReader reader, String field) throws IOException {
+  public DocTermOrds(AtomicReader reader, String field) throws IOException {
     this(reader, field, null, Integer.MAX_VALUE);
   }
 
   /** Inverts only terms starting w/ prefix */
-  public DocTermOrds(IndexReader reader, String field, BytesRef termPrefix) throws IOException {
+  public DocTermOrds(AtomicReader reader, String field, BytesRef termPrefix) throws IOException {
     this(reader, field, termPrefix, Integer.MAX_VALUE);
   }
 
   /** Inverts only terms starting w/ prefix, and only terms
    *  whose docFreq (not taking deletions into account) is
    *  <=  maxTermDocFreq */
-  public DocTermOrds(IndexReader reader, String field, BytesRef termPrefix, int maxTermDocFreq) throws IOException {
+  public DocTermOrds(AtomicReader reader, String field, BytesRef termPrefix, int maxTermDocFreq) throws IOException {
     this(reader, field, termPrefix, maxTermDocFreq, DEFAULT_INDEX_INTERVAL_BITS);
     uninvert(reader, termPrefix);
   }
@@ -170,7 +168,7 @@ public class DocTermOrds {
    *  whose docFreq (not taking deletions into account) is
    *  <=  maxTermDocFreq, with a custom indexing interval
    *  (default is every 128nd term). */
-  public DocTermOrds(IndexReader reader, String field, BytesRef termPrefix, int maxTermDocFreq, int indexIntervalBits) throws IOException {
+  public DocTermOrds(AtomicReader reader, String field, BytesRef termPrefix, int maxTermDocFreq, int indexIntervalBits) throws IOException {
     this(field, maxTermDocFreq, indexIntervalBits);
     uninvert(reader, termPrefix);
   }
@@ -196,7 +194,7 @@ public class DocTermOrds {
    *
    *  <p><b>NOTE</b>: you must pass the same reader that was
    *  used when creating this class */
-  public TermsEnum getOrdTermsEnum(IndexReader reader) throws IOException {
+  public TermsEnum getOrdTermsEnum(AtomicReader reader) throws IOException {
     if (termInstances == 0) {
       return null;
     }
@@ -226,7 +224,7 @@ public class DocTermOrds {
   }
 
   // Call this only once (if you subclass!)
-  protected void uninvert(final IndexReader reader, final BytesRef termPrefix) throws IOException {
+  protected void uninvert(final AtomicReader reader, final BytesRef termPrefix) throws IOException {
     //System.out.println("DTO uninvert field=" + field + " prefix=" + termPrefix);
     final long startTime = System.currentTimeMillis();
     prefix = termPrefix == null ? null : BytesRef.deepCopyOf(termPrefix);
@@ -644,12 +642,12 @@ public class DocTermOrds {
    * ord; in this case we "wrap" our own terms index
    * around it. */
   private final class OrdWrappedTermsEnum extends TermsEnum {
-    private final IndexReader reader;
+    private final AtomicReader reader;
     private final TermsEnum termsEnum;
     private BytesRef term;
     private long ord = -indexInterval-1;          // force "real" seek
     
-    public OrdWrappedTermsEnum(IndexReader reader) throws IOException {
+    public OrdWrappedTermsEnum(AtomicReader reader) throws IOException {
       this.reader = reader;
       assert indexedTermsArray != null;
       termsEnum = reader.fields().terms(field).iterator(null);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocValues.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocValues.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocValues.java Mon Jan 30 23:34:03 2012
@@ -33,7 +33,7 @@ import org.apache.lucene.util.packed.Pac
  * <li>via {@link #getSource()} providing RAM resident random access</li>
  * <li>via {@link #getDirectSource()} providing on disk random access</li>
  * </ul> {@link DocValues} are exposed via
- * {@link IndexReader#docValues(String)} on a per-segment basis. For best
+ * {@link AtomicReader#docValues(String)} on a per-segment basis. For best
  * performance {@link DocValues} should be consumed per-segment just like
  * IndexReader.
  * <p>

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java Mon Jan 30 23:34:03 2012
@@ -17,12 +17,10 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 
 import java.io.IOException;
-import java.util.Map;
 import java.util.Comparator;
 
 /**  A <code>FilterIndexReader</code> contains another IndexReader, which it
@@ -33,13 +31,8 @@ import java.util.Comparator;
  * contained index reader. Subclasses of <code>FilterIndexReader</code> may
  * further override some of these methods and may also provide additional
  * methods and fields.
- * <p><b>Note:</b> The default implementation of {@link FilterIndexReader#doOpenIfChanged}
- * throws {@link UnsupportedOperationException} (like the base class),
- * so it's not possible to reopen a <code>FilterIndexReader</code>.
- * To reopen, you have to first reopen the underlying reader
- * and wrap it again with the custom filter.
  */
-public class FilterIndexReader extends IndexReader {
+public class FilterIndexReader extends AtomicReader {
 
   /** Base class for filtering {@link Fields}
    *  implementations. */
@@ -279,25 +272,19 @@ public class FilterIndexReader extends I
     }
   }
 
-  protected IndexReader in;
+  protected AtomicReader in;
 
   /**
    * <p>Construct a FilterIndexReader based on the specified base reader.
    * <p>Note that base reader is closed if this FilterIndexReader is closed.</p>
    * @param in specified base reader.
    */
-  public FilterIndexReader(IndexReader in) {
+  public FilterIndexReader(AtomicReader in) {
     super();
     this.in = in;
   }
 
   @Override
-  public Directory directory() {
-    ensureOpen();
-    return in.directory();
-  }
-  
-  @Override
   public Bits getLiveDocs() {
     ensureOpen();
     return in.getLiveDocs();
@@ -346,43 +333,9 @@ public class FilterIndexReader extends I
   }
 
   @Override
-  public int docFreq(String field, BytesRef t) throws IOException {
-    ensureOpen();
-    return in.docFreq(field, t);
-  }
-  
-  @Override
   protected void doClose() throws IOException {
     in.close();
   }
-
-  @Override
-  public long getVersion() {
-    ensureOpen();
-    return in.getVersion();
-  }
-
-  @Override
-  public boolean isCurrent() throws CorruptIndexException, IOException {
-    ensureOpen();
-    return in.isCurrent();
-  }
-  
-  @Override
-  public IndexReader[] getSequentialSubReaders() {
-    return in.getSequentialSubReaders();
-  }
-  
-  @Override
-  public ReaderContext getTopReaderContext() {
-    ensureOpen();
-    return in.getTopReaderContext();
-  }
-
-  @Override
-  public Map<String, String> getCommitUserData() { 
-    return in.getCommitUserData();
-  }
   
   @Override
   public Fields fields() throws IOException {
@@ -410,7 +363,7 @@ public class FilterIndexReader extends I
 
   @Override
   public String toString() {
-    final StringBuilder buffer = new StringBuilder("FilterReader(");
+    final StringBuilder buffer = new StringBuilder("FilterIndexReader(");
     buffer.append(in);
     buffer.append(')');
     return buffer.toString();
@@ -427,14 +380,4 @@ public class FilterIndexReader extends I
     ensureOpen();
     return in.normValues(field);
   }
-
-  @Override
-  public IndexCommit getIndexCommit() throws IOException {
-    return in.getIndexCommit();
-  }
-
-  @Override
-  public int getTermInfosIndexDivisor() {
-    return in.getTermInfosIndexDivisor();
-  }  
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java Mon Jan 30 23:34:03 2012
@@ -21,8 +21,6 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -38,26 +36,32 @@ import org.apache.lucene.util.ReaderUtil
  index.  Search of an index is done entirely through this abstract interface,
  so that any subclass which implements it is searchable.
 
- <p> Concrete subclasses of IndexReader are usually constructed with a call to
- one of the static <code>open()</code> methods, e.g. {@link
- #open(Directory)}.
+ <p>There are two different types of IndexReaders:
+ <ul>
+  <li>{@link AtomicReader}: These indexes do not consist of several sub-readers,
+  they are atomic. They support retrieval of stored fields, doc values, terms,
+  and postings.
+  <li>{@link CompositeReader}: Instances (like {@link DirectoryReader})
+  of this reader can only
+  be used to get stored fields from the underlying AtomicReaders,
+  but it is not possible to directly retrieve postings. To do that, get
+  the sub-readers via {@link CompositeReader#getSequentialSubReaders}.
+  Alternatively, you can mimic an {@link AtomicReader} (with a serious slowdown),
+  by wrapping composite readers with {@link SlowCompositeReaderWrapper}.
+ </ul>
+ 
+ <p>IndexReader instances for indexes on disk are usually constructed
+ with a call to one of the static <code>DirectoryReader,open()</code> methods,
+ e.g. {@link DirectoryReader#open(Directory)}. {@link DirectoryReader} implements
+ the {@link CompositeReader} interface, it is not possible to directly get postings.
 
  <p> For efficiency, in this API documents are often referred to via
  <i>document numbers</i>, non-negative integers which each name a unique
- document in the index.  These document numbers are ephemeral--they may change
+ document in the index.  These document numbers are ephemeral -- they may change
  as documents are added to and deleted from an index.  Clients should thus not
  rely on a given document having the same number between sessions.
 
  <p>
- <b>NOTE</b>: for backwards API compatibility, several methods are not listed 
- as abstract, but have no useful implementations in this base class and 
- instead always throw UnsupportedOperationException.  Subclasses are 
- strongly encouraged to override these methods, but in many cases may not 
- need to.
- </p>
-
- <p>
-
  <a name="thread-safety"></a><p><b>NOTE</b>: {@link
  IndexReader} instances are completely thread
  safe, meaning multiple threads can call any of its methods,
@@ -67,7 +71,13 @@ import org.apache.lucene.util.ReaderUtil
  (non-Lucene) objects instead.
 */
 public abstract class IndexReader implements Closeable {
-
+  
+  IndexReader() {
+    if (!(this instanceof CompositeReader || this instanceof AtomicReader))
+      throw new Error("This class should never be directly extended, subclass AtomicReader or CompositeReader instead!");
+    refCount.set(1);
+  }
+  
   /**
    * A custom listener that's invoked when the IndexReader
    * is closed.
@@ -110,8 +120,6 @@ public abstract class IndexReader implem
   
   private final AtomicInteger refCount = new AtomicInteger();
 
-  static int DEFAULT_TERMS_INDEX_DIVISOR = 1;
-
   /** Expert: returns the current refCount for this reader */
   public final int getRefCount() {
     // NOTE: don't ensureOpen, so that callers can see
@@ -172,23 +180,6 @@ public abstract class IndexReader implem
     return false;
   }
 
-  /** {@inheritDoc} */
-  @Override
-  public String toString() {
-    final StringBuilder buffer = new StringBuilder();
-    buffer.append(getClass().getSimpleName());
-    buffer.append('(');
-    final IndexReader[] subReaders = getSequentialSubReaders();
-    if ((subReaders != null) && (subReaders.length > 0)) {
-      buffer.append(subReaders[0]);
-      for (int i = 1; i < subReaders.length; ++i) {
-        buffer.append(" ").append(subReaders[i]);
-      }
-    }
-    buffer.append(')');
-    return buffer.toString();
-  }
-
   /**
    * Expert: decreases the refCount of this IndexReader
    * instance.  If the refCount drops to 0, then this
@@ -219,10 +210,6 @@ public abstract class IndexReader implem
     }
   }
   
-  protected IndexReader() { 
-    refCount.set(1);
-  }
-  
   /**
    * @throws AlreadyClosedException if this IndexReader is closed
    */
@@ -237,9 +224,11 @@ public abstract class IndexReader implem
    * @param directory the index directory
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
+   * @deprecated Use {@link DirectoryReader#open(Directory)}
    */
-  public static IndexReader open(final Directory directory) throws CorruptIndexException, IOException {
-    return DirectoryReader.open(directory, null, DEFAULT_TERMS_INDEX_DIVISOR);
+  @Deprecated
+  public static DirectoryReader open(final Directory directory) throws CorruptIndexException, IOException {
+    return DirectoryReader.open(directory);
   }
   
   /** Expert: Returns a IndexReader reading the index in the given
@@ -257,9 +246,11 @@ public abstract class IndexReader implem
    *  to -1 to skip loading the terms index entirely.
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
+   * @deprecated Use {@link DirectoryReader#open(Directory,int)}
    */
-  public static IndexReader open(final Directory directory, int termInfosIndexDivisor) throws CorruptIndexException, IOException {
-    return DirectoryReader.open(directory, null, termInfosIndexDivisor);
+  @Deprecated
+  public static DirectoryReader open(final Directory directory, int termInfosIndexDivisor) throws CorruptIndexException, IOException {
+    return DirectoryReader.open(directory, termInfosIndexDivisor);
   }
   
   /**
@@ -277,12 +268,14 @@ public abstract class IndexReader implem
    * @throws CorruptIndexException
    * @throws IOException if there is a low-level IO error
    *
-   * @see #openIfChanged(IndexReader,IndexWriter,boolean)
+   * @see DirectoryReader#openIfChanged(DirectoryReader,IndexWriter,boolean)
    *
    * @lucene.experimental
+   * @deprecated Use {@link DirectoryReader#open(IndexWriter,boolean)}
    */
-  public static IndexReader open(final IndexWriter writer, boolean applyAllDeletes) throws CorruptIndexException, IOException {
-    return writer.getReader(applyAllDeletes);
+  @Deprecated
+  public static DirectoryReader open(final IndexWriter writer, boolean applyAllDeletes) throws CorruptIndexException, IOException {
+    return DirectoryReader.open(writer, applyAllDeletes);
   }
 
   /** Expert: returns an IndexReader reading the index in the given
@@ -290,9 +283,11 @@ public abstract class IndexReader implem
    * @param commit the commit point to open
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
+   * @deprecated Use {@link DirectoryReader#open(IndexCommit)}
    */
-  public static IndexReader open(final IndexCommit commit) throws CorruptIndexException, IOException {
-    return DirectoryReader.open(commit.getDirectory(), commit, DEFAULT_TERMS_INDEX_DIVISOR);
+  @Deprecated
+  public static DirectoryReader open(final IndexCommit commit) throws CorruptIndexException, IOException {
+    return DirectoryReader.open(commit);
   }
 
 
@@ -311,240 +306,11 @@ public abstract class IndexReader implem
    *  to -1 to skip loading the terms index entirely.
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
+   * @deprecated Use {@link DirectoryReader#open(IndexCommit,int)}
    */
-  public static IndexReader open(final IndexCommit commit, int termInfosIndexDivisor) throws CorruptIndexException, IOException {
-    return DirectoryReader.open(commit.getDirectory(), commit, termInfosIndexDivisor);
-  }
-
-  /**
-   * If the index has changed since the provided reader was
-   * opened, open and return a new reader; else, return
-   * null.  The new reader, if not null, will be the same
-   * type of reader as the previous one, ie an NRT reader
-   * will open a new NRT reader, a MultiReader will open a
-   * new MultiReader,  etc.
-   *
-   * <p>This method is typically far less costly than opening a
-   * fully new <code>IndexReader</code> as it shares
-   * resources (for example sub-readers) with the provided
-   * <code>IndexReader</code>, when possible.
-   *
-   * <p>The provided reader is not closed (you are responsible
-   * for doing so); if a new reader is returned you also
-   * must eventually close it.  Be sure to never close a
-   * reader while other threads are still using it; see
-   * {@link SearcherManager} to simplify managing this.
-   *
-   * @throws CorruptIndexException if the index is corrupt
-   * @throws IOException if there is a low-level IO error
-   * @return null if there are no changes; else, a new
-   * IndexReader instance which you must eventually close
-   */  
-  public static IndexReader openIfChanged(IndexReader oldReader) throws IOException {
-    final IndexReader newReader = oldReader.doOpenIfChanged();
-    assert newReader != oldReader;
-    return newReader;
-  }
-
-  /**
-   * If the IndexCommit differs from what the
-   * provided reader is searching, open and return a new
-   * reader; else, return null.
-   *
-   * @see #openIfChanged(IndexReader)
-   */
-  public static IndexReader openIfChanged(IndexReader oldReader, IndexCommit commit) throws IOException {
-    final IndexReader newReader = oldReader.doOpenIfChanged(commit);
-    assert newReader != oldReader;
-    return newReader;
-  }
-
-  /**
-   * Expert: If there changes (committed or not) in the
-   * {@link IndexWriter} versus what the provided reader is
-   * searching, then open and return a new
-   * IndexReader searching both committed and uncommitted
-   * changes from the writer; else, return null (though, the
-   * current implementation never returns null).
-   *
-   * <p>This provides "near real-time" searching, in that
-   * changes made during an {@link IndexWriter} session can be
-   * quickly made available for searching without closing
-   * the writer nor calling {@link IndexWriter#commit}.
-   *
-   * <p>It's <i>near</i> real-time because there is no hard
-   * guarantee on how quickly you can get a new reader after
-   * making changes with IndexWriter.  You'll have to
-   * experiment in your situation to determine if it's
-   * fast enough.  As this is a new and experimental
-   * feature, please report back on your findings so we can
-   * learn, improve and iterate.</p>
-   *
-   * <p>The very first time this method is called, this
-   * writer instance will make every effort to pool the
-   * readers that it opens for doing merges, applying
-   * deletes, etc.  This means additional resources (RAM,
-   * file descriptors, CPU time) will be consumed.</p>
-   *
-   * <p>For lower latency on reopening a reader, you should
-   * call {@link IndexWriterConfig#setMergedSegmentWarmer} to
-   * pre-warm a newly merged segment before it's committed
-   * to the index.  This is important for minimizing
-   * index-to-search delay after a large merge.  </p>
-   *
-   * <p>If an addIndexes* call is running in another thread,
-   * then this reader will only search those segments from
-   * the foreign index that have been successfully copied
-   * over, so far.</p>
-   *
-   * <p><b>NOTE</b>: Once the writer is closed, any
-   * outstanding readers may continue to be used.  However,
-   * if you attempt to reopen any of those readers, you'll
-   * hit an {@link AlreadyClosedException}.</p>
-   *
-   * @return IndexReader that covers entire index plus all
-   * changes made so far by this IndexWriter instance, or
-   * null if there are no new changes
-   *
-   * @param writer The IndexWriter to open from
-   *
-   * @param applyAllDeletes If true, all buffered deletes will
-   * be applied (made visible) in the returned reader.  If
-   * false, the deletes are not applied but remain buffered
-   * (in IndexWriter) so that they will be applied in the
-   * future.  Applying deletes can be costly, so if your app
-   * can tolerate deleted documents being returned you might
-   * gain some performance by passing false.
-   *
-   * @throws IOException
-   *
-   * @lucene.experimental
-   */
-  public static IndexReader openIfChanged(IndexReader oldReader, IndexWriter writer, boolean applyAllDeletes) throws IOException {
-    final IndexReader newReader = oldReader.doOpenIfChanged(writer, applyAllDeletes);
-    assert newReader != oldReader;
-    return newReader;
-  }
-
-  /**
-   * If the index has changed since it was opened, open and return a new reader;
-   * else, return {@code null}.
-   * 
-   * @see #openIfChanged(IndexReader)
-   */
-  protected IndexReader doOpenIfChanged() throws CorruptIndexException, IOException {
-    throw new UnsupportedOperationException("This reader does not support reopen().");
-  }
-  
-  /**
-   * If the index has changed since it was opened, open and return a new reader;
-   * else, return {@code null}.
-   * 
-   * @see #openIfChanged(IndexReader, IndexCommit)
-   */
-  protected IndexReader doOpenIfChanged(final IndexCommit commit) throws CorruptIndexException, IOException {
-    throw new UnsupportedOperationException("This reader does not support reopen(IndexCommit).");
-  }
-
-  /**
-   * If the index has changed since it was opened, open and return a new reader;
-   * else, return {@code null}.
-   * 
-   * @see #openIfChanged(IndexReader, IndexWriter, boolean)
-   */
-  protected IndexReader doOpenIfChanged(IndexWriter writer, boolean applyAllDeletes) throws CorruptIndexException, IOException {
-    return writer.getReader(applyAllDeletes);
-  }
-
-  /** 
-   * Returns the directory associated with this index.  The Default 
-   * implementation returns the directory specified by subclasses when 
-   * delegating to the IndexReader(Directory) constructor, or throws an 
-   * UnsupportedOperationException if one was not specified.
-   * @throws UnsupportedOperationException if no directory
-   */
-  public Directory directory() {
-    ensureOpen();
-    throw new UnsupportedOperationException("This reader does not support this method.");  
-  }
-
-  /**
-   * Reads commitUserData, previously passed to {@link
-   * IndexWriter#commit(Map)}, from current index
-   * segments file.  This will return null if {@link
-   * IndexWriter#commit(Map)} has never been called for
-   * this index.
-   * 
-   * @param directory where the index resides.
-   * @return commit userData.
-   * @throws CorruptIndexException if the index is corrupt
-   * @throws IOException if there is a low-level IO error
-   *
-   * @see #getCommitUserData()
-   */
-  public static Map<String, String> getCommitUserData(Directory directory) throws CorruptIndexException, IOException {
-    return SegmentInfos.readCurrentUserData(directory);
-  }
-
-  /**
-   * Version number when this IndexReader was opened. Not
-   * implemented in the IndexReader base class.
-   *
-   * <p>If this reader is based on a Directory (ie, was
-   * created by calling {@link #open}, or {@link #openIfChanged} on
-   * a reader based on a Directory), then this method
-   * returns the version recorded in the commit that the
-   * reader opened.  This version is advanced every time
-   * a change is made with {@link IndexWriter}.</p>
-   *
-   * @throws UnsupportedOperationException unless overridden in subclass
-   */
-  public long getVersion() {
-    throw new UnsupportedOperationException("This reader does not support this method.");
-  }
-
-  /**
-   * Retrieve the String userData optionally passed to
-   * IndexWriter#commit.  This will return null if {@link
-   * IndexWriter#commit(Map)} has never been called for
-   * this index.
-   *
-   * @see #getCommitUserData(Directory)
-   */
-  public Map<String,String> getCommitUserData() {
-    throw new UnsupportedOperationException("This reader does not support this method.");
-  }
-
-
-  /**
-   * Check whether any new changes have occurred to the
-   * index since this reader was opened.
-   *
-   * <p>If this reader is based on a Directory (ie, was
-   * created by calling {@link #open}, or {@link #openIfChanged} on
-   * a reader based on a Directory), then this method checks
-   * if any further commits (see {@link IndexWriter#commit}
-   * have occurred in that directory).</p>
-   *
-   * <p>If instead this reader is a near real-time reader
-   * (ie, obtained by a call to {@link
-   * IndexWriter#getReader}, or by calling {@link #openIfChanged}
-   * on a near real-time reader), then this method checks if
-   * either a new commit has occurred, or any new
-   * uncommitted changes have taken place via the writer.
-   * Note that even if the writer has only performed
-   * merging, this method will still return false.</p>
-   *
-   * <p>In any event, if this returns false, you should call
-   * {@link #openIfChanged} to get a new reader that sees the
-   * changes.</p>
-   *
-   * @throws CorruptIndexException if the index is corrupt
-   * @throws IOException           if there is a low-level IO error
-   * @throws UnsupportedOperationException unless overridden in subclass
-   */
-  public boolean isCurrent() throws CorruptIndexException, IOException {
-    throw new UnsupportedOperationException("This reader does not support this method.");
+  @Deprecated
+  public static DirectoryReader open(final IndexCommit commit, int termInfosIndexDivisor) throws CorruptIndexException, IOException {
+    return DirectoryReader.open(commit, termInfosIndexDivisor);
   }
 
   /** Retrieve term vectors for this document, or null if
@@ -567,21 +333,6 @@ public abstract class IndexReader implem
     return vectors.terms(field);
   }
 
-  /**
-   * Returns <code>true</code> if an index exists at the specified directory.
-   * @param  directory the directory to check for an index
-   * @return <code>true</code> if an index exists; <code>false</code> otherwise
-   * @throws IOException if there is a problem with accessing the index
-   */
-  public static boolean indexExists(Directory directory) throws IOException {
-    try {
-      new SegmentInfos().read(directory);
-      return true;
-    } catch (IOException ioe) {
-      return false;
-    }
-  }
-
   /** Returns the number of documents in this index. */
   public abstract int numDocs();
 
@@ -646,166 +397,6 @@ public abstract class IndexReader implem
   /** Returns true if any documents have been deleted */
   public abstract boolean hasDeletions();
 
-  /** Returns true if there are norms stored for this field. */
-  public boolean hasNorms(String field) throws IOException {
-    // backward compatible implementation.
-    // SegmentReader has an efficient implementation.
-    ensureOpen();
-    return normValues(field) != null;
-  }
-
-  /**
-   * Returns {@link Fields} for this reader.
-   * This method may return null if the reader has no
-   * postings.
-   *
-   * <p><b>NOTE</b>: if this is a multi reader ({@link
-   * #getSequentialSubReaders} is not null) then this
-   * method will throw UnsupportedOperationException.  If
-   * you really need a {@link Fields} for such a reader,
-   * use {@link MultiFields#getFields}.  However, for
-   * performance reasons, it's best to get all sub-readers
-   * using {@link ReaderUtil#gatherSubReaders} and iterate
-   * through them yourself. */
-  public abstract Fields fields() throws IOException;
-  
-  public final int docFreq(Term term) throws IOException {
-    return docFreq(term.field(), term.bytes());
-  }
-
-  /** Returns the number of documents containing the term
-   * <code>t</code>.  This method returns 0 if the term or
-   * field does not exists.  This method does not take into
-   * account deleted documents that have not yet been merged
-   * away. */
-  public int docFreq(String field, BytesRef term) throws IOException {
-    final Fields fields = fields();
-    if (fields == null) {
-      return 0;
-    }
-    final Terms terms = fields.terms(field);
-    if (terms == null) {
-      return 0;
-    }
-    final TermsEnum termsEnum = terms.iterator(null);
-    if (termsEnum.seekExact(term, true)) {
-      return termsEnum.docFreq();
-    } else {
-      return 0;
-    }
-  }
-
-  /** Returns the number of documents containing the term
-   * <code>t</code>.  This method returns 0 if the term or
-   * field does not exists.  This method does not take into
-   * account deleted documents that have not yet been merged
-   * away. */
-  public final long totalTermFreq(String field, BytesRef term) throws IOException {
-    final Fields fields = fields();
-    if (fields == null) {
-      return 0;
-    }
-    final Terms terms = fields.terms(field);
-    if (terms == null) {
-      return 0;
-    }
-    final TermsEnum termsEnum = terms.iterator(null);
-    if (termsEnum.seekExact(term, true)) {
-      return termsEnum.totalTermFreq();
-    } else {
-      return 0;
-    }
-  }
-
-  /** This may return null if the field does not exist.*/
-  public final Terms terms(String field) throws IOException {
-    final Fields fields = fields();
-    if (fields == null) {
-      return null;
-    }
-    return fields.terms(field);
-  }
-
-  /** Returns {@link DocsEnum} for the specified field &
-   *  term.  This may return null, if either the field or
-   *  term does not exist. */
-  public final DocsEnum termDocsEnum(Bits liveDocs, String field, BytesRef term, boolean needsFreqs) throws IOException {
-    assert field != null;
-    assert term != null;
-    final Fields fields = fields();
-    if (fields != null) {
-      final Terms terms = fields.terms(field);
-      if (terms != null) {
-        final TermsEnum termsEnum = terms.iterator(null);
-        if (termsEnum.seekExact(term, true)) {
-          return termsEnum.docs(liveDocs, null, needsFreqs);
-        }
-      }
-    }
-    return null;
-  }
-
-  /** Returns {@link DocsAndPositionsEnum} for the specified
-   *  field & term.  This may return null, if either the
-   *  field or term does not exist, or needsOffsets is
-   *  true but offsets were not indexed for this field. */
-  public final DocsAndPositionsEnum termPositionsEnum(Bits liveDocs, String field, BytesRef term, boolean needsOffsets) throws IOException {
-    assert field != null;
-    assert term != null;
-    final Fields fields = fields();
-    if (fields != null) {
-      final Terms terms = fields.terms(field);
-      if (terms != null) {
-        final TermsEnum termsEnum = terms.iterator(null);
-        if (termsEnum.seekExact(term, true)) {
-          return termsEnum.docsAndPositions(liveDocs, null, needsOffsets);
-        }
-      }
-    }
-    return null;
-  }
-  
-  /**
-   * Returns {@link DocsEnum} for the specified field and
-   * {@link TermState}. This may return null, if either the field or the term
-   * does not exists or the {@link TermState} is invalid for the underlying
-   * implementation.*/
-  public final DocsEnum termDocsEnum(Bits liveDocs, String field, BytesRef term, TermState state, boolean needsFreqs) throws IOException {
-    assert state != null;
-    assert field != null;
-    final Fields fields = fields();
-    if (fields != null) {
-      final Terms terms = fields.terms(field);
-      if (terms != null) {
-        final TermsEnum termsEnum = terms.iterator(null);
-        termsEnum.seekExact(term, state);
-        return termsEnum.docs(liveDocs, null, needsFreqs);
-      }
-    }
-    return null;
-  }
-  
-  /**
-   * Returns {@link DocsAndPositionsEnum} for the specified field and
-   * {@link TermState}. This may return null, if either the field or the term
-   * does not exists, the {@link TermState} is invalid for the underlying
-   * implementation, or needsOffsets is true but offsets
-   * were not indexed for this field. */
-  public final DocsAndPositionsEnum termPositionsEnum(Bits liveDocs, String field, BytesRef term, TermState state, boolean needsOffsets) throws IOException {
-    assert state != null;
-    assert field != null;
-    final Fields fields = fields();
-    if (fields != null) {
-      final Terms terms = fields.terms(field);
-      if (terms != null) {
-        final TermsEnum termsEnum = terms.iterator(null);
-        termsEnum.seekExact(term, state);
-        return termsEnum.docsAndPositions(liveDocs, null, needsOffsets);
-      }
-    }
-    return null;
-  }
-
   /**
    * Closes files associated with this index.
    * Also saves any new deletions to disk.
@@ -823,76 +414,12 @@ public abstract class IndexReader implem
   protected abstract void doClose() throws IOException;
 
   /**
-   * Get the {@link FieldInfos} describing all fields in
-   * this reader.  NOTE: do not make any changes to the
-   * returned FieldInfos!
-   *
-   * @lucene.experimental
-   */
-  public abstract FieldInfos getFieldInfos();
-
-  /** Returns the {@link Bits} representing live (not
-   *  deleted) docs.  A set bit indicates the doc ID has not
-   *  been deleted.  If this method returns null it means
-   *  there are no deleted documents (all documents are
-   *  live).
-   *
-   *  The returned instance has been safely published for
-   *  use by multiple threads without additional
-   *  synchronization.
-   * @lucene.experimental */
-  public abstract Bits getLiveDocs();
-
-  /**
-   * Expert: return the IndexCommit that this reader has
-   * opened.  This method is only implemented by those
-   * readers that correspond to a Directory with its own
-   * segments_N file.
-   *
-   * @lucene.experimental
-   */
-  public IndexCommit getIndexCommit() throws IOException {
-    throw new UnsupportedOperationException("This reader does not support this method.");
-  }
-  
-  /** Returns all commit points that exist in the Directory.
-   *  Normally, because the default is {@link
-   *  KeepOnlyLastCommitDeletionPolicy}, there would be only
-   *  one commit point.  But if you're using a custom {@link
-   *  IndexDeletionPolicy} then there could be many commits.
-   *  Once you have a given commit, you can open a reader on
-   *  it by calling {@link IndexReader#open(IndexCommit)}
-   *  There must be at least one commit in
-   *  the Directory, else this method throws {@link
-   *  IndexNotFoundException}.  Note that if a commit is in
-   *  progress while this method is running, that commit
-   *  may or may not be returned.
-   *  
-   *  @return a sorted list of {@link IndexCommit}s, from oldest 
-   *  to latest. */
-  public static List<IndexCommit> listCommits(Directory dir) throws IOException {
-    return DirectoryReader.listCommits(dir);
-  }
-
-  /** Expert: returns the sequential sub readers that this
-   *  reader is logically composed of. If this reader is not composed
-   *  of sequential child readers, it should return null.
-   *  If this method returns an empty array, that means this
-   *  reader is a null reader (for example a MultiReader
-   *  that has no sub readers).
-   */
-  public IndexReader[] getSequentialSubReaders() {
-    ensureOpen();
-    return null;
-  }
-  
-  /**
-   * Expert: Returns a the root {@link ReaderContext} for this
+   * Expert: Returns a the root {@link IndexReaderContext} for this
    * {@link IndexReader}'s sub-reader tree. Iff this reader is composed of sub
    * readers ,ie. this reader being a composite reader, this method returns a
    * {@link CompositeReaderContext} holding the reader's direct children as well as a
    * view of the reader tree's atomic leaf contexts. All sub-
-   * {@link ReaderContext} instances referenced from this readers top-level
+   * {@link IndexReaderContext} instances referenced from this readers top-level
    * context are private to this reader and are not shared with another context
    * tree. For example, IndexSearcher uses this API to drive searching by one
    * atomic leaf reader at a time. If this reader is not composed of child
@@ -905,7 +432,7 @@ public abstract class IndexReader implem
    * 
    * @lucene.experimental
    */
-  public abstract ReaderContext getTopReaderContext();
+  public abstract IndexReaderContext getTopReaderContext();
 
   /** Expert: Returns a key for this IndexReader, so FieldCache/CachingWrapperFilter can find
    * it again.
@@ -924,191 +451,15 @@ public abstract class IndexReader implem
     // on close
     return this;
   }
-
-  /** Returns the number of unique terms (across all fields)
-   *  in this reader.
-   *
-   *  @return number of unique terms or -1 if this count
-   *  cannot be easily determined (eg Multi*Readers).
-   *  Instead, you should call {@link
-   *  #getSequentialSubReaders} and ask each sub reader for
-   *  its unique term count. */
-  public final long getUniqueTermCount() throws IOException {
-    if (!getTopReaderContext().isAtomic) {
-      return -1;
-    }
-    final Fields fields = fields();
-    if (fields == null) {
-      return 0;
-    }
-    return fields.getUniqueTermCount();
-  }
-
-  /** For IndexReader implementations that use
-   *  TermInfosReader to read terms, this returns the
-   *  current indexDivisor as specified when the reader was
-   *  opened.
-   */
-  public int getTermInfosIndexDivisor() {
-    throw new UnsupportedOperationException("This reader does not support this method.");
-  }
   
-  /**
-   * Returns {@link DocValues} for this field.
-   * This method may return null if the reader has no per-document
-   * values stored.
-   *
-   * <p><b>NOTE</b>: if this is a multi reader ({@link
-   * #getSequentialSubReaders} is not null) then this
-   * method will throw UnsupportedOperationException.  If
-   * you really need {@link DocValues} for such a reader,
-   * use {@link MultiDocValues#getDocValues(IndexReader,String)}.  However, for
-   * performance reasons, it's best to get all sub-readers
-   * using {@link ReaderUtil#gatherSubReaders} and iterate
-   * through them yourself. */
-  public abstract DocValues docValues(String field) throws IOException;
-  
-  public abstract DocValues normValues(String field) throws IOException;
-
-  private volatile Fields fields;
-
-  /** @lucene.internal */
-  void storeFields(Fields fields) {
-    ensureOpen();
-    this.fields = fields;
-  }
-
-  /** @lucene.internal */
-  Fields retrieveFields() {
-    ensureOpen();
-    return fields;
-  }
-  
-  /**
-   * A struct like class that represents a hierarchical relationship between
-   * {@link IndexReader} instances. 
-   * @lucene.experimental
-   */
-  public static abstract class ReaderContext {
-    /** The reader context for this reader's immediate parent, or null if none */
-    public final ReaderContext parent;
-    /** The actual reader */
-    public final IndexReader reader;
-    /** <code>true</code> iff the reader is an atomic reader */
-    public final boolean isAtomic;
-    /** <code>true</code> if this context struct represents the top level reader within the hierarchical context */
-    public final boolean isTopLevel;
-    /** the doc base for this reader in the parent, <tt>0</tt> if parent is null */
-    public final int docBaseInParent;
-    /** the ord for this reader in the parent, <tt>0</tt> if parent is null */
-    public final int ordInParent;
-    
-    ReaderContext(ReaderContext parent, IndexReader reader,
-        boolean isAtomic, int ordInParent, int docBaseInParent) {
-      this.parent = parent;
-      this.reader = reader;
-      this.isAtomic = isAtomic;
-      this.docBaseInParent = docBaseInParent;
-      this.ordInParent = ordInParent;
-      this.isTopLevel = parent==null;
-    }
-    
-    /**
-     * Returns the context's leaves if this context is a top-level context
-     * otherwise <code>null</code>.
-     * <p>
-     * Note: this is convenience method since leaves can always be obtained by
-     * walking the context tree.
-     */
-    public AtomicReaderContext[] leaves() {
-      return null;
-    }
-    
-    /**
-     * Returns the context's children iff this context is a composite context
-     * otherwise <code>null</code>.
-     * <p>
-     * Note: this method is a convenience method to prevent
-     * <code>instanceof</code> checks and type-casts to
-     * {@link CompositeReaderContext}.
-     */
-    public ReaderContext[] children() {
-      return null;
-    }
+  public final int docFreq(Term term) throws IOException {
+    return docFreq(term.field(), term.bytes());
   }
-  
-  /**
-   * {@link ReaderContext} for composite {@link IndexReader} instance.
-   * @lucene.experimental
-   */
-  public static final class CompositeReaderContext extends ReaderContext {
-    /** the composite readers immediate children */
-    public final ReaderContext[] children;
-    /** the composite readers leaf reader contexts if this is the top level reader in this context */
-    public final AtomicReaderContext[] leaves;
-
-    /**
-     * Creates a {@link CompositeReaderContext} for intermediate readers that aren't
-     * not top-level readers in the current context
-     */
-    public CompositeReaderContext(ReaderContext parent, IndexReader reader,
-        int ordInParent, int docbaseInParent, ReaderContext[] children) {
-      this(parent, reader, ordInParent, docbaseInParent, children, null);
-    }
-    
-    /**
-     * Creates a {@link CompositeReaderContext} for top-level readers with parent set to <code>null</code>
-     */
-    public CompositeReaderContext(IndexReader reader, ReaderContext[] children, AtomicReaderContext[] leaves) {
-      this(null, reader, 0, 0, children, leaves);
-    }
-    
-    private CompositeReaderContext(ReaderContext parent, IndexReader reader,
-        int ordInParent, int docbaseInParent, ReaderContext[] children,
-        AtomicReaderContext[] leaves) {
-      super(parent, reader, false, ordInParent, docbaseInParent);
-      this.children = children;
-      this.leaves = leaves;
-    }
 
-    @Override
-    public AtomicReaderContext[] leaves() {
-      return leaves;
-    }
-    
-    
-    @Override
-    public ReaderContext[] children() {
-      return children;
-    }
-  }
-  
-  /**
-   * {@link ReaderContext} for atomic {@link IndexReader} instances
-   * @lucene.experimental
-   */
-  public static final class AtomicReaderContext extends ReaderContext {
-    /** The readers ord in the top-level's leaves array */
-    public final int ord;
-    /** The readers absolute doc base */
-    public final int docBase;
-    /**
-     * Creates a new {@link AtomicReaderContext} 
-     */    
-    public AtomicReaderContext(ReaderContext parent, IndexReader reader,
-        int ord, int docBase, int leafOrd, int leafDocBase) {
-      super(parent, reader, true, ord, docBase);
-      assert reader.getSequentialSubReaders() == null : "Atomic readers must not have subreaders";
-      this.ord = leafOrd;
-      this.docBase = leafDocBase;
-    }
-    
-    /**
-     * Creates a new {@link AtomicReaderContext} for a atomic reader without an immediate
-     * parent.
-     */
-    public AtomicReaderContext(IndexReader atomicReader) {
-      this(null, atomicReader, 0, 0, 0, 0);
-    }
-  }
+  /** Returns the number of documents containing the term
+   * <code>t</code>.  This method returns 0 if the term or
+   * field does not exists.  This method does not take into
+   * account deleted documents that have not yet been merged
+   * away. */
+  public abstract int docFreq(String field, BytesRef term) throws IOException;
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java Mon Jan 30 23:34:03 2012
@@ -134,7 +134,7 @@ public final class IndexUpgrader {
   }
   
   public void upgrade() throws IOException {
-    if (!IndexReader.indexExists(dir)) {
+    if (!DirectoryReader.indexExists(dir)) {
       throw new IndexNotFoundException(dir.toString());
     }
   

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java Mon Jan 30 23:34:03 2012
@@ -264,7 +264,7 @@ public class IndexWriter implements Clos
   // The PayloadProcessorProvider to use when segments are merged
   private PayloadProcessorProvider payloadProcessorProvider;
 
-  IndexReader getReader() throws IOException {
+  DirectoryReader getReader() throws IOException {
     return getReader(true);
   }
 
@@ -327,7 +327,7 @@ public class IndexWriter implements Clos
    *
    * @throws IOException
    */
-  IndexReader getReader(boolean applyAllDeletes) throws IOException {
+  DirectoryReader getReader(boolean applyAllDeletes) throws IOException {
     ensureOpen();
 
     final long tStart = System.currentTimeMillis();
@@ -339,7 +339,7 @@ public class IndexWriter implements Clos
     // obtained during this flush are pooled, the first time
     // this method is called:
     poolReaders = true;
-    final IndexReader r;
+    final DirectoryReader r;
     doBeforeFlush();
     boolean anySegmentFlushed = false;
     /*
@@ -871,7 +871,7 @@ public class IndexWriter implements Clos
         create = false;
       } else {
         // CREATE_OR_APPEND - create only if an index does not exist
-        create = !IndexReader.indexExists(directory);
+        create = !DirectoryReader.indexExists(directory);
       }
 
       // If index is too old, reading the segments will throw
@@ -2631,7 +2631,7 @@ public class IndexWriter implements Clos
    *  @param commitUserData Opaque Map (String->String)
    *  that's recorded into the segments file in the index,
    *  and retrievable by {@link
-   *  IndexReader#getCommitUserData}.  Note that when
+   *  DirectoryReader#getCommitUserData}.  Note that when
    *  IndexWriter commits itself during {@link #close}, the
    *  commitUserData is unchanged (just carried over from
    *  the prior commit).  If this is null then the previous
@@ -3954,7 +3954,7 @@ public class IndexWriter implements Clos
    * <p><b>NOTE</b>: warm is called before any deletes have
    * been carried over to the merged segment. */
   public static abstract class IndexReaderWarmer {
-    public abstract void warm(IndexReader reader) throws IOException;
+    public abstract void warm(AtomicReader reader) throws IOException;
   }
 
   private void handleOOM(OutOfMemoryError oom, String location) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java Mon Jan 30 23:34:03 2012
@@ -90,7 +90,7 @@ public final class IndexWriterConfig imp
   public final static boolean DEFAULT_READER_POOLING = false;
 
   /** Default value is 1. Change using {@link #setReaderTermsIndexDivisor(int)}. */
-  public static final int DEFAULT_READER_TERMS_INDEX_DIVISOR = IndexReader.DEFAULT_TERMS_INDEX_DIVISOR;
+  public static final int DEFAULT_READER_TERMS_INDEX_DIVISOR = DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR;
 
   /** Default value is 1945. Change using {@link #setRAMPerThreadHardLimitMB(int)} */
   public static final int DEFAULT_RAM_PER_THREAD_HARD_LIMIT_MB = 1945;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergeState.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergeState.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergeState.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergeState.java Mon Jan 30 23:34:03 2012
@@ -31,10 +31,10 @@ import org.apache.lucene.util.InfoStream
 public class MergeState {
 
   public static class IndexReaderAndLiveDocs {
-    public final IndexReader reader;
+    public final AtomicReader reader;
     public final Bits liveDocs;
 
-    public IndexReaderAndLiveDocs(IndexReader reader, Bits liveDocs) {
+    public IndexReaderAndLiveDocs(AtomicReader reader, Bits liveDocs) {
       this.reader = reader;
       this.liveDocs = liveDocs;
     }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultiDocValues.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultiDocValues.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultiDocValues.java Mon Jan 30 23:34:03 2012
@@ -33,7 +33,7 @@ import org.apache.lucene.util.ReaderUtil
 import org.apache.lucene.util.packed.PackedInts.Reader;
 
 /**
- * A wrapper for compound IndexReader providing access to per segment
+ * A wrapper for CompositeIndexReader providing access to per segment
  * {@link DocValues}
  * 
  * @lucene.experimental
@@ -43,11 +43,11 @@ public class MultiDocValues extends DocV
   
   private static DocValuesPuller DEFAULT_PULLER = new DocValuesPuller();
   private static final DocValuesPuller NORMS_PULLER = new DocValuesPuller() {
-    public DocValues pull(IndexReader reader, String field) throws IOException {
+    public DocValues pull(AtomicReader reader, String field) throws IOException {
       return reader.normValues(field);
     }
     
-    public boolean stopLoadingOnNull(IndexReader reader, String field) throws IOException {
+    public boolean stopLoadingOnNull(AtomicReader reader, String field) throws IOException {
       // for norms we drop all norms if one leaf reader has no norms and the field is present
       FieldInfos fieldInfos = reader.getFieldInfos();
       FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
@@ -69,11 +69,11 @@ public class MultiDocValues extends DocV
   }
   
   private static class DocValuesPuller {
-    public DocValues pull(IndexReader reader, String field) throws IOException {
+    public DocValues pull(AtomicReader reader, String field) throws IOException {
       return reader.docValues(field);
     }
     
-    public boolean stopLoadingOnNull(IndexReader reader, String field) throws IOException {
+    public boolean stopLoadingOnNull(AtomicReader reader, String field) throws IOException {
       return false;
     }
   }
@@ -115,11 +115,13 @@ public class MultiDocValues extends DocV
   
  
   private static DocValues getDocValues(IndexReader r, final String field, final DocValuesPuller puller) throws IOException {
-    final IndexReader[] subs = r.getSequentialSubReaders();
-    if (subs == null) {
+    if (r instanceof AtomicReader) {
       // already an atomic reader
-      return puller.pull(r, field);
-    } else if (subs.length == 0) {
+      return puller.pull((AtomicReader) r, field);
+    }
+    assert r instanceof CompositeReader;
+    final IndexReader[] subs = ((CompositeReader) r).getSequentialSubReaders();
+    if (subs.length == 0) {
       // no fields
       return null;
     } else if (subs.length == 1) {
@@ -136,7 +138,7 @@ public class MultiDocValues extends DocV
       new ReaderUtil.Gather(r) {
         boolean stop = false;
         @Override
-        protected void add(int base, IndexReader r) throws IOException {
+        protected void add(int base, AtomicReader r) throws IOException {
           if (stop) {
             return;
           }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultiFields.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultiFields.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultiFields.java Mon Jan 30 23:34:03 2012
@@ -21,6 +21,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.Collection;
+import java.util.HashSet;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.lucene.util.Bits;
@@ -59,59 +61,50 @@ public final class MultiFields extends F
    *  Gather}) and iterate through them
    *  yourself. */
   public static Fields getFields(IndexReader r) throws IOException {
-    final IndexReader[] subs = r.getSequentialSubReaders();
-    if (subs == null) {
+    if (r instanceof AtomicReader) {
       // already an atomic reader
-      return r.fields();
-    } else if (subs.length == 0) {
+      return ((AtomicReader) r).fields();
+    }
+    assert r instanceof CompositeReader;
+    final IndexReader[] subs = ((CompositeReader) r).getSequentialSubReaders();
+    if (subs.length == 0) {
       // no fields
       return null;
-    } else if (subs.length == 1) {
-      return getFields(subs[0]);
     } else {
+      final List<Fields> fields = new ArrayList<Fields>();
+      final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
 
-      Fields currentFields = r.retrieveFields();
-      if (currentFields == null) {
-      
-        final List<Fields> fields = new ArrayList<Fields>();
-        final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
-
-        new ReaderUtil.Gather(r) {
-          @Override
-          protected void add(int base, IndexReader r) throws IOException {
-            final Fields f = r.fields();
-            if (f != null) {
-              fields.add(f);
-              slices.add(new ReaderUtil.Slice(base, r.maxDoc(), fields.size()-1));
-            }
+      new ReaderUtil.Gather(r) {
+        @Override
+        protected void add(int base, AtomicReader r) throws IOException {
+          final Fields f = r.fields();
+          if (f != null) {
+            fields.add(f);
+            slices.add(new ReaderUtil.Slice(base, r.maxDoc(), fields.size()-1));
           }
-        }.run();
-
-        if (fields.size() == 0) {
-          return null;
-        } else if (fields.size() == 1) {
-          currentFields = fields.get(0);
-        } else {
-          currentFields = new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
-                                         slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY));
         }
-        r.storeFields(currentFields);
+      }.run();
+
+      if (fields.isEmpty()) {
+        return null;
+      } else if (fields.size() == 1) {
+        return fields.get(0);
+      } else {
+        return new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
+                                       slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY));
       }
-      return currentFields;
     }
   }
 
   public static Bits getLiveDocs(IndexReader r) {
-    Bits result;
     if (r.hasDeletions()) {
-
       final List<Bits> liveDocs = new ArrayList<Bits>();
       final List<Integer> starts = new ArrayList<Integer>();
 
       try {
         final int maxDoc = new ReaderUtil.Gather(r) {
             @Override
-            protected void add(int base, IndexReader r) throws IOException {
+            protected void add(int base, AtomicReader r) throws IOException {
               // record all liveDocs, even if they are null
               liveDocs.add(r.getLiveDocs());
               starts.add(base);
@@ -126,16 +119,13 @@ public final class MultiFields extends F
       assert liveDocs.size() > 0;
       if (liveDocs.size() == 1) {
         // Only one actual sub reader -- optimize this case
-        result = liveDocs.get(0);
+        return liveDocs.get(0);
       } else {
-        result = new MultiBits(liveDocs, starts, true);
+        return new MultiBits(liveDocs, starts, true);
       }
-
     } else {
-      result = null;
+      return null;
     }
-
-    return result;
   }
 
   /**  This method may return null if the field does not exist.*/
@@ -237,6 +227,11 @@ public final class MultiFields extends F
     return result;
   }
 
+  @Override
+  public int getUniqueFieldCount() {
+    return -1;
+  }
+
   public static long totalTermFreq(IndexReader r, String field, BytesRef text) throws IOException {
     final Terms terms = getTerms(r, field);
     if (terms != null) {
@@ -248,9 +243,26 @@ public final class MultiFields extends F
     return 0;
   }
 
-  @Override
-  public int getUniqueFieldCount() {
-    return -1;
+  /** Call this to get the (merged) FieldInfos for a
+   *  composite reader */
+  public static FieldInfos getMergedFieldInfos(IndexReader reader) {
+    final List<AtomicReader> subReaders = new ArrayList<AtomicReader>();
+    ReaderUtil.gatherSubReaders(subReaders, reader);
+    final FieldInfos fieldInfos = new FieldInfos();
+    for(AtomicReader subReader : subReaders) {
+      fieldInfos.add(subReader.getFieldInfos());
+    }
+    return fieldInfos;
+  }
+
+  public static Collection<String> getIndexedFields(IndexReader reader) {
+    final Collection<String> fields = new HashSet<String>();
+    for(FieldInfo fieldInfo : getMergedFieldInfos(reader)) {
+      if (fieldInfo.isIndexed) {
+        fields.add(fieldInfo.name);
+      }
+    }
+    return fields;
   }
 }
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultiReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultiReader.java?rev=1238085&r1=1238084&r2=1238085&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultiReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultiReader.java Mon Jan 30 23:34:03 2012
@@ -22,7 +22,7 @@ import java.io.IOException;
 /** An IndexReader which reads multiple indexes, appending
  *  their content. */
 public class MultiReader extends BaseMultiReader<IndexReader> {
-  private final boolean[] decrefOnClose; // remember which subreaders to decRef on close
+  private final boolean closeSubReaders;
   
  /**
   * <p>Construct a MultiReader aggregating the named set of (sub)readers.
@@ -41,68 +41,11 @@ public class MultiReader extends BaseMul
    */
   public MultiReader(IndexReader[] subReaders, boolean closeSubReaders) throws IOException {
     super(subReaders.clone());
-    decrefOnClose = new boolean[subReaders.length];
-    for (int i = 0; i < subReaders.length; i++) {
-      if (!closeSubReaders) {
-        subReaders[i].incRef();
-        decrefOnClose[i] = true;
-      } else {
-        decrefOnClose[i] = false;
-      }
-    }
-  }
-  
-  // used only by openIfChaged
-  private MultiReader(IndexReader[] subReaders, boolean[] decrefOnClose)
-                      throws IOException {
-    super(subReaders);
-    this.decrefOnClose = decrefOnClose;
-  }
-
-  @Override
-  protected synchronized IndexReader doOpenIfChanged() throws CorruptIndexException, IOException {
-    ensureOpen();
-    
-    boolean changed = false;
-    IndexReader[] newSubReaders = new IndexReader[subReaders.length];
-    
-    boolean success = false;
-    try {
-      for (int i = 0; i < subReaders.length; i++) {
-        final IndexReader newSubReader = IndexReader.openIfChanged(subReaders[i]);
-        if (newSubReader != null) {
-          newSubReaders[i] = newSubReader;
-          changed = true;
-        } else {
-          newSubReaders[i] = subReaders[i];
-        }
-      }
-      success = true;
-    } finally {
-      if (!success && changed) {
-        for (int i = 0; i < newSubReaders.length; i++) {
-          if (newSubReaders[i] != subReaders[i]) {
-            try {
-              newSubReaders[i].close();
-            } catch (IOException ignore) {
-              // keep going - we want to clean up as much as possible
-            }
-          }
-        }
-      }
-    }
-
-    if (changed) {
-      boolean[] newDecrefOnClose = new boolean[subReaders.length];
+    this.closeSubReaders = closeSubReaders;
+    if (!closeSubReaders) {
       for (int i = 0; i < subReaders.length; i++) {
-        if (newSubReaders[i] == subReaders[i]) {
-          newSubReaders[i].incRef();
-          newDecrefOnClose[i] = true;
-        }
+        subReaders[i].incRef();
       }
-      return new MultiReader(newSubReaders, newDecrefOnClose);
-    } else {
-      return null;
     }
   }
 
@@ -111,10 +54,10 @@ public class MultiReader extends BaseMul
     IOException ioe = null;
     for (int i = 0; i < subReaders.length; i++) {
       try {
-        if (decrefOnClose[i]) {
-          subReaders[i].decRef();
-        } else {
+        if (closeSubReaders) {
           subReaders[i].close();
+        } else {
+          subReaders[i].decRef();
         }
       } catch (IOException e) {
         if (ioe == null) ioe = e;
@@ -123,25 +66,4 @@ public class MultiReader extends BaseMul
     // throw the first exception
     if (ioe != null) throw ioe;
   }
-  
-  @Override
-  public boolean isCurrent() throws CorruptIndexException, IOException {
-    ensureOpen();
-    for (int i = 0; i < subReaders.length; i++) {
-      if (!subReaders[i].isCurrent()) {
-        return false;
-      }
-    }
-    
-    // all subreaders are up to date
-    return true;
-  }
-  
-  /** Not implemented.
-   * @throws UnsupportedOperationException
-   */
-  @Override
-  public long getVersion() {
-    throw new UnsupportedOperationException("MultiReader does not support this method.");
-  }
 }