You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by us...@apache.org on 2010/04/06 21:19:36 UTC

svn commit: r931278 [5/10] - in /lucene/dev/trunk: lucene/ lucene/backwards/src/ lucene/backwards/src/java/org/apache/lucene/index/ lucene/backwards/src/java/org/apache/lucene/index/codecs/ lucene/backwards/src/java/org/apache/lucene/search/ lucene/bac...

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=931278&r1=931277&r2=931278&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 Tue Apr  6 19:19:27 2010
@@ -20,7 +20,11 @@ package org.apache.lucene.index;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldSelector;
 import org.apache.lucene.search.Similarity;
+import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.store.*;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.ReaderUtil;         // for javadocs
 
 import java.io.File;
 import java.io.FileOutputStream;
@@ -213,7 +217,7 @@ public abstract class IndexReader implem
    * @throws IOException if there is a low-level IO error
    */
   public static IndexReader open(final Directory directory) throws CorruptIndexException, IOException {
-    return open(directory, null, null, true, DEFAULT_TERMS_INDEX_DIVISOR);
+    return open(directory, null, null, true, DEFAULT_TERMS_INDEX_DIVISOR, null);
   }
 
   /** Returns an IndexReader reading the index in the given
@@ -227,7 +231,7 @@ public abstract class IndexReader implem
    * @throws IOException if there is a low-level IO error
    */
   public static IndexReader open(final Directory directory, boolean readOnly) throws CorruptIndexException, IOException {
-    return open(directory, null, null, readOnly, DEFAULT_TERMS_INDEX_DIVISOR);
+    return open(directory, null, null, readOnly, DEFAULT_TERMS_INDEX_DIVISOR, null);
   }
 
   /** Expert: returns an IndexReader reading the index in the given
@@ -241,7 +245,7 @@ public abstract class IndexReader implem
    * @throws IOException if there is a low-level IO error
    */
   public static IndexReader open(final IndexCommit commit, boolean readOnly) throws CorruptIndexException, IOException {
-    return open(commit.getDirectory(), null, commit, readOnly, DEFAULT_TERMS_INDEX_DIVISOR);
+    return open(commit.getDirectory(), null, commit, readOnly, DEFAULT_TERMS_INDEX_DIVISOR, null);
   }
 
   /** Expert: returns an IndexReader reading the index in
@@ -259,7 +263,7 @@ public abstract class IndexReader implem
    * @throws IOException if there is a low-level IO error
    */
   public static IndexReader open(final Directory directory, IndexDeletionPolicy deletionPolicy, boolean readOnly) throws CorruptIndexException, IOException {
-    return open(directory, deletionPolicy, null, readOnly, DEFAULT_TERMS_INDEX_DIVISOR);
+    return open(directory, deletionPolicy, null, readOnly, DEFAULT_TERMS_INDEX_DIVISOR, null);
   }
 
   /** Expert: returns an IndexReader reading the index in
@@ -287,7 +291,7 @@ public abstract class IndexReader implem
    * @throws IOException if there is a low-level IO error
    */
   public static IndexReader open(final Directory directory, IndexDeletionPolicy deletionPolicy, boolean readOnly, int termInfosIndexDivisor) throws CorruptIndexException, IOException {
-    return open(directory, deletionPolicy, null, readOnly, termInfosIndexDivisor);
+    return open(directory, deletionPolicy, null, readOnly, termInfosIndexDivisor, null);
   }
 
   /** Expert: returns an IndexReader reading the index in
@@ -307,7 +311,7 @@ public abstract class IndexReader implem
    * @throws IOException if there is a low-level IO error
    */
   public static IndexReader open(final IndexCommit commit, IndexDeletionPolicy deletionPolicy, boolean readOnly) throws CorruptIndexException, IOException {
-    return open(commit.getDirectory(), deletionPolicy, commit, readOnly, DEFAULT_TERMS_INDEX_DIVISOR);
+    return open(commit.getDirectory(), deletionPolicy, commit, readOnly, DEFAULT_TERMS_INDEX_DIVISOR, null);
   }
 
   /** Expert: returns an IndexReader reading the index in
@@ -337,11 +341,78 @@ public abstract class IndexReader implem
    * @throws IOException if there is a low-level IO error
    */
   public static IndexReader open(final IndexCommit commit, IndexDeletionPolicy deletionPolicy, boolean readOnly, int termInfosIndexDivisor) throws CorruptIndexException, IOException {
-    return open(commit.getDirectory(), deletionPolicy, commit, readOnly, termInfosIndexDivisor);
+    return open(commit.getDirectory(), deletionPolicy, commit, readOnly, termInfosIndexDivisor, null);
   }
 
-  private static IndexReader open(final Directory directory, final IndexDeletionPolicy deletionPolicy, final IndexCommit commit, final boolean readOnly, int termInfosIndexDivisor) throws CorruptIndexException, IOException {
-    return DirectoryReader.open(directory, deletionPolicy, commit, readOnly, termInfosIndexDivisor);
+  /** Expert: returns an IndexReader reading the index in
+   *  the given Directory, with a custom {@link
+   *  IndexDeletionPolicy}, and specified {@link CodecProvider}.
+   *  You should pass readOnly=true, since it gives much
+   *  better concurrent performance, unless you intend to do
+   *  write operations (delete documents or change norms)
+   *  with the reader.
+   * @param directory the index directory
+   * @param deletionPolicy a custom deletion policy (only used
+   *  if you use this reader to perform deletes or to set
+   *  norms); see {@link IndexWriter} for details.
+   * @param readOnly true if no changes (deletions, norms) will be made with this IndexReader
+   * @param termInfosIndexDivisor Subsamples which indexed
+   *  terms are loaded into RAM. This has the same effect as {@link
+   *  IndexWriter#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.
+   * @param codecs CodecProvider to use when opening index
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
+   */
+  public static IndexReader open(final Directory directory, IndexDeletionPolicy deletionPolicy, boolean readOnly, int termInfosIndexDivisor, CodecProvider codecs) throws CorruptIndexException, IOException {
+    return open(directory, deletionPolicy, null, readOnly, termInfosIndexDivisor, codecs);
+  }
+
+  /** Expert: returns an IndexReader reading the index in
+   *  the given Directory, using a specific commit and with
+   *  a custom {@link IndexDeletionPolicy} and specified
+   *  {@link CodecProvider}.  You should pass readOnly=true, since
+   *  it gives much better concurrent performance, unless
+   *  you intend to do write operations (delete documents or
+   *  change norms) with the reader.
+
+   * @param commit the specific {@link IndexCommit} to open;
+   * see {@link IndexReader#listCommits} to list all commits
+   * in a directory
+   * @param deletionPolicy a custom deletion policy (only used
+   *  if you use this reader to perform deletes or to set
+   *  norms); see {@link IndexWriter} for details.
+   * @param readOnly true if no changes (deletions, norms) will be made with this IndexReader
+   * @param termInfosIndexDivisor Subsamples which indexed
+   *  terms are loaded into RAM. This has the same effect as {@link
+   *  IndexWriter#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.
+   * @param codecs CodecProvider to use when opening index
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
+   */
+  public static IndexReader open(final IndexCommit commit, IndexDeletionPolicy deletionPolicy, boolean readOnly, int termInfosIndexDivisor, CodecProvider codecs) throws CorruptIndexException, IOException {
+    return open(commit.getDirectory(), deletionPolicy, commit, readOnly, termInfosIndexDivisor, codecs);
+  }
+
+  private static IndexReader open(final Directory directory, final IndexDeletionPolicy deletionPolicy, final IndexCommit commit, final boolean readOnly, int termInfosIndexDivisor,
+      CodecProvider codecs) throws CorruptIndexException, IOException {
+    if (codecs == null) {
+      codecs = CodecProvider.getDefault();
+    }
+    return DirectoryReader.open(directory, deletionPolicy, commit, readOnly, termInfosIndexDivisor, codecs);
   }
 
   /**
@@ -483,7 +554,7 @@ public abstract class IndexReader implem
    * @throws IOException if there is a low-level IO error
    */
   public static long getCurrentVersion(Directory directory) throws CorruptIndexException, IOException {
-    return SegmentInfos.readCurrentVersion(directory);
+    return SegmentInfos.readCurrentVersion(directory, CodecProvider.getDefault());
   }
 
   /**
@@ -501,7 +572,7 @@ public abstract class IndexReader implem
    * @see #getCommitUserData()
    */
   public static Map<String,String> getCommitUserData(Directory directory) throws CorruptIndexException, IOException {
-    return SegmentInfos.readCurrentUserData(directory);
+    return SegmentInfos.readCurrentUserData(directory, CodecProvider.getDefault());
   }
 
   /**
@@ -803,24 +874,63 @@ public abstract class IndexReader implem
    * calling terms(), {@link TermEnum#next()} must be called
    * on the resulting enumeration before calling other methods such as
    * {@link TermEnum#term()}.
+   * @deprecated Use the new flex API ({@link #fields()}) instead.
    * @throws IOException if there is a low-level IO error
    */
+  @Deprecated
   public abstract TermEnum terms() throws IOException;
 
+  /** Flex API: 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 Fields fields() throws IOException {
+    return new LegacyFields(this);
+  }
+  
   /** Returns an enumeration of all terms starting at a given term. If
    * the given term does not exist, the enumeration is positioned at the
    * first term greater than the supplied term. The enumeration is
    * ordered by Term.compareTo(). Each term is greater than all that
    * precede it in the enumeration.
+   * @deprecated Use the new flex API ({@link #fields()}) instead.
    * @throws IOException if there is a low-level IO error
    */
+  @Deprecated
   public abstract TermEnum terms(Term t) throws IOException;
 
   /** Returns the number of documents containing the term <code>t</code>.
    * @throws IOException if there is a low-level IO error
+   * @deprecated Use {@link #docFreq(String,BytesRef)} instead.
    */
+  @Deprecated
   public abstract int docFreq(Term t) throws IOException;
 
+  /** 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;
+    }
+    return terms.docFreq(term);
+  }
+
   /** Returns an enumeration of all the documents which contain
    * <code>term</code>. For each document, the document number, the frequency of
    * the term in that document is also provided, for use in
@@ -832,8 +942,10 @@ public abstract class IndexReader implem
    * </ul>
    * <p>The enumeration is ordered by document number.  Each document number
    * is greater than all that precede it in the enumeration.
+   * @deprecated Use the new flex API ({@link #termDocsEnum}) instead.
    * @throws IOException if there is a low-level IO error
    */
+  @Deprecated
   public TermDocs termDocs(Term term) throws IOException {
     ensureOpen();
     TermDocs termDocs = termDocs();
@@ -841,9 +953,57 @@ public abstract class IndexReader implem
     return termDocs;
   }
 
+  /** This may return null if the field does not exist.*/
+  public 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 DocsEnum termDocsEnum(Bits skipDocs, String field, BytesRef term) throws IOException {
+    assert field != null;
+    assert term != null;
+    final Fields fields = fields();
+    if (fields == null) {
+      return null;
+    }
+    final Terms terms = fields.terms(field);
+    if (terms != null) {
+      return terms.docs(skipDocs, term, null);
+    } else {
+      return null;
+    }
+  }
+
+  /** Returns {@link DocsAndPositionsEnum} for the specified
+   *  field & term.  This may return null, if either the
+   *  field or term does not exist, or, positions were not
+   *  stored for this term. */
+  public DocsAndPositionsEnum termPositionsEnum(Bits skipDocs, String field, BytesRef term) throws IOException {
+    assert field != null;
+    assert term != null;
+    final Fields fields = fields();
+    if (fields == null) {
+      return null;
+    }
+    final Terms terms = fields.terms(field);
+    if (terms != null) {
+      return terms.docsAndPositions(skipDocs, term, null);
+    } else {
+      return null;
+    }
+  }
+
   /** Returns an unpositioned {@link TermDocs} enumerator.
+   * @deprecated Use the new flex API ({@link #fields()}) instead.
    * @throws IOException if there is a low-level IO error
    */
+  @Deprecated
   public abstract TermDocs termDocs() throws IOException;
 
   /** Returns an enumeration of all the documents which contain
@@ -861,8 +1021,11 @@ public abstract class IndexReader implem
    * <p> This positional information facilitates phrase and proximity searching.
    * <p>The enumeration is ordered by document number.  Each document number is
    * greater than all that precede it in the enumeration.
+   * @deprecated Please switch the flex API ({@link
+   * #termDocsEnum}) instead
    * @throws IOException if there is a low-level IO error
    */
+  @Deprecated
   public TermPositions termPositions(Term term) throws IOException {
     ensureOpen();
     TermPositions termPositions = termPositions();
@@ -871,14 +1034,17 @@ public abstract class IndexReader implem
   }
 
   /** Returns an unpositioned {@link TermPositions} enumerator.
+   * @deprecated Please switch the flex API ({@link
+   * #termDocsEnum}) instead
    * @throws IOException if there is a low-level IO error
    */
+  @Deprecated
   public abstract TermPositions termPositions() throws IOException;
 
 
 
   /** Deletes the document numbered <code>docNum</code>.  Once a document is
-   * deleted it will not appear in TermDocs or TermPostitions enumerations.
+   * deleted it will not appear in TermDocs or TermPositions enumerations.
    * Attempts to read its field with the {@link #document}
    * method will result in an error.  The presence of this document may still be
    * reflected in the {@link #docFreq} statistic, though
@@ -1009,9 +1175,7 @@ public abstract class IndexReader implem
    * @throws IOException if there is a low-level IO error
    */
   public final synchronized void commit(Map<String, String> commitUserData) throws IOException {
-    if (hasChanges) {
-      doCommit(commitUserData);
-    }
+    doCommit(commitUserData);
     hasChanges = false;
   }
 
@@ -1044,6 +1208,33 @@ public abstract class IndexReader implem
    */
   public abstract Collection<String> getFieldNames(FieldOption fldOption);
 
+  // Only used by external subclasses of IndexReader; all
+  // internal classes should implement Bits more
+  // efficiently:
+  private final class DeletedDocsBits implements Bits {
+    public boolean get(int docID) {
+      return isDeleted(docID);
+    }
+    public int length() {
+      return maxDoc();
+    }
+  }
+
+  private Bits deletedDocsBits;
+
+  /** Returns the {@link Bits} representing deleted docs.  A
+   *  set bit indicates the doc ID has been deleted.  This
+   *  method should return null when there are no deleted
+   *  docs.
+   *
+   * @lucene.experimental */
+  public Bits getDeletedDocs() throws IOException {
+    if (deletedDocsBits == null) {
+      deletedDocsBits = new DeletedDocsBits();
+    }
+    return deletedDocsBits;
+  }
+
   /**
    * Expert: return the IndexCommit that this reader has
    * opened.  This method is only implemented by those
@@ -1169,6 +1360,12 @@ public abstract class IndexReader implem
     return null;
   }
 
+
+  /** Expert: returns the docID base for this subReader. */
+  public int getSubReaderDocBase(IndexReader subReader) {
+    throw new UnsupportedOperationException();
+  }
+
   /** Expert */
   public Object getFieldCacheKey() {
     return this;
@@ -1177,17 +1374,26 @@ public abstract class IndexReader implem
   /** Returns the number of unique terms (across all fields)
    *  in this reader.
    *
-   *  This method returns long, even though internally
-   *  Lucene cannot handle more than 2^31 unique terms, for
-   *  a possible future when this limitation is removed.
-   *
    *  @throws UnsupportedOperationException 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 long getUniqueTermCount() throws IOException {
-    throw new UnsupportedOperationException("this reader does not implement getUniqueTermCount()");
+    long numTerms = 0;
+    final Fields fields = fields();
+    if (fields == null) {
+      return 0;
+    }
+    FieldsEnum it = fields.iterator();
+    while(true) {
+      String field = it.next();
+      if (field == null) {
+        break;
+      }
+      numTerms += fields.terms(field).getUniqueTermCount();
+    }
+    return numTerms;
   }
 
   /** For IndexReader implementations that use
@@ -1198,4 +1404,29 @@ public abstract class IndexReader implem
   public int getTermInfosIndexDivisor() {
     throw new UnsupportedOperationException("This reader does not support this method.");
   }
+
+
+  private Fields fields;
+
+  /** lucene.internal */
+  void storeFields(Fields fields) {
+    this.fields = fields;
+  }
+
+  /** lucene.internal */
+  Fields retrieveFields() {
+    return fields;
+  }
+
+  private Bits storedDelDocs;
+
+  /** lucene.internal */
+  void storeDelDocs(Bits delDocs) {
+    this.storedDelDocs = delDocs;
+  }
+
+  /** lucene.internal */
+  Bits retrieveDelDocs() {
+    return storedDelDocs;
+  }
 }

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=931278&r1=931277&r2=931278&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 Tue Apr  6 19:19:27 2010
@@ -28,6 +28,7 @@ import org.apache.lucene.store.LockObtai
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.BufferedIndexInput;
 import org.apache.lucene.util.Constants;
+import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.util.ThreadInterruptedException;
 import org.apache.lucene.util.Version;
 
@@ -232,12 +233,13 @@ public class IndexWriter implements Clos
   public final static int DEFAULT_TERM_INDEX_INTERVAL = IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL;
 
   /**
-   * Absolute hard maximum length for a term.  If a term
-   * arrives from the analyzer longer than this length, it
-   * is skipped and a message is printed to infoStream, if
-   * set (see {@link #setInfoStream}).
+   * Absolute hard maximum length for a term, in bytes once
+   * encoded as UTF8.  If a term arrives from the analyzer
+   * longer than this length, it is skipped and a message is
+   * printed to infoStream, if set (see {@link
+   * #setInfoStream}).
    */
-  public final static int MAX_TERM_LENGTH = DocumentsWriter.MAX_TERM_LENGTH;
+  public final static int MAX_TERM_LENGTH = DocumentsWriter.MAX_TERM_LENGTH_UTF8;
 
   // The normal read buffer size defaults to 1024, but
   // increasing this during merging seems to yield
@@ -334,7 +336,7 @@ public class IndexWriter implements Clos
    *
    * <p>Note that this is functionally equivalent to calling
    * {#commit} and then using {@link IndexReader#open} to
-   * open a new reader.  But the turarnound time of this
+   * open a new reader.  But the turnaround time of this
    * method should be faster since it avoids the potentially
    * costly {@link #commit}.</p>
    *
@@ -420,7 +422,7 @@ public class IndexWriter implements Clos
     // just like we do when loading segments_N
     synchronized(this) {
       applyDeletes();
-      final IndexReader r = new ReadOnlyDirectoryReader(this, segmentInfos, termInfosIndexDivisor);
+      final IndexReader r = new ReadOnlyDirectoryReader(this, segmentInfos, termInfosIndexDivisor, codecs);
       if (infoStream != null) {
         message("return reader version=" + r.getVersion() + " reader=" + r);
       }
@@ -629,7 +631,7 @@ public class IndexWriter implements Clos
         // TODO: we may want to avoid doing this while
         // synchronized
         // Returns a ref, which we xfer to readerMap:
-        sr = SegmentReader.get(false, info.dir, info, readBufferSize, doOpenStores, termsIndexDivisor);
+        sr = SegmentReader.get(false, info.dir, info, readBufferSize, doOpenStores, termsIndexDivisor, codecs);
 
         if (info.dir == directory) {
           // Only pool if reader is not external
@@ -639,7 +641,7 @@ public class IndexWriter implements Clos
         if (doOpenStores) {
           sr.openDocStores();
         }
-        if (termsIndexDivisor != -1 && !sr.termsIndexLoaded()) {
+        if (termsIndexDivisor != -1) {
           // If this reader was originally opened because we
           // needed to merge it, we didn't load the terms
           // index.  But now, if the caller wants the terms
@@ -1038,6 +1040,8 @@ public class IndexWriter implements Clos
         .setOpenMode(OpenMode.APPEND).setMaxFieldLength(mfl.getLimit())
         .setIndexDeletionPolicy(deletionPolicy).setIndexCommit(commit));
   }
+  
+  CodecProvider codecs;
 
   /**
    * Constructs a new IndexWriter per the settings given in <code>conf</code>.
@@ -1081,6 +1085,8 @@ public class IndexWriter implements Clos
     mergePolicy.setIndexWriter(this);
     mergeScheduler = conf.getMergeScheduler();
     mergedSegmentWarmer = conf.getMergedSegmentWarmer();
+    codecs = conf.getCodecProvider();
+    
     poolReaders = conf.getReaderPooling();
 
     OpenMode mode = conf.getOpenMode();
@@ -1111,7 +1117,7 @@ public class IndexWriter implements Clos
         // segments_N file with no segments:
         boolean doCommit;
         try {
-          segmentInfos.read(directory);
+          segmentInfos.read(directory, codecs);
           segmentInfos.clear();
           doCommit = false;
         } catch (IOException e) {
@@ -1129,7 +1135,7 @@ public class IndexWriter implements Clos
           changeCount++;
         }
       } else {
-        segmentInfos.read(directory);
+        segmentInfos.read(directory, codecs);
 
         IndexCommit commit = conf.getIndexCommit();
         if (commit != null) {
@@ -1141,7 +1147,7 @@ public class IndexWriter implements Clos
           if (commit.getDirectory() != directory)
             throw new IllegalArgumentException("IndexCommit's directory doesn't match my directory");
           SegmentInfos oldInfos = new SegmentInfos();
-          oldInfos.read(directory, commit.getSegmentsFileName());
+          oldInfos.read(directory, commit.getSegmentsFileName(), codecs);
           segmentInfos.replace(oldInfos);
           changeCount++;
           if (infoStream != null)
@@ -1159,7 +1165,7 @@ public class IndexWriter implements Clos
       // KeepOnlyLastCommitDeleter:
       deleter = new IndexFileDeleter(directory,
                                      conf.getIndexDeletionPolicy(),
-                                     segmentInfos, infoStream, docWriter);
+                                     segmentInfos, infoStream, docWriter, this.codecs);
 
       if (deleter.startingCommitDeleted)
         // Deletion policy deleted the "head" commit point.
@@ -1174,6 +1180,7 @@ public class IndexWriter implements Clos
       pushMaxBufferedDocs();
 
       if (infoStream != null) {
+        message("init: create=" + create);
         messageState();
       }
 
@@ -3135,7 +3142,7 @@ public class IndexWriter implements Clos
             }
 
             SegmentInfos sis = new SegmentInfos(); // read infos from dir
-            sis.read(dirs[i]);
+            sis.read(dirs[i], codecs);
             for (int j = 0; j < sis.size(); j++) {
               SegmentInfo info = sis.info(j);
               assert !segmentInfos.contains(info): "dup info dir=" + info.dir + " name=" + info.name;
@@ -3321,7 +3328,7 @@ public class IndexWriter implements Clos
 
       try {
         mergedName = newSegmentName();
-        merger = new SegmentMerger(this, mergedName, null);
+        merger = new SegmentMerger(directory, termIndexInterval, mergedName, null, codecs);
 
         SegmentReader sReader = null;
         synchronized(this) {
@@ -3344,7 +3351,7 @@ public class IndexWriter implements Clos
           synchronized(this) {
             segmentInfos.clear();                      // pop old infos & add new
             info = new SegmentInfo(mergedName, docCount, directory, false, true,
-                                   -1, null, false, merger.hasProx());
+                                   -1, null, false, merger.hasProx(), merger.getCodec());
             setDiagnostics(info, "addIndexes(IndexReader...)");
             segmentInfos.add(info);
           }
@@ -3391,7 +3398,7 @@ public class IndexWriter implements Clos
           startTransaction(false);
 
           try {
-            merger.createCompoundFile(mergedName + ".cfs");
+            merger.createCompoundFile(mergedName + ".cfs", info);
             synchronized(this) {
               info.setUseCompoundFile(true);
             }
@@ -3742,7 +3749,9 @@ public class IndexWriter implements Clos
                                      directory, false, true,
                                      docStoreOffset, docStoreSegment,
                                      docStoreIsCompoundFile,    
-                                     docWriter.hasProx());
+                                     docWriter.hasProx(),
+                                     docWriter.getCodec());
+
         setDiagnostics(newSegment, "flush");
       }
 
@@ -3956,7 +3965,7 @@ public class IndexWriter implements Clos
         }
       }
     }
-
+    
     merge.info.setHasProx(merger.hasProx());
 
     segmentInfos.subList(start, start + merge.segments.size()).clear();
@@ -4032,7 +4041,7 @@ public class IndexWriter implements Clos
           mergeInit(merge);
 
           if (infoStream != null)
-            message("now merge\n  merge=" + merge.segString(directory) + "\n  merge=" + merge + "\n  index=" + segString());
+            message("now merge\n  merge=" + merge.segString(directory) + "\n  index=" + segString());
 
           mergeMiddle(merge);
           mergeSuccess(merge);
@@ -4258,7 +4267,8 @@ public class IndexWriter implements Clos
                                  docStoreOffset,
                                  docStoreSegment,
                                  docStoreIsCompoundFile,
-                                 false);
+                                 false,
+                                 null);
 
 
     Map<String,String> details = new HashMap<String,String>();
@@ -4338,7 +4348,7 @@ public class IndexWriter implements Clos
     if (infoStream != null)
       message("merging " + merge.segString(directory));
 
-    merger = new SegmentMerger(this, mergedName, merge);
+    merger = new SegmentMerger(directory, termIndexInterval, mergedName, merge, codecs);
 
     merge.readers = new SegmentReader[numSegments];
     merge.readersClone = new SegmentReader[numSegments];
@@ -4411,8 +4421,17 @@ public class IndexWriter implements Clos
       // This is where all the work happens:
       mergedDocCount = merge.info.docCount = merger.merge(merge.mergeDocStores);
 
+      // Record which codec was used to write the segment
+      merge.info.setCodec(merger.getCodec());
+      
       assert mergedDocCount == totDocCount;
 
+      // Very important to do this before opening the reader
+      // because codec must know if prox was written for
+      // this segment:
+      //System.out.println("merger set hasProx=" + merger.hasProx() + " seg=" + merge.info.name);
+      merge.info.setHasProx(merger.hasProx());
+
       // TODO: in the non-realtime case, we may want to only
       // keep deletes (it's costly to open entire reader
       // when we just need deletes)
@@ -4450,8 +4469,9 @@ public class IndexWriter implements Clos
                 merge.readersClone[i].close();
               } catch (Throwable t) {
               }
-              // This was a private clone and we had the only reference
-              assert merge.readersClone[i].getRefCount() == 0;
+              // This was a private clone and we had the
+              // only reference
+              assert merge.readersClone[i].getRefCount() == 0: "refCount should be 0 but is " + merge.readersClone[i].getRefCount();
             }
           }
         } else {
@@ -4484,7 +4504,7 @@ public class IndexWriter implements Clos
       final String compoundFileName = IndexFileNames.segmentFileName(mergedName, IndexFileNames.COMPOUND_FILE_EXTENSION);
 
       try {
-        merger.createCompoundFile(compoundFileName);
+        merger.createCompoundFile(compoundFileName, merge.info);
         success = true;
       } catch (IOException ioe) {
         synchronized(this) {

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=931278&r1=931277&r2=931278&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 Tue Apr  6 19:19:27 2010
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.DocumentsWriter.IndexingChain;
 import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
+import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.util.Version;
 
@@ -78,6 +79,9 @@ public final class IndexWriterConfig imp
    */
   public static long WRITE_LOCK_TIMEOUT = 1000;
 
+  /** Default {@link CodecProvider}. */
+  public final static CodecProvider DEFAULT_CODEC_PROVIDER = CodecProvider.getDefault();
+
   /** The maximum number of simultaneous threads that may be
    *  indexing documents at once in IndexWriter; if more
    *  than this many threads arrive they will wait for
@@ -119,6 +123,7 @@ public final class IndexWriterConfig imp
   private int maxBufferedDocs;
   private IndexingChain indexingChain;
   private IndexReaderWarmer mergedSegmentWarmer;
+  private CodecProvider codecProvider;
   private MergePolicy mergePolicy;
   private int maxThreadStates;
   private boolean readerPooling;
@@ -149,6 +154,7 @@ public final class IndexWriterConfig imp
     maxBufferedDocs = DEFAULT_MAX_BUFFERED_DOCS;
     indexingChain = DocumentsWriter.defaultIndexingChain;
     mergedSegmentWarmer = null;
+    codecProvider = DEFAULT_CODEC_PROVIDER;
     mergePolicy = new LogByteSizeMergePolicy();
     maxThreadStates = DEFAULT_MAX_THREAD_STATES;
     readerPooling = DEFAULT_READER_POOLING;
@@ -509,6 +515,18 @@ public final class IndexWriterConfig imp
     this.mergePolicy = mergePolicy == null ? new LogByteSizeMergePolicy() : mergePolicy;
     return this;
   }
+
+  /** Set the CodecProvider. See {@link CodecProvider}. */
+  public IndexWriterConfig setCodecProvider(CodecProvider codecProvider) {
+    this.codecProvider = codecProvider;
+    return this;
+  }
+
+  /** Returns the current merged segment warmer. See {@link IndexReaderWarmer}. */
+  public CodecProvider getCodecProvider() {
+    return codecProvider;
+  }
+
   
   /**
    * Returns the current MergePolicy in use by this writer.
@@ -584,6 +602,7 @@ public final class IndexWriterConfig imp
     sb.append("ramBufferSizeMB=").append(ramBufferSizeMB).append("\n");
     sb.append("maxBufferedDocs=").append(maxBufferedDocs).append("\n");
     sb.append("mergedSegmentWarmer=").append(mergedSegmentWarmer).append("\n");
+    sb.append("codecProvider=").append(codecProvider).append("\n");
     sb.append("mergePolicy=").append(mergePolicy).append("\n");
     sb.append("maxThreadStates=").append(maxThreadStates).append("\n");
     sb.append("readerPooling=").append(readerPooling).append("\n");

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=931278&r1=931277&r2=931278&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 Tue Apr  6 19:19:27 2010
@@ -25,17 +25,21 @@ import java.util.Map;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldSelector;
-import org.apache.lucene.index.DirectoryReader.MultiTermDocs;
-import org.apache.lucene.index.DirectoryReader.MultiTermEnum;
-import org.apache.lucene.index.DirectoryReader.MultiTermPositions;
+import org.apache.lucene.index.DirectoryReader.MultiTermDocs;       // deprecated
+import org.apache.lucene.index.DirectoryReader.MultiTermEnum;       // deprecated
+import org.apache.lucene.index.DirectoryReader.MultiTermPositions;  // deprecated
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.search.FieldCache; // not great (circular); used only to purge FieldCache entry on close
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.ReaderUtil;
 
 /** An IndexReader which reads multiple indexes, appending
- * their content. */
+ *  their content. */
 public class MultiReader extends IndexReader implements Cloneable {
   protected IndexReader[] subReaders;
   private int[] starts;                           // 1st docno for each segment
+  private final Map<IndexReader,ReaderUtil.Slice> subReaderToSlice = new HashMap<IndexReader,ReaderUtil.Slice>();
   private boolean[] decrefOnClose;                // remember which subreaders to decRef on close
   private Map<String,byte[]> normsCache = new HashMap<String,byte[]>();
   private int maxDoc = 0;
@@ -49,7 +53,7 @@ public class MultiReader extends IndexRe
   * <p>Note that all subreaders are closed if this Multireader is closed.</p>
   * @param subReaders set of (sub)readers
   */
-  public MultiReader(IndexReader... subReaders) {
+  public MultiReader(IndexReader... subReaders) throws IOException {
     initialize(subReaders, true);
   }
 
@@ -61,14 +65,15 @@ public class MultiReader extends IndexRe
    * when this MultiReader is closed
    * @param subReaders set of (sub)readers
    */
-  public MultiReader(IndexReader[] subReaders, boolean closeSubReaders) {
+  public MultiReader(IndexReader[] subReaders, boolean closeSubReaders) throws IOException {
     initialize(subReaders, closeSubReaders);
   }
   
-  private void initialize(IndexReader[] subReaders, boolean closeSubReaders) {
+  private void initialize(IndexReader[] subReaders, boolean closeSubReaders) throws IOException {
     this.subReaders =  subReaders.clone();
     starts = new int[subReaders.length + 1];    // build starts array
     decrefOnClose = new boolean[subReaders.length];
+
     for (int i = 0; i < subReaders.length; i++) {
       starts[i] = maxDoc;
       maxDoc += subReaders[i].maxDoc();      // compute maxDocs
@@ -80,12 +85,34 @@ public class MultiReader extends IndexRe
         decrefOnClose[i] = false;
       }
       
-      if (subReaders[i].hasDeletions())
+      if (subReaders[i].hasDeletions()) {
         hasDeletions = true;
+      }
+
+      final ReaderUtil.Slice slice = new ReaderUtil.Slice(starts[i],
+                                                          subReaders[i].maxDoc(),
+                                                          i);
+      subReaderToSlice.put(subReaders[i], slice);
     }
+
     starts[subReaders.length] = maxDoc;
   }
-  
+
+  @Override
+  public long getUniqueTermCount() throws IOException {
+    throw new UnsupportedOperationException("");
+  }
+
+  @Override
+  public int getSubReaderDocBase(IndexReader subReader) {
+    return subReaderToSlice.get(subReader).start;
+  }
+
+  @Override
+  public Fields fields() throws IOException {
+    throw new UnsupportedOperationException("please use MultiFields.getFields if you really need a top level Fields (NOTE that it's usually better to work per segment instead)");
+  }
+
   /**
    * Tries to reopen the subreaders.
    * <br>
@@ -128,6 +155,11 @@ public class MultiReader extends IndexRe
     }
   }
   
+  @Override
+  public Bits getDeletedDocs() throws IOException {
+    throw new UnsupportedOperationException("please use MultiFields.getDeletedDocs if you really need a top level Bits deletedDocs (NOTE that it's usually better to work per segment instead)");
+  }
+
   /**
    * If clone is true then we clone each of the subreaders
    * @param doClone
@@ -367,7 +399,17 @@ public class MultiReader extends IndexRe
       total += subReaders[i].docFreq(t);
     return total;
   }
-
+  
+  @Override
+  public int docFreq(String field, BytesRef t) throws IOException {
+    ensureOpen();
+    int total = 0;          // sum freqs in segments
+    for (int i = 0; i < subReaders.length; i++) {
+      total += subReaders[i].docFreq(field, t);
+    }
+    return total;
+  }
+  
   @Override
   public TermDocs termDocs() throws IOException {
     ensureOpen();

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultipleTermPositions.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultipleTermPositions.java?rev=931278&r1=931277&r2=931278&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultipleTermPositions.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MultipleTermPositions.java Tue Apr  6 19:19:27 2010
@@ -28,8 +28,10 @@ import org.apache.lucene.util.ArrayUtil;
 /**
  * Allows you to iterate over the {@link TermPositions} for multiple {@link Term}s as
  * a single {@link TermPositions}.
- *
+ * @deprecated This class is being replaced by the package
+ * private MultiDocsEnum on org.apache.lucene.search.
  */
+@Deprecated
 public class MultipleTermPositions implements TermPositions {
 
   private static final class TermPositionsQueue extends PriorityQueue<TermPositions> {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ParallelPostingsArray.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ParallelPostingsArray.java?rev=931278&r1=931277&r2=931278&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ParallelPostingsArray.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ParallelPostingsArray.java Tue Apr  6 19:19:27 2010
@@ -1,5 +1,7 @@
 package org.apache.lucene.index;
 
+import org.apache.lucene.util.ArrayUtil;
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -21,25 +23,49 @@ package org.apache.lucene.index;
 class ParallelPostingsArray {
   final static int BYTES_PER_POSTING = 3 * DocumentsWriter.INT_NUM_BYTE;
 
+  final int size;
   final int[] textStarts;
   final int[] intStarts;
   final int[] byteStarts;
-  
-  public ParallelPostingsArray(final int size) {
+
+  ParallelPostingsArray(final int size) {
+    this.size = size;
     textStarts = new int[size];
     intStarts = new int[size];
     byteStarts = new int[size];
   }
-  
-  ParallelPostingsArray resize(int newSize) {
-    ParallelPostingsArray newArray = new ParallelPostingsArray(newSize);
-    copy(this, newArray);
+
+  int bytesPerPosting() {
+    return BYTES_PER_POSTING;
+  }
+
+  ParallelPostingsArray newInstance(int size) {
+    return new ParallelPostingsArray(size);
+  }
+
+  final ParallelPostingsArray grow() {
+    int newSize = ArrayUtil.oversize(size + 1, bytesPerPosting());
+    ParallelPostingsArray newArray = newInstance(newSize);
+    copyTo(newArray, size);
     return newArray;
   }
-  
-  void copy(ParallelPostingsArray fromArray, ParallelPostingsArray toArray) {
-    System.arraycopy(fromArray.textStarts, 0, toArray.textStarts, 0, fromArray.textStarts.length);
-    System.arraycopy(fromArray.intStarts, 0, toArray.intStarts, 0, fromArray.intStarts.length);
-    System.arraycopy(fromArray.byteStarts, 0, toArray.byteStarts, 0, fromArray.byteStarts.length);
+
+  final ParallelPostingsArray shrink(int targetSize, boolean doCopy) {
+    int shrinkSize = ArrayUtil.getShrinkSize(size, targetSize, bytesPerPosting());
+    if (shrinkSize != size) {
+      ParallelPostingsArray newArray = newInstance(targetSize);
+      if (doCopy) {
+        copyTo(newArray, targetSize);
+      }
+      return newArray;
+    } else {
+      return this;
+    }
+  }
+
+  void copyTo(ParallelPostingsArray toArray, int numToCopy) {
+    System.arraycopy(textStarts, 0, toArray.textStarts, 0, numToCopy);
+    System.arraycopy(intStarts, 0, toArray.intStarts, 0, numToCopy);
+    System.arraycopy(byteStarts, 0, toArray.byteStarts, 0, numToCopy);
   }
 }

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=931278&r1=931277&r2=931278&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 Tue Apr  6 19:19:27 2010
@@ -21,7 +21,9 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.FieldSelector;
 import org.apache.lucene.document.FieldSelectorResult;
 import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.search.FieldCache; // not great (circular); used only to purge FieldCache entry on close
+import org.apache.lucene.util.BytesRef;
 
 import java.io.IOException;
 import java.util.*;
@@ -56,6 +58,8 @@ public class ParallelReader extends Inde
   private int numDocs;
   private boolean hasDeletions;
 
+  private ParallelFields fields = new ParallelFields();
+
  /** Construct a ParallelReader. 
   * <p>Note that all subreaders are closed if this ParallelReader is closed.</p>
   */
@@ -122,9 +126,11 @@ public class ParallelReader extends Inde
 
     Collection<String> fields = reader.getFieldNames(IndexReader.FieldOption.ALL);
     readerToFields.put(reader, fields);
-    for (final String field : fields) {                         // update fieldToReader map
-      if (fieldToReader.get(field) == null)
+    for (final String field : fields) {               // update fieldToReader map
+      if (fieldToReader.get(field) == null) {
         fieldToReader.put(field, reader);
+      }
+      this.fields.addField(field, reader);
     }
 
     if (!ignoreStoredFields)
@@ -136,6 +142,67 @@ public class ParallelReader extends Inde
     }
     decrefOnClose.add(Boolean.valueOf(incRefReaders));
   }
+
+  private class ParallelFieldsEnum extends FieldsEnum {
+    String currentField;
+    IndexReader currentReader;
+    Iterator<String> keys;
+
+    ParallelFieldsEnum() {
+      keys = fieldToReader.keySet().iterator();
+    }
+
+    @Override
+    public String next() throws IOException {
+      if (keys.hasNext()) {
+        currentField = (String) keys.next();
+        currentReader = (IndexReader) fieldToReader.get(currentField);
+      } else {
+        currentField = null;
+        currentReader = null;
+      }
+      return currentField;
+    }
+
+    @Override
+    public TermsEnum terms() throws IOException {
+      assert currentReader != null;
+      Terms terms = MultiFields.getTerms(currentReader, currentField);
+      if (terms != null) {
+        return terms.iterator();
+      } else {
+        return TermsEnum.EMPTY;
+      }
+    }
+  }
+
+  // Single instance of this, per ParallelReader instance
+  private class ParallelFields extends Fields {
+    final HashMap<String,Terms> fields = new HashMap<String,Terms>();
+
+    public void addField(String field, IndexReader r) throws IOException {
+      fields.put(field, MultiFields.getFields(r).terms(field));
+    }
+
+    @Override
+    public FieldsEnum iterator() throws IOException {
+      return new ParallelFieldsEnum();
+    }
+    @Override
+    public Terms terms(String field) throws IOException {
+      return fields.get(field);
+    }
+  }
+
+  @Override
+  public Bits getDeletedDocs() throws IOException {
+    return MultiFields.getDeletedDocs(readers.get(0));
+  }
+
+  @Override
+  public Fields fields() {
+    return fields;
+  }
   
   @Override
   public synchronized Object clone() {
@@ -404,6 +471,13 @@ public class ParallelReader extends Inde
   }
 
   @Override
+  public int docFreq(String field, BytesRef term) throws IOException {
+    ensureOpen();
+    IndexReader reader = ((IndexReader)fieldToReader.get(field));
+    return reader == null? 0 : reader.docFreq(field, term);
+  }
+
+  @Override
   public TermDocs termDocs(Term term) throws IOException {
     ensureOpen();
     return new ParallelTermDocs(term);
@@ -501,6 +575,7 @@ public class ParallelReader extends Inde
     return fieldSet;
   }
 
+  @Deprecated
   private class ParallelTermEnum extends TermEnum {
     private String field;
     private Iterator<String> fieldIterator;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ReadOnlyDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ReadOnlyDirectoryReader.java?rev=931278&r1=931277&r2=931278&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ReadOnlyDirectoryReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/ReadOnlyDirectoryReader.java Tue Apr  6 19:19:27 2010
@@ -18,22 +18,23 @@ package org.apache.lucene.index;
  */
 
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.index.codecs.CodecProvider;
 
 import java.io.IOException;
 import java.util.Map;
 
 class ReadOnlyDirectoryReader extends DirectoryReader {
-  ReadOnlyDirectoryReader(Directory directory, SegmentInfos sis, IndexDeletionPolicy deletionPolicy, int termInfosIndexDivisor) throws IOException {
-    super(directory, sis, deletionPolicy, true, termInfosIndexDivisor);
+  ReadOnlyDirectoryReader(Directory directory, SegmentInfos sis, IndexDeletionPolicy deletionPolicy, int termInfosIndexDivisor, CodecProvider codecs) throws IOException {
+    super(directory, sis, deletionPolicy, true, termInfosIndexDivisor, codecs);
   }
 
   ReadOnlyDirectoryReader(Directory directory, SegmentInfos infos, SegmentReader[] oldReaders, int[] oldStarts,  Map<String,byte[]> oldNormsCache, boolean doClone,
-                          int termInfosIndexDivisor) throws IOException {
-    super(directory, infos, oldReaders, oldStarts, oldNormsCache, true, doClone, termInfosIndexDivisor);
+                          int termInfosIndexDivisor, CodecProvider codecs) throws IOException {
+    super(directory, infos, oldReaders, oldStarts, oldNormsCache, true, doClone, termInfosIndexDivisor, codecs);
   }
   
-  ReadOnlyDirectoryReader(IndexWriter writer, SegmentInfos infos, int termInfosIndexDivisor) throws IOException {
-    super(writer, infos, termInfosIndexDivisor);
+  ReadOnlyDirectoryReader(IndexWriter writer, SegmentInfos infos, int termInfosIndexDivisor, CodecProvider codecs) throws IOException {
+    super(writer, infos, termInfosIndexDivisor, codecs);
   }
   
   @Override

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java?rev=931278&r1=931277&r2=931278&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java Tue Apr  6 19:19:27 2010
@@ -21,9 +21,13 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.BitVector;
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.CodecProvider;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.HashSet;
 import java.util.HashMap;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -87,10 +91,13 @@ public final class SegmentInfo {
                                                   // (if it's an older index)
 
   private boolean hasProx;                        // True if this segment has any fields with omitTermFreqAndPositions==false
+  
+  private Codec codec;
+
 
   private Map<String,String> diagnostics;
 
-  public SegmentInfo(String name, int docCount, Directory dir) {
+  public SegmentInfo(String name, int docCount, Directory dir, Codec codec) {
     this.name = name;
     this.docCount = docCount;
     this.dir = dir;
@@ -103,15 +110,13 @@ public final class SegmentInfo {
     docStoreIsCompoundFile = false;
     delCount = 0;
     hasProx = true;
+    this.codec = codec;
   }
 
-  public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile, boolean hasSingleNormFile) { 
-    this(name, docCount, dir, isCompoundFile, hasSingleNormFile, -1, null, false, true);
-  }
-
-  public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile, boolean hasSingleNormFile,
-                     int docStoreOffset, String docStoreSegment, boolean docStoreIsCompoundFile, boolean hasProx) { 
-    this(name, docCount, dir);
+  public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile, boolean hasSingleNormFile, 
+                     int docStoreOffset, String docStoreSegment, boolean docStoreIsCompoundFile, boolean hasProx,
+                     Codec codec) { 
+    this(name, docCount, dir, codec);
     this.isCompoundFile = (byte) (isCompoundFile ? YES : NO);
     this.hasSingleNormFile = hasSingleNormFile;
     preLockless = false;
@@ -119,6 +124,7 @@ public final class SegmentInfo {
     this.docStoreSegment = docStoreSegment;
     this.docStoreIsCompoundFile = docStoreIsCompoundFile;
     this.hasProx = hasProx;
+    this.codec = codec;
     delCount = 0;
     assert docStoreOffset == -1 || docStoreSegment != null: "dso=" + docStoreOffset + " dss=" + docStoreSegment + " docCount=" + docCount;
   }
@@ -144,6 +150,7 @@ public final class SegmentInfo {
     isCompoundFile = src.isCompoundFile;
     hasSingleNormFile = src.hasSingleNormFile;
     delCount = src.delCount;
+    codec = src.codec;
   }
 
   void setDiagnostics(Map<String, String> diagnostics) {
@@ -162,10 +169,11 @@ public final class SegmentInfo {
    * @param format format of the segments info file
    * @param input input handle to read segment info from
    */
-  SegmentInfo(Directory dir, int format, IndexInput input) throws IOException {
+  SegmentInfo(Directory dir, int format, IndexInput input, CodecProvider codecs) throws IOException {
     this.dir = dir;
     name = input.readString();
     docCount = input.readInt();
+    final String codecName;
     if (format <= SegmentInfos.FORMAT_LOCKLESS) {
       delGen = input.readLong();
       if (format <= SegmentInfos.FORMAT_SHARED_DOC_STORE) {
@@ -208,6 +216,13 @@ public final class SegmentInfo {
       else
         hasProx = true;
 
+      // System.out.println(Thread.currentThread().getName() + ": si.read hasProx=" + hasProx + " seg=" + name);
+      
+      if (format <= SegmentInfos.FORMAT_FLEX_POSTINGS)
+        codecName = input.readString();
+      else
+        codecName = "PreFlex";
+
       if (format <= SegmentInfos.FORMAT_DIAGNOSTICS) {
         diagnostics = input.readStringStringMap();
       } else {
@@ -224,8 +239,10 @@ public final class SegmentInfo {
       docStoreSegment = null;
       delCount = -1;
       hasProx = true;
+      codecName = "PreFlex";
       diagnostics = Collections.<String,String>emptyMap();
     }
+    codec = codecs.lookup(codecName);
   }
   
   void setNumFields(int numFields) {
@@ -309,7 +326,7 @@ public final class SegmentInfo {
 
   @Override
   public Object clone () {
-    SegmentInfo si = new SegmentInfo(name, docCount, dir);
+    SegmentInfo si = new SegmentInfo(name, docCount, dir, codec);
     si.isCompoundFile = isCompoundFile;
     si.delGen = delGen;
     si.delCount = delCount;
@@ -323,6 +340,7 @@ public final class SegmentInfo {
     si.docStoreOffset = docStoreOffset;
     si.docStoreSegment = docStoreSegment;
     si.docStoreIsCompoundFile = docStoreIsCompoundFile;
+    si.codec = codec;
     return si;
   }
 
@@ -373,14 +391,12 @@ public final class SegmentInfo {
         if (result == null)
           throw new IOException("cannot read directory " + dir + ": listAll() returned null");
 
-        final IndexFileNameFilter filter = IndexFileNameFilter.getFilter();
-        String pattern;
-        pattern = name + ".s";
-        int patternLength = pattern.length();
+        final String pattern = name + ".s\\d+";
         for(int i = 0; i < result.length; i++){
           String fileName = result[i];
-          if (filter.accept(null, fileName) && fileName.startsWith(pattern) && Character.isDigit(fileName.charAt(patternLength)))
-              return true;
+          if (fileName.matches(pattern)) {
+            return true;
+          }
         }
         return false;
       }
@@ -550,6 +566,7 @@ public final class SegmentInfo {
     output.writeByte(isCompoundFile);
     output.writeInt(delCount);
     output.writeByte((byte) (hasProx ? 1:0));
+    output.writeString(codec.name);
     output.writeStringStringMap(diagnostics);
   }
 
@@ -562,7 +579,20 @@ public final class SegmentInfo {
     return hasProx;
   }
 
-  private void addIfExists(List<String> files, String fileName) throws IOException {
+  /** Can only be called once. */
+  public void setCodec(Codec codec) {
+    assert this.codec == null;
+    if (codec == null) {
+      throw new IllegalArgumentException("codec must be non-null");
+    }
+    this.codec = codec;
+  }
+
+  Codec getCodec() {
+    return codec;
+  }
+
+  private void addIfExists(Set<String> files, String fileName) throws IOException {
     if (dir.fileExists(fileName))
       files.add(fileName);
   }
@@ -580,15 +610,17 @@ public final class SegmentInfo {
       return files;
     }
     
-    files = new ArrayList<String>();
+    Set<String> fileSet = new HashSet<String>();
     
     boolean useCompoundFile = getUseCompoundFile();
 
     if (useCompoundFile) {
-      files.add(IndexFileNames.segmentFileName(name, IndexFileNames.COMPOUND_FILE_EXTENSION));
+      fileSet.add(IndexFileNames.segmentFileName(name, IndexFileNames.COMPOUND_FILE_EXTENSION));
     } else {
-      for (String ext : IndexFileNames.NON_STORE_INDEX_EXTENSIONS)
-        addIfExists(files, IndexFileNames.segmentFileName(name, ext));
+      for(String ext : IndexFileNames.NON_STORE_INDEX_EXTENSIONS) {
+        addIfExists(fileSet, IndexFileNames.segmentFileName(name, ext));
+      }
+      codec.files(dir, this, fileSet);
     }
 
     if (docStoreOffset != -1) {
@@ -596,19 +628,19 @@ public final class SegmentInfo {
       // vectors) with other segments
       assert docStoreSegment != null;
       if (docStoreIsCompoundFile) {
-        files.add(IndexFileNames.segmentFileName(docStoreSegment, IndexFileNames.COMPOUND_FILE_STORE_EXTENSION));
+        fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, IndexFileNames.COMPOUND_FILE_STORE_EXTENSION));
       } else {
         for (String ext : IndexFileNames.STORE_INDEX_EXTENSIONS)
-          addIfExists(files, IndexFileNames.segmentFileName(docStoreSegment, ext));
+          addIfExists(fileSet, IndexFileNames.segmentFileName(docStoreSegment, ext));
       }
     } else if (!useCompoundFile) {
       for (String ext : IndexFileNames.STORE_INDEX_EXTENSIONS)
-        addIfExists(files, IndexFileNames.segmentFileName(name, ext));
+        addIfExists(fileSet, IndexFileNames.segmentFileName(name, ext));
     }
 
     String delFileName = IndexFileNames.fileNameFromGeneration(name, IndexFileNames.DELETES_EXTENSION, delGen);
     if (delFileName != null && (delGen >= YES || dir.fileExists(delFileName))) {
-      files.add(delFileName);
+      fileSet.add(delFileName);
     }
 
     // Careful logic for norms files    
@@ -617,14 +649,14 @@ public final class SegmentInfo {
         long gen = normGen[i];
         if (gen >= YES) {
           // Definitely a separate norm file, with generation:
-          files.add(IndexFileNames.fileNameFromGeneration(name, IndexFileNames.SEPARATE_NORMS_EXTENSION + i, gen));
+          fileSet.add(IndexFileNames.fileNameFromGeneration(name, IndexFileNames.SEPARATE_NORMS_EXTENSION + i, gen));
         } else if (NO == gen) {
           // No separate norms but maybe plain norms
           // in the non compound file case:
           if (!hasSingleNormFile && !useCompoundFile) {
             String fileName = IndexFileNames.segmentFileName(name, IndexFileNames.PLAIN_NORMS_EXTENSION + i);
             if (dir.fileExists(fileName)) {
-              files.add(fileName);
+              fileSet.add(fileName);
             }
           }
         } else if (CHECK_DIR == gen) {
@@ -636,7 +668,7 @@ public final class SegmentInfo {
             fileName = IndexFileNames.segmentFileName(name, IndexFileNames.PLAIN_NORMS_EXTENSION + i);
           }
           if (fileName != null && dir.fileExists(fileName)) {
-            files.add(fileName);
+            fileSet.add(fileName);
           }
         }
       }
@@ -644,20 +676,24 @@ public final class SegmentInfo {
       // Pre-2.1: we have to scan the dir to find all
       // matching _X.sN/_X.fN files for our segment:
       String prefix;
-      if (useCompoundFile)
+      if (useCompoundFile) {
         prefix = IndexFileNames.segmentFileName(name, IndexFileNames.SEPARATE_NORMS_EXTENSION);
-      else
+      } else {
         prefix = IndexFileNames.segmentFileName(name, IndexFileNames.PLAIN_NORMS_EXTENSION);
-      int prefixLength = prefix.length();
+      }
+      final String pattern = prefix + "\\d+";
+
       String[] allFiles = dir.listAll();
-      final IndexFileNameFilter filter = IndexFileNameFilter.getFilter();
       for(int i=0;i<allFiles.length;i++) {
         String fileName = allFiles[i];
-        if (filter.accept(null, fileName) && fileName.length() > prefixLength && Character.isDigit(fileName.charAt(prefixLength)) && fileName.startsWith(prefix)) {
-          files.add(fileName);
+        if (fileName.matches(pattern)) {
+          fileSet.add(fileName);
         }
       }
     }
+
+    files = new ArrayList<String>(fileSet);
+
     return files;
   }
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java?rev=931278&r1=931277&r2=931278&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java Tue Apr  6 19:19:27 2010
@@ -23,6 +23,7 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.store.ChecksumIndexOutput;
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.NoSuchDirectoryException;
+import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.util.ThreadInterruptedException;
 
 import java.io.FileNotFoundException;
@@ -88,9 +89,13 @@ public final class SegmentInfos extends 
   /** This format adds optional per-segment String
    *  diagnostics storage, and switches userData to Map */
   public static final int FORMAT_DIAGNOSTICS = -9;
+  
+  /** Each segment records whether its postings are written
+   *  in the new flex format */
+  public static final int FORMAT_FLEX_POSTINGS = -10;
 
   /* This must always point to the most recent file format. */
-  static final int CURRENT_FORMAT = FORMAT_DIAGNOSTICS;
+  static final int CURRENT_FORMAT = FORMAT_FLEX_POSTINGS;
   
   public int counter = 0;    // used to name new segments
   /**
@@ -228,7 +233,8 @@ public final class SegmentInfos extends 
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public final void read(Directory directory, String segmentFileName) throws CorruptIndexException, IOException {
+  public final void read(Directory directory, String segmentFileName, 
+                         CodecProvider codecs) throws CorruptIndexException, IOException {
     boolean success = false;
 
     // Clear any previous segments:
@@ -254,7 +260,7 @@ public final class SegmentInfos extends 
       }
       
       for (int i = input.readInt(); i > 0; i--) { // read segmentInfos
-        add(new SegmentInfo(directory, format, input));
+        add(new SegmentInfo(directory, format, input, codecs));
       }
       
       if(format >= 0){    // in old format the version number may be at the end of the file
@@ -301,14 +307,17 @@ public final class SegmentInfos extends 
    * @throws IOException if there is a low-level IO error
    */
   public final void read(Directory directory) throws CorruptIndexException, IOException {
-
+    read(directory, CodecProvider.getDefault());
+  }
+  
+  public final void read(Directory directory, final CodecProvider codecs) throws CorruptIndexException, IOException {
     generation = lastGeneration = -1;
 
     new FindSegmentsFile(directory) {
 
       @Override
       protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
-        read(directory, segmentFileName);
+        read(directory, segmentFileName, codecs);
         return null;
       }
     }.run();
@@ -375,9 +384,11 @@ public final class SegmentInfos extends 
   public Object clone() {
     SegmentInfos sis = (SegmentInfos) super.clone();
     for(int i=0;i<sis.size();i++) {
-      sis.set(i, (SegmentInfo) sis.info(i).clone());
+      final SegmentInfo info = sis.info(i);
+      assert info.getCodec() != null;
+      sis.set(i, (SegmentInfo) info.clone());
     }
-    sis.userData = new HashMap<String, String>(userData);
+    sis.userData = new HashMap<String,String>(userData);
     return sis;
   }
 
@@ -399,7 +410,7 @@ public final class SegmentInfos extends 
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public static long readCurrentVersion(Directory directory)
+  public static long readCurrentVersion(Directory directory, final CodecProvider codecs)
     throws CorruptIndexException, IOException {
 
     // Fully read the segments file: this ensures that it's
@@ -417,10 +428,10 @@ public final class SegmentInfos extends 
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public static Map<String,String> readCurrentUserData(Directory directory)
+  public static Map<String,String> readCurrentUserData(Directory directory, CodecProvider codecs)
     throws CorruptIndexException, IOException {
     SegmentInfos sis = new SegmentInfos();
-    sis.read(directory);
+    sis.read(directory, codecs);
     return sis.getUserData();
   }
 

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=931278&r1=931277&r2=931278&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 Tue Apr  6 19:19:27 2010
@@ -20,15 +20,23 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-
+import java.util.Set;
+import java.util.HashSet;
 import java.util.List;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexReader.FieldOption;
 import org.apache.lucene.index.MergePolicy.MergeAbortedException;
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.MergeState;
+import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.ReaderUtil;
+import org.apache.lucene.util.MultiBits;
 
 /**
  * The SegmentMerger class combines two or more Segments, represented by an IndexReader ({@link #add},
@@ -66,25 +74,14 @@ final class SegmentMerger {
   /** Maximum number of contiguous documents to bulk-copy
       when merging stored fields */
   private final static int MAX_RAW_MERGE_DOCS = 4192;
+  
+  private final CodecProvider codecs;
+  private Codec codec;
+  private SegmentWriteState segmentWriteState;
 
-  /** This ctor used only by test code.
-   * 
-   * @param dir The Directory to merge the other segments into
-   * @param name The name of the new segment
-   */
-  SegmentMerger(Directory dir, String name) {
+  SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, CodecProvider codecs) {
     directory = dir;
-    segment = name;
-    checkAbort = new CheckAbort(null, null) {
-      @Override
-      public void work(double units) throws MergeAbortedException {
-        // do nothing
-      }
-    };
-  }
-
-  SegmentMerger(IndexWriter writer, String name, MergePolicy.OneMerge merge) {
-    directory = writer.getDirectory();
+    this.codecs = codecs;
     segment = name;
     if (merge != null) {
       checkAbort = new CheckAbort(merge, directory);
@@ -96,7 +93,7 @@ final class SegmentMerger {
         }
       };
     }
-    termIndexInterval = writer.getConfig().getTermIndexInterval();
+    this.termIndexInterval = termIndexInterval;
   }
   
   boolean hasProx() {
@@ -171,30 +168,27 @@ final class SegmentMerger {
     }
   }
 
-  final List<String> createCompoundFile(String fileName)
+  final List<String> createCompoundFile(String fileName, final SegmentInfo info)
           throws IOException {
-    CompoundFileWriter cfsWriter =
-      new CompoundFileWriter(directory, fileName, checkAbort);
+    CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, fileName, checkAbort);
 
-    List<String> files =
-      new ArrayList<String>(IndexFileNames.COMPOUND_EXTENSIONS.length + 1);    
-    
-    // Basic files
-    for (String ext : IndexFileNames.COMPOUND_EXTENSIONS) {
-      if (ext.equals(IndexFileNames.PROX_EXTENSION) && !hasProx())
-        continue;
+    Set<String> fileSet = new HashSet<String>();
 
+    // Basic files
+    for (String ext : IndexFileNames.COMPOUND_EXTENSIONS_NOT_CODEC) {
       if (mergeDocStores || (!ext.equals(IndexFileNames.FIELDS_EXTENSION) &&
-                            !ext.equals(IndexFileNames.FIELDS_INDEX_EXTENSION)))
-        files.add(IndexFileNames.segmentFileName(segment, ext));
+                             !ext.equals(IndexFileNames.FIELDS_INDEX_EXTENSION)))
+        fileSet.add(IndexFileNames.segmentFileName(segment, ext));
     }
 
+    codec.files(directory, info, fileSet);
+    
     // Fieldable norm files
     int numFIs = fieldInfos.size();
     for (int i = 0; i < numFIs; i++) {
       FieldInfo fi = fieldInfos.fieldInfo(i);
       if (fi.isIndexed && !fi.omitNorms) {
-        files.add(IndexFileNames.segmentFileName(segment, IndexFileNames.NORMS_EXTENSION));
+        fileSet.add(IndexFileNames.segmentFileName(segment, IndexFileNames.NORMS_EXTENSION));
         break;
       }
     }
@@ -202,19 +196,19 @@ final class SegmentMerger {
     // Vector files
     if (fieldInfos.hasVectors() && mergeDocStores) {
       for (String ext : IndexFileNames.VECTOR_EXTENSIONS) {
-        files.add(IndexFileNames.segmentFileName(segment, ext));
+        fileSet.add(IndexFileNames.segmentFileName(segment, ext));
       }
     }
 
     // Now merge all added files
-    for (String file : files) {
+    for (String file : fileSet) {
       cfsWriter.addFile(file);
     }
     
     // Perform the merge
     cfsWriter.close();
    
-    return files;
+    return new ArrayList<String>(fileSet);
   }
 
   private void addIndexed(IndexReader reader, FieldInfos fInfos,
@@ -351,13 +345,16 @@ final class SegmentMerger {
         // details.
         throw new RuntimeException("mergeFields produced an invalid result: docCount is " + docCount + " but fdx file size is " + fdxFileLength + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
 
-    } else
+    } else {
       // If we are skipping the doc stores, that means there
       // are no deletions in any of these segments, so we
       // just sum numDocs() of each segment to get total docCount
       for (final IndexReader reader : readers) {
         docCount += reader.numDocs();
       }
+    }
+
+    segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, null, docCount, 0, termIndexInterval, codecs);
 
     return docCount;
   }
@@ -552,156 +549,116 @@ final class SegmentMerger {
     }
   }
 
-  private SegmentMergeQueue queue = null;
+  Codec getCodec() {
+    return codec;
+  }
 
   private final void mergeTerms() throws CorruptIndexException, IOException {
 
-    SegmentWriteState state = new SegmentWriteState(null, directory, segment, null, mergedDocs, 0, termIndexInterval);
-
-    final FormatPostingsFieldsConsumer consumer = new FormatPostingsFieldsWriter(state, fieldInfos);
-
-    try {
-      queue = new SegmentMergeQueue(readers.size());
-
-      mergeTermInfos(consumer);
+    // Let CodecProvider decide which codec will be used to write
+    // the new segment:
+    codec = codecs.getWriter(segmentWriteState);
+    
+    int docBase = 0;
 
-    } finally {
-      consumer.finish();
-      if (queue != null) queue.close();
+    final List<Fields> fields = new ArrayList<Fields>();
+    final List<IndexReader> subReaders = new ArrayList<IndexReader>();
+    final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
+    final List<Bits> bits = new ArrayList<Bits>();
+    final List<Integer> bitsStarts = new ArrayList<Integer>();
+
+    final int numReaders = readers.size();
+    for(int i=0;i<numReaders;i++) {
+      docBase = new ReaderUtil.Gather(readers.get(i)) {
+          @Override
+          protected void add(int base, IndexReader r) throws IOException {
+            subReaders.add(r);
+            fields.add(r.fields());
+            slices.add(new ReaderUtil.Slice(base, r.maxDoc(), fields.size()-1));
+            bits.add(r.getDeletedDocs());
+            bitsStarts.add(base);
+          }
+        }.run(docBase);
     }
-  }
 
-  boolean omitTermFreqAndPositions;
+    bitsStarts.add(docBase);
 
-  private final void mergeTermInfos(final FormatPostingsFieldsConsumer consumer) throws CorruptIndexException, IOException {
-    int base = 0;
-    final int readerCount = readers.size();
-    for (int i = 0; i < readerCount; i++) {
-      IndexReader reader = readers.get(i);
-      TermEnum termEnum = reader.terms();
-      SegmentMergeInfo smi = new SegmentMergeInfo(base, termEnum, reader);
-      int[] docMap  = smi.getDocMap();
-      if (docMap != null) {
-        if (docMaps == null) {
-          docMaps = new int[readerCount][];
-          delCounts = new int[readerCount];
+    // we may gather more readers than mergeState.readerCount
+    mergeState = new MergeState();
+    mergeState.readers = subReaders;
+    mergeState.readerCount = subReaders.size();
+    mergeState.fieldInfos = fieldInfos;
+    mergeState.mergedDocCount = mergedDocs;
+    
+    // Remap docIDs
+    mergeState.delCounts = new int[mergeState.readerCount];
+    mergeState.docMaps = new int[mergeState.readerCount][];
+    mergeState.docBase = new int[mergeState.readerCount];
+
+    docBase = 0;
+    int inputDocBase = 0;
+
+    final int[] starts = new int[mergeState.readerCount+1];
+
+    for(int i=0;i<mergeState.readerCount;i++) {
+
+      final IndexReader reader = subReaders.get(i);
+
+      starts[i] = inputDocBase;
+
+      mergeState.delCounts[i] = reader.numDeletedDocs();
+      mergeState.docBase[i] = docBase;
+      docBase += reader.numDocs();
+      inputDocBase += reader.maxDoc();
+      if (mergeState.delCounts[i] != 0) {
+        int delCount = 0;
+        Bits deletedDocs = reader.getDeletedDocs();
+        final int maxDoc = reader.maxDoc();
+        final int[] docMap = mergeState.docMaps[i] = new int[maxDoc];
+        int newDocID = 0;
+        for(int j=0;j<maxDoc;j++) {
+          if (deletedDocs.get(j)) {
+            docMap[j] = -1;
+            delCount++;  // only for assert
+          } else {
+            docMap[j] = newDocID++;
+          }
         }
-        docMaps[i] = docMap;
-        delCounts[i] = smi.reader.maxDoc() - smi.reader.numDocs();
+        assert delCount == mergeState.delCounts[i]: "reader delCount=" + mergeState.delCounts[i] + " vs recomputed delCount=" + delCount;
       }
-      
-      base += reader.numDocs();
-
-      assert reader.numDocs() == reader.maxDoc() - smi.delCount;
-
-      if (smi.next())
-        queue.add(smi);				  // initialize queue
-      else
-        smi.close();
     }
+    starts[mergeState.readerCount] = inputDocBase;
 
-    SegmentMergeInfo[] match = new SegmentMergeInfo[readers.size()];
-
-    String currentField = null;
-    FormatPostingsTermsConsumer termsConsumer = null;
-
-    while (queue.size() > 0) {
-      int matchSize = 0;			  // pop matching terms
-      match[matchSize++] = queue.pop();
-      Term term = match[0].term;
-      SegmentMergeInfo top = queue.top();
-
-      while (top != null && term.compareTo(top.term) == 0) {
-        match[matchSize++] =  queue.pop();
-        top =  queue.top();
-      }
-
-      if (currentField != term.field) {
-        currentField = term.field;
-        if (termsConsumer != null)
-          termsConsumer.finish();
-        final FieldInfo fieldInfo = fieldInfos.fieldInfo(currentField);
-        termsConsumer = consumer.addField(fieldInfo);
-        omitTermFreqAndPositions = fieldInfo.omitTermFreqAndPositions;
-      }
-
-      int df = appendPostings(termsConsumer, match, matchSize);		  // add new TermInfo
+    final FieldsConsumer consumer = codec.fieldsConsumer(segmentWriteState);
 
-      checkAbort.work(df/3.0);
-
-      while (matchSize > 0) {
-        SegmentMergeInfo smi = match[--matchSize];
-        if (smi.next())
-          queue.add(smi);			  // restore queue
-        else
-          smi.close();				  // done with a segment
-      }
+    // NOTE: this is silly, yet, necessary -- we create a
+    // MultiBits as our skip docs only to have it broken
+    // apart when we step through the docs enums in
+    // MultidDcsEnum.... this only matters when we are
+    // interacting with a non-core IR subclass, because
+    // LegacyFieldsEnum.LegacyDocs[AndPositions]Enum checks
+    // that the skipDocs matches the delDocs for the reader
+    mergeState.multiDeletedDocs = new MultiBits(bits, bitsStarts);
+    
+    try {
+      consumer.merge(mergeState,
+                     new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
+                                     slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY)));
+    } finally {
+      consumer.close();
     }
   }
 
-  private byte[] payloadBuffer;
-  private int[][] docMaps;
+  private MergeState mergeState;
+
   int[][] getDocMaps() {
-    return docMaps;
+    return mergeState.docMaps;
   }
-  private int[] delCounts;
-  int[] getDelCounts() {
-    return delCounts;
-  }
-
-  /** Process postings from multiple segments all positioned on the
-   *  same term. Writes out merged entries into freqOutput and
-   *  the proxOutput streams.
-   *
-   * @param smis array of segments
-   * @param n number of cells in the array actually occupied
-   * @return number of documents across all segments where this term was found
-   * @throws CorruptIndexException if the index is corrupt
-   * @throws IOException if there is a low-level IO error
-   */
-  private final int appendPostings(final FormatPostingsTermsConsumer termsConsumer, SegmentMergeInfo[] smis, int n)
-        throws CorruptIndexException, IOException {
-
-    final FormatPostingsDocsConsumer docConsumer = termsConsumer.addTerm(smis[0].term.text);
-    int df = 0;
-    for (int i = 0; i < n; i++) {
-      SegmentMergeInfo smi = smis[i];
-      TermPositions postings = smi.getPositions();
-      assert postings != null;
-      int base = smi.base;
-      int[] docMap = smi.getDocMap();
-      postings.seek(smi.termEnum);
-
-      while (postings.next()) {
-        df++;
-        int doc = postings.doc();
-        if (docMap != null)
-          doc = docMap[doc];                      // map around deletions
-        doc += base;                              // convert to merged space
-
-        final int freq = postings.freq();
-        final FormatPostingsPositionsConsumer posConsumer = docConsumer.addDoc(doc, freq);
-
-        if (!omitTermFreqAndPositions) {
-          for (int j = 0; j < freq; j++) {
-            final int position = postings.nextPosition();
-            final int payloadLength = postings.getPayloadLength();
-            if (payloadLength > 0) {
-              if (payloadBuffer == null || payloadBuffer.length < payloadLength)
-                payloadBuffer = new byte[payloadLength];
-              postings.getPayload(payloadBuffer, 0);
-            }
-            posConsumer.addPosition(position, payloadBuffer, 0, payloadLength);
-          }
-          posConsumer.finish();
-        }
-      }
-    }
-    docConsumer.finish();
 
-    return df;
+  int[] getDelCounts() {
+    return mergeState.delCounts;
   }
-
+  
   private void mergeNorms() throws IOException {
     byte[] normBuffer = null;
     IndexOutput output = null;