You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2011/10/31 19:00:00 UTC

svn commit: r1195585 - in /lucene/dev/branches/lucene2621/lucene/src: java/org/apache/lucene/index/ java/org/apache/lucene/index/codecs/ java/org/apache/lucene/index/values/ java/org/apache/lucene/search/ test-framework/org/apache/lucene/index/ test-fr...

Author: rmuir
Date: Mon Oct 31 17:59:59 2011
New Revision: 1195585

URL: http://svn.apache.org/viewvc?rev=1195585&view=rev
Log:
LUCENE-3490: die CodecProvider die (step 1)

Added:
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosFormat.java   (with props)
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosFormat.java   (with props)
Removed:
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/CoreCodecProvider.java
Modified:
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/CheckIndex.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DirectoryReader.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexFileNameFilter.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexReader.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/Codec.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosReader.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/IndexDocValues.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/CheckIndex.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/CheckIndex.java Mon Oct 31 17:59:59 2011
@@ -26,7 +26,6 @@ import org.apache.lucene.store.IOContext
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
 import java.io.File;
 import java.io.IOException;
@@ -323,10 +322,6 @@ public class CheckIndex {
   public Status checkIndex() throws IOException {
     return checkIndex(null);
   }
-
-  public Status checkIndex(List<String> onlySegments) throws IOException {
-    return checkIndex(onlySegments, CodecProvider.getDefault());
-  }
   
   /** Returns a {@link Status} instance detailing
    *  the state of the index.
@@ -340,9 +335,9 @@ public class CheckIndex {
    *  <p><b>WARNING</b>: make sure
    *  you only call this when the index is not opened by any
    *  writer. */
-  public Status checkIndex(List<String> onlySegments, CodecProvider codecs) throws IOException {
+  public Status checkIndex(List<String> onlySegments) throws IOException {
     NumberFormat nf = NumberFormat.getInstance();
-    SegmentInfos sis = new SegmentInfos(codecs);
+    SegmentInfos sis = new SegmentInfos();
     Status result = new Status();
     result.dir = dir;
     try {
@@ -378,6 +373,7 @@ public class CheckIndex {
 
     final int numSegments = sis.size();
     final String segmentsFileName = sis.getCurrentSegmentFileName();
+    // nocommit: abstraction violation.
     IndexInput input = null;
     try {
       input = dir.openInput(segmentsFileName, IOContext.DEFAULT);
@@ -1183,11 +1179,11 @@ public class CheckIndex {
    *
    * <p><b>WARNING</b>: Make sure you only call this when the
    *  index is not opened  by any writer. */
-  public void fixIndex(Status result) throws IOException {
+  public void fixIndex(Status result, Codec codec) throws IOException {
     if (result.partial)
       throw new IllegalArgumentException("can only fix an index that was fully checked (this status checked a subset of segments)");
     result.newSegments.changed();
-    result.newSegments.commit(result.dir);
+    result.newSegments.commit(result.dir, codec);
   }
 
   private static boolean assertsOn;
@@ -1330,7 +1326,8 @@ public class CheckIndex {
           System.out.println("  " + (5-s) + "...");
         }
         System.out.println("Writing...");
-        checker.fixIndex(result);
+        // nocommit: do we need a commandline flag here, e.g. checkindex on AppendingCodec)
+        checker.fixIndex(result, Codec.getDefault());
         System.out.println("OK");
         System.out.println("Wrote new segments file \"" + result.newSegments.getCurrentSegmentFileName() + "\"");
       }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DirectoryReader.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/DirectoryReader.java Mon Oct 31 17:59:59 2011
@@ -33,7 +33,6 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.Lock;
 import org.apache.lucene.store.LockObtainFailedException;
-import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -45,8 +44,6 @@ import org.apache.lucene.util.MapBackedS
 class DirectoryReader extends IndexReader implements Cloneable {
   protected Directory directory;
   protected boolean readOnly;
-  
-  protected CodecProvider codecs;
 
   IndexWriter writer;
 
@@ -78,14 +75,13 @@ class DirectoryReader extends IndexReade
 //  }
   
   static IndexReader open(final Directory directory, final IndexDeletionPolicy deletionPolicy, final IndexCommit commit, final boolean readOnly,
-                          final int termInfosIndexDivisor, final CodecProvider codecs) throws CorruptIndexException, IOException {
-    assert codecs != null;
+                          final int termInfosIndexDivisor) throws CorruptIndexException, IOException {
     return (IndexReader) new SegmentInfos.FindSegmentsFile(directory) {
       @Override
       protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
-        SegmentInfos infos = new SegmentInfos(codecs);
+        SegmentInfos infos = new SegmentInfos();
         infos.read(directory, segmentFileName);
-        return new DirectoryReader(directory, infos, deletionPolicy, readOnly, termInfosIndexDivisor, codecs);
+        return new DirectoryReader(directory, infos, deletionPolicy, readOnly, termInfosIndexDivisor);
       }
     }.run(commit);
   }
@@ -96,14 +92,12 @@ class DirectoryReader extends IndexReade
 //  }
   
   /** Construct reading the named set of readers. */
-  DirectoryReader(Directory directory, SegmentInfos sis, IndexDeletionPolicy deletionPolicy, boolean readOnly, int termInfosIndexDivisor, CodecProvider codecs) throws IOException {
+  DirectoryReader(Directory directory, SegmentInfos sis, IndexDeletionPolicy deletionPolicy, boolean readOnly, int termInfosIndexDivisor) throws IOException {
     this.directory = directory;
     this.readOnly = readOnly;
     this.segmentInfos = sis;
     this.deletionPolicy = deletionPolicy;
     this.termInfosIndexDivisor = termInfosIndexDivisor;
-    this.codecs = codecs;
-    assert codecs != null;
     readerFinishedListeners = new MapBackedSet<ReaderFinishedListener>(new ConcurrentHashMap<ReaderFinishedListener,Boolean>());
     applyAllDeletes = false;
 
@@ -137,14 +131,12 @@ class DirectoryReader extends IndexReade
   }
 
   // Used by near real-time search
-  DirectoryReader(IndexWriter writer, SegmentInfos infos, CodecProvider codecs, boolean applyAllDeletes) throws IOException {
+  DirectoryReader(IndexWriter writer, SegmentInfos infos, boolean applyAllDeletes) throws IOException {
     this.directory = writer.getDirectory();
     this.readOnly = true;
     this.applyAllDeletes = applyAllDeletes;       // saved for reopen
 
     this.termInfosIndexDivisor = writer.getConfig().getReaderTermsIndexDivisor();
-    this.codecs = codecs;
-    assert codecs != null;
     readerFinishedListeners = writer.getReaderFinishedListeners();
 
     // IndexWriter synchronizes externally before calling
@@ -193,8 +185,7 @@ class DirectoryReader extends IndexReade
 
   /** This constructor is only used for {@link #doOpenIfChanged()} */
   DirectoryReader(Directory directory, SegmentInfos infos, SegmentReader[] oldReaders,
-                  boolean readOnly, boolean doClone, int termInfosIndexDivisor, CodecProvider codecs,
-                  Collection<ReaderFinishedListener> readerFinishedListeners) throws IOException {
+                  boolean readOnly, boolean doClone, int termInfosIndexDivisor, Collection<ReaderFinishedListener> readerFinishedListeners) throws IOException {
     this.directory = directory;
     this.readOnly = readOnly;
     this.segmentInfos = infos;
@@ -202,9 +193,6 @@ class DirectoryReader extends IndexReade
     this.readerFinishedListeners = readerFinishedListeners;
     applyAllDeletes = false;
 
-    assert codecs != null;
-    this.codecs = codecs; 
-
     // we put the old SegmentReaders in a map, that allows us
     // to lookup a reader using its segment name
     Map<String,Integer> segmentReaders = new HashMap<String,Integer>();
@@ -336,7 +324,7 @@ class DirectoryReader extends IndexReade
     starts[subReaders.length] = maxDoc;
 
     if (!readOnly) {
-      maxIndexVersion = SegmentInfos.readCurrentVersion(directory, codecs);
+      maxIndexVersion = SegmentInfos.readCurrentVersion(directory);
     }
   }
 
@@ -487,7 +475,7 @@ class DirectoryReader extends IndexReade
     return (IndexReader) new SegmentInfos.FindSegmentsFile(directory) {
       @Override
       protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
-        final SegmentInfos infos = new SegmentInfos(codecs);
+        final SegmentInfos infos = new SegmentInfos();
         infos.read(directory, segmentFileName);
         return doOpenIfChanged(infos, false, openReadOnly);
       }
@@ -495,7 +483,7 @@ class DirectoryReader extends IndexReade
   }
 
   private synchronized DirectoryReader doOpenIfChanged(SegmentInfos infos, boolean doClone, boolean openReadOnly) throws CorruptIndexException, IOException {
-    return new DirectoryReader(directory, infos, subReaders, openReadOnly, doClone, termInfosIndexDivisor, codecs, readerFinishedListeners);
+    return new DirectoryReader(directory, infos, subReaders, openReadOnly, doClone, termInfosIndexDivisor, readerFinishedListeners);
   }
 
   /** Version number when this IndexReader was opened. */
@@ -701,7 +689,7 @@ class DirectoryReader extends IndexReade
 
         // we have to check whether index has changed since this reader was opened.
         // if so, this reader is no longer valid for deletion
-        if (SegmentInfos.readCurrentVersion(directory, codecs) > maxIndexVersion) {
+        if (SegmentInfos.readCurrentVersion(directory) > maxIndexVersion) {
           stale = true;
           this.writeLock.release();
           this.writeLock = null;
@@ -732,7 +720,7 @@ class DirectoryReader extends IndexReade
       // KeepOnlyLastCommitDeleter:
       IndexFileDeleter deleter = new IndexFileDeleter(directory,
                                                       deletionPolicy == null ? new KeepOnlyLastCommitDeletionPolicy() : deletionPolicy,
-                                                      segmentInfos, null, codecs, null);
+                                                      segmentInfos, null, null);
       segmentInfos.updateGeneration(deleter.getLastSegmentInfos());
       segmentInfos.changed();
 
@@ -753,7 +741,7 @@ class DirectoryReader extends IndexReade
 
         // Sync all files we just wrote
         directory.sync(segmentInfos.files(directory, false));
-        segmentInfos.commit(directory);
+        segmentInfos.commit(directory, segmentInfos.codecFormat());
         success = true;
       } finally {
 
@@ -831,7 +819,7 @@ class DirectoryReader extends IndexReade
     ensureOpen();
     if (writer == null || writer.isClosed()) {
       // we loaded SegmentInfos from the directory
-      return SegmentInfos.readCurrentVersion(directory, codecs) == segmentInfos.getVersion();
+      return SegmentInfos.readCurrentVersion(directory) == segmentInfos.getVersion();
     } else {
       return writer.nrtIsCurrent(segmentInfos);
     }
@@ -914,16 +902,11 @@ class DirectoryReader extends IndexReade
 
   /** @see org.apache.lucene.index.IndexReader#listCommits */
   public static List<IndexCommit> listCommits(Directory dir) throws IOException {
-    return listCommits(dir, CodecProvider.getDefault());
-  }
-
-  /** @see org.apache.lucene.index.IndexReader#listCommits */
-  public static List<IndexCommit> listCommits(Directory dir, CodecProvider codecs) throws IOException {
     final String[] files = dir.listAll();
 
     List<IndexCommit> commits = new ArrayList<IndexCommit>();
 
-    SegmentInfos latest = new SegmentInfos(codecs);
+    SegmentInfos latest = new SegmentInfos();
     latest.read(dir);
     final long currentGen = latest.getGeneration();
 
@@ -937,7 +920,7 @@ class DirectoryReader extends IndexReade
           !fileName.equals(IndexFileNames.SEGMENTS_GEN) &&
           SegmentInfos.generationFromSegmentsFileName(fileName) < currentGen) {
 
-        SegmentInfos sis = new SegmentInfos(codecs);
+        SegmentInfos sis = new SegmentInfos();
         try {
           // IOException allowed to throw there, in case
           // segments_N is corrupt

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java Mon Oct 31 17:59:59 2011
@@ -28,7 +28,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.NoSuchDirectoryException;
 import org.apache.lucene.util.CollectionUtil;
@@ -135,7 +134,7 @@ final class IndexFileDeleter {
    * @throws IOException if there is a low-level IO error
    */
   public IndexFileDeleter(Directory directory, IndexDeletionPolicy policy, SegmentInfos segmentInfos,
-                          PrintStream infoStream, CodecProvider codecs, IndexWriter writer) throws CorruptIndexException, IOException {
+                          PrintStream infoStream, IndexWriter writer) throws CorruptIndexException, IOException {
     this.infoStream = infoStream;
     this.writer = writer;
 
@@ -176,7 +175,7 @@ final class IndexFileDeleter {
           if (infoStream != null) {
             message("init: load commit \"" + fileName + "\"");
           }
-          SegmentInfos sis = new SegmentInfos(codecs);
+          SegmentInfos sis = new SegmentInfos();
           try {
             sis.read(directory, fileName);
           } catch (FileNotFoundException e) {
@@ -249,7 +248,7 @@ final class IndexFileDeleter {
       // listing was stale (eg when index accessed via NFS
       // client with stale directory listing cache).  So we
       // try now to explicitly open this commit point:
-      SegmentInfos sis = new SegmentInfos(codecs);
+      SegmentInfos sis = new SegmentInfos();
       try {
         sis.read(directory, currentSegmentsFile);
       } catch (IOException e) {

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexFileNameFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexFileNameFilter.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexFileNameFilter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexFileNameFilter.java Mon Oct 31 17:59:59 2011
@@ -21,7 +21,6 @@ import java.io.File;
 import java.io.FilenameFilter;
 import java.util.HashSet;
 import java.util.regex.Pattern;
-import org.apache.lucene.index.codecs.CodecProvider;
 
 /**
  * Filename filter that attempts to accept only filenames

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexReader.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexReader.java Mon Oct 31 17:59:59 2011
@@ -29,7 +29,6 @@ import java.util.concurrent.atomic.Atomi
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DocumentStoredFieldVisitor;
 import org.apache.lucene.index.codecs.PostingsFormat;
-import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.index.values.IndexDocValues;
 import org.apache.lucene.search.FieldCache; // javadocs
@@ -312,7 +311,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, CodecProvider.getDefault());
+    return open(directory, null, null, true, DEFAULT_TERMS_INDEX_DIVISOR);
   }
 
   /** Returns an IndexReader reading the index in the given
@@ -326,24 +325,9 @@ 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, CodecProvider.getDefault());
+    return open(directory, null, null, readOnly, DEFAULT_TERMS_INDEX_DIVISOR);
   }
   
-  /** Returns an IndexReader reading the index in the given
-   *  Directory.  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 readOnly true if no changes (deletions, norms) will be made with this IndexReader
-   * @param codecProvider {@link CodecProvider} to use for reading the 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, boolean readOnly, CodecProvider codecProvider) throws CorruptIndexException, IOException {
-    return open(directory, null, null, readOnly, DEFAULT_TERMS_INDEX_DIVISOR, codecProvider);
-  }
-
   /**
    * Open a near real time IndexReader from the {@link org.apache.lucene.index.IndexWriter}.
    *
@@ -378,7 +362,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, CodecProvider.getDefault());
+    return open(commit.getDirectory(), null, commit, readOnly, DEFAULT_TERMS_INDEX_DIVISOR);
   }
 
   /** Expert: returns an IndexReader reading the index in
@@ -396,7 +380,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, CodecProvider.getDefault());
+    return open(directory, deletionPolicy, null, readOnly, DEFAULT_TERMS_INDEX_DIVISOR);
   }
 
   /** Expert: returns an IndexReader reading the index in
@@ -424,7 +408,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, CodecProvider.getDefault());
+    return open(directory, deletionPolicy, null, readOnly, termInfosIndexDivisor);
   }
 
   /** Expert: returns an IndexReader reading the index in
@@ -444,7 +428,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, CodecProvider.getDefault());
+    return open(commit.getDirectory(), deletionPolicy, commit, readOnly, DEFAULT_TERMS_INDEX_DIVISOR);
   }
 
   /** Expert: returns an IndexReader reading the index in
@@ -477,76 +461,11 @@ 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, CodecProvider.getDefault());
+    return open(commit.getDirectory(), 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
-   *  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.
-   * @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
-   *  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.
-   * @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 {
-    assert codecs != null;
-    return DirectoryReader.open(directory, deletionPolicy, commit, readOnly, termInfosIndexDivisor, codecs);
+  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);
   }
 
   /**
@@ -756,20 +675,6 @@ public abstract class IndexReader implem
         }
       }.run()).longValue();
   }
-
-  /**
-   * 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 getCurrentVersion(directory, CodecProvider.getDefault());
-  }
   
   /**
    * Reads version number from segments files. The version number is
@@ -777,34 +682,14 @@ public abstract class IndexReader implem
    * the index.
    * 
    * @param directory where the index resides.
-   * @param codecs the {@link CodecProvider} holding all {@link PostingsFormat}s required to open the index
    * @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, CodecProvider codecs) throws CorruptIndexException, IOException {
-    return SegmentInfos.readCurrentVersion(directory, codecs);
-  }
-
-  /**
-   * 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 getCommitUserData(directory,  CodecProvider.getDefault());
+  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
@@ -820,8 +705,8 @@ public abstract class IndexReader implem
    *
    * @see #getCommitUserData()
    */
-  public static Map<String, String> getCommitUserData(Directory directory, CodecProvider codecs) throws CorruptIndexException, IOException {
-    return SegmentInfos.readCurrentUserData(directory, codecs);
+  public static Map<String, String> getCommitUserData(Directory directory) throws CorruptIndexException, IOException {
+    return SegmentInfos.readCurrentUserData(directory);
   }
 
   /**
@@ -966,19 +851,8 @@ public abstract class IndexReader implem
    * @throws IOException if there is a problem with accessing the index
    */
   public static boolean indexExists(Directory directory) throws IOException {
-    return indexExists(directory, CodecProvider.getDefault());
-  }
-
-  /**
-   * Returns <code>true</code> if an index exists at the specified directory.
-   * @param  directory the directory to check for an index
-   * @param  codecProvider provides a CodecProvider in case the index uses non-core codecs
-   * @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, CodecProvider codecProvider) throws IOException {
     try {
-      new SegmentInfos(codecProvider).read(directory);
+      new SegmentInfos().read(directory);
       return true;
     } catch (IOException ioe) {
       return false;

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java Mon Oct 31 17:59:59 2011
@@ -114,12 +114,12 @@ public final class IndexUpgrader {
   }
   
   public void upgrade() throws IOException {
-    if (!IndexReader.indexExists(dir, iwc.getCodecProvider())) {
+    if (!IndexReader.indexExists(dir)) {
       throw new IndexNotFoundException(dir.toString());
     }
   
     if (!deletePriorCommits) {
-      final Collection<IndexCommit> commits = DirectoryReader.listCommits(dir, iwc.getCodecProvider());
+      final Collection<IndexCommit> commits = DirectoryReader.listCommits(dir);
       if (commits.size() > 1) {
         throw new IllegalArgumentException("This tool was invoked to not delete prior commit points, but the following commits were found: " + commits);
       }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexWriter.java Mon Oct 31 17:59:59 2011
@@ -40,7 +40,6 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.index.PayloadProcessorProvider.DirPayloadProcessor;
 import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.CompoundFileDirectory;
@@ -375,7 +374,7 @@ public class IndexWriter implements Clos
         // just like we do when loading segments_N
         synchronized(this) {
           maybeApplyDeletes(applyAllDeletes);
-          r = new DirectoryReader(this, segmentInfos, codecs, applyAllDeletes);
+          r = new DirectoryReader(this, segmentInfos, applyAllDeletes);
           if (infoStream != null) {
             message("return reader version=" + r.getVersion() + " reader=" + r);
           }
@@ -802,8 +801,7 @@ public class IndexWriter implements Clos
       infoStream.println("IW " + messageID + " [" + new Date() + "; " + Thread.currentThread().getName() + "]: " + message);
   }
 
-  private final CodecProvider codecs; // for reading
-  final Codec defaultCodec; // for writing new segments
+  final Codec codec; // for writing new segments
 
   /**
    * Constructs a new IndexWriter per the settings given in <code>conf</code>.
@@ -838,7 +836,7 @@ public class IndexWriter implements Clos
     mergePolicy = conf.getMergePolicy();
     mergePolicy.setIndexWriter(this);
     mergeScheduler = conf.getMergeScheduler();
-    codecs = conf.getCodecProvider();
+    codec = conf.getCodec();
 
     bufferedDeletesStream = new BufferedDeletesStream(messageID);
     bufferedDeletesStream.setInfoStream(infoStream);
@@ -863,7 +861,7 @@ public class IndexWriter implements Clos
 
     // If index is too old, reading the segments will throw
     // IndexFormatTooOldException.
-    segmentInfos = new SegmentInfos(codecs);
+    segmentInfos = new SegmentInfos();
     try {
       if (create) {
         // Try to read first.  This is to allow create
@@ -893,7 +891,7 @@ public class IndexWriter implements Clos
           // points.
           if (commit.getDirectory() != directory)
             throw new IllegalArgumentException("IndexCommit's directory doesn't match my directory");
-          SegmentInfos oldInfos = new SegmentInfos(codecs);
+          SegmentInfos oldInfos = new SegmentInfos();
           oldInfos.read(directory, commit.getSegmentsFileName());
           segmentInfos.replace(oldInfos);
           changeCount++;
@@ -907,11 +905,7 @@ public class IndexWriter implements Clos
 
       // start with previous field numbers, but new FieldInfos
       globalFieldNumberMap = segmentInfos.getOrLoadGlobalFieldNumberMap(directory);
-      defaultCodec = codecs.getDefaultCodec();
-      if (defaultCodec == null) {
-        throw new IllegalArgumentException("CodecProvider returns null for getDefaultCodec()");
-      }
-      docWriter = new DocumentsWriter(defaultCodec, config, directory, this, globalFieldNumberMap, bufferedDeletesStream);
+      docWriter = new DocumentsWriter(codec, config, directory, this, globalFieldNumberMap, bufferedDeletesStream);
       docWriter.setInfoStream(infoStream);
 
       // Default deleter (for backwards compatibility) is
@@ -919,8 +913,7 @@ public class IndexWriter implements Clos
       synchronized(this) {
         deleter = new IndexFileDeleter(directory,
                                        conf.getIndexDeletionPolicy(),
-                                       segmentInfos, infoStream,
-                                       codecs, this);
+                                       segmentInfos, infoStream, this);
       }
 
       if (deleter.startingCommitDeleted) {
@@ -2481,7 +2474,7 @@ public class IndexWriter implements Clos
         if (infoStream != null) {
           message("addIndexes: process directory " + dir);
         }
-        SegmentInfos sis = new SegmentInfos(codecs); // read infos from dir
+        SegmentInfos sis = new SegmentInfos(); // read infos from dir
         sis.read(dir);
         final Set<String> dsFilesCopied = new HashSet<String>();
         final Map<String, String> dsNames = new HashMap<String, String>();
@@ -2572,7 +2565,7 @@ public class IndexWriter implements Clos
       // abortable so that IW.close(false) is able to stop it
       SegmentMerger merger = new SegmentMerger(directory, config.getTermIndexInterval(),
                                                mergedName, null, payloadProcessorProvider,
-                                               new FieldInfos(globalFieldNumberMap), defaultCodec, context);
+                                               new FieldInfos(globalFieldNumberMap), codec, context);
 
       for (IndexReader reader : readers)      // add new indexes
         merger.add(reader);
@@ -2921,7 +2914,7 @@ public class IndexWriter implements Clos
       try {
         if (infoStream != null)
     	  message("commit: pendingCommit != null");
-        pendingCommit.finishCommit(directory);
+        pendingCommit.finishCommit(directory, codec);
         if (infoStream != null)
           message("commit: wrote segments file \"" + pendingCommit.getCurrentSegmentFileName() + "\"");
         lastCommitChangeCount = pendingCommitChangeCount;
@@ -3638,7 +3631,7 @@ public class IndexWriter implements Clos
     IOContext context = new IOContext(merge.getMergeInfo());
 
     SegmentMerger merger = new SegmentMerger(directory, config.getTermIndexInterval(), mergedName, merge,
-                                             payloadProcessorProvider, merge.info.getFieldInfos(), defaultCodec, context);
+                                             payloadProcessorProvider, merge.info.getFieldInfos(), codec, context);
 
     if (infoStream != null) {
       message("merging " + merge.segString(directory) + " mergeVectors=" + merge.info.getFieldInfos().hasVectors());
@@ -3980,7 +3973,7 @@ public class IndexWriter implements Clos
           // Exception here means nothing is prepared
           // (this method unwinds everything it did on
           // an exception)
-          toSync.prepareCommit(directory);
+          toSync.prepareCommit(directory, codec);
 
           pendingCommitSet = true;
           pendingCommit = toSync;

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java Mon Oct 31 17:59:59 2011
@@ -20,7 +20,7 @@ package org.apache.lucene.index;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
 import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
-import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.similarities.SimilarityProvider;
 import org.apache.lucene.util.Version;
@@ -121,7 +121,7 @@ public final class IndexWriterConfig imp
   private volatile int maxBufferedDocs;
   private volatile IndexingChain indexingChain;
   private volatile IndexReaderWarmer mergedSegmentWarmer;
-  private volatile CodecProvider codecProvider;
+  private volatile Codec codec;
   private volatile MergePolicy mergePolicy;
   private volatile DocumentsWriterPerThreadPool indexerThreadPool;
   private volatile boolean readerPooling;
@@ -158,7 +158,7 @@ public final class IndexWriterConfig imp
     maxBufferedDocs = DEFAULT_MAX_BUFFERED_DOCS;
     indexingChain = DocumentsWriterPerThread.defaultIndexingChain;
     mergedSegmentWarmer = null;
-    codecProvider = CodecProvider.getDefault();
+    codec = Codec.getDefault();
     if (matchVersion.onOrAfter(Version.LUCENE_32)) {
       mergePolicy = new TieredMergePolicy();
     } else {
@@ -521,17 +521,17 @@ public final class IndexWriterConfig imp
     return this;
   }
 
-  /** Set the CodecProvider. See {@link CodecProvider}.
+  /** Set the Codec. See {@link Codec}.
    *
    * <p>Only takes effect when IndexWriter is first created. */
-  public IndexWriterConfig setCodecProvider(CodecProvider codecProvider) {
-    this.codecProvider = codecProvider;
+  public IndexWriterConfig setCodec(Codec codec) {
+    this.codec = codec;
     return this;
   }
 
-  /** Returns the current merged segment warmer. See {@link IndexReaderWarmer}. */
-  public CodecProvider getCodecProvider() {
-    return codecProvider;
+  /** Returns the current Codec. See {@link Codec}. */
+  public Codec getCodec() {
+    return codec;
   }
 
 
@@ -694,7 +694,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("codec=").append(codec).append("\n");
     sb.append("mergePolicy=").append(mergePolicy).append("\n");
     sb.append("indexerThreadPool=").append(indexerThreadPool).append("\n");
     sb.append("readerPooling=").append(readerPooling).append("\n");

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentInfo.java Mon Oct 31 17:59:59 2011
@@ -28,7 +28,6 @@ import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
@@ -177,7 +176,7 @@ public final class SegmentInfo implement
    * @param format format of the segments info file
    * @param input input handle to read segment info from
    */
-  public SegmentInfo(Directory dir, int format, IndexInput input, CodecProvider codecs) throws IOException {
+  public SegmentInfo(Directory dir, int format, IndexInput input) throws IOException {
     this.dir = dir;
     if (format <= DefaultSegmentInfosWriter.FORMAT_3_1) {
       version = input.readString();
@@ -221,16 +220,19 @@ public final class SegmentInfo implement
 
     hasProx = input.readByte();
 
+    // nocommit: who should handle the case of codec not found?
+    // Codec.forName() itself throw an exception? or callers check for null?
+    
     // System.out.println(Thread.currentThread().getName() + ": si.read hasProx=" + hasProx + " seg=" + name);
     if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
       String codecName = input.readString();
-      codec = codecs.lookup(codecName);
+      codec = Codec.forName(codecName);
       if (codec == null) {
         throw new IllegalArgumentException("Required codec '" + codecName + "' not found!");
       }
     } else {
       // TODO what todo if preflex is not available in the provider? register it or fail?
-      codec = codecs.lookup("Lucene3x");
+      codec = Codec.forName("Lucene3x");
     }
     diagnostics = input.readStringStringMap();
 

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentInfos.java Mon Oct 31 17:59:59 2011
@@ -32,7 +32,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
-import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
 import org.apache.lucene.index.codecs.SegmentInfosReader;
 import org.apache.lucene.index.codecs.SegmentInfosWriter;
@@ -83,8 +83,6 @@ public final class SegmentInfos implemen
                                    // there was an IOException that had interrupted a commit
 
   public Map<String,String> userData = Collections.<String,String>emptyMap();       // Opaque Map<String, String> that user can specify during IndexWriter.commit
-  
-  private final CodecProvider codecs;
 
   private int format;
   
@@ -95,16 +93,14 @@ public final class SegmentInfos implemen
   private transient List<SegmentInfo> cachedUnmodifiableList;
   private transient Set<SegmentInfo> cachedUnmodifiableSet;  
   
+  private Codec codecFormat;
+  
   /**
    * If non-null, information about loading segments_N files
    * will be printed here.  @see #setInfoStream.
    */
   private static PrintStream infoStream = null;
   
-  public SegmentInfos(CodecProvider codecs) {
-    this.codecs = codecs;
-  }
-
   public void setFormat(int format) {
     this.format = format;
   }
@@ -248,8 +244,10 @@ public final class SegmentInfos implemen
     lastGeneration = generation;
 
     try {
-      SegmentInfosReader infosReader = codecs.getSegmentInfosReader();
-      infosReader.read(directory, segmentFileName, codecs, this, IOContext.READ);
+      // nocommit: read preamble here then forward to codec
+      codecFormat = Codec.forName("Lucene40");
+      SegmentInfosReader infosReader = codecFormat.segmentInfosFormat().getSegmentInfosReader();
+      infosReader.read(directory, segmentFileName, this, IOContext.READ);
       success = true;
     }
     finally {
@@ -280,7 +278,7 @@ public final class SegmentInfos implemen
   // before finishCommit is called
   IndexOutput pendingSegnOutput;
 
-  private void write(Directory directory) throws IOException {
+  private void write(Directory directory, Codec codec) throws IOException {
 
     String segmentFileName = getNextSegmentFileName();
     final String globalFieldMapFile;
@@ -305,7 +303,7 @@ public final class SegmentInfos implemen
     boolean success = false;
 
     try {
-      SegmentInfosWriter infosWriter = codecs.getSegmentInfosWriter();
+      SegmentInfosWriter infosWriter = codec.segmentInfosFormat().getSegmentInfosWriter();
       segnOutput = infosWriter.writeInfos(directory, segmentFileName, this, IOContext.DEFAULT);
       infosWriter.prepareCommit(segnOutput);
       pendingSegnOutput = segnOutput;
@@ -392,7 +390,7 @@ public final class SegmentInfos implemen
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public static long readCurrentVersion(Directory directory, final CodecProvider codecs)
+  public static long readCurrentVersion(Directory directory)
     throws CorruptIndexException, IOException {
 
     // Fully read the segments file: this ensures that it's
@@ -400,7 +398,7 @@ public final class SegmentInfos implemen
     // IndexWriter.prepareCommit has been called (but not
     // yet commit), then the reader will still see itself as
     // current:
-    SegmentInfos sis = new SegmentInfos(codecs);
+    SegmentInfos sis = new SegmentInfos();
     sis.read(directory);
     return sis.version;
   }
@@ -410,9 +408,9 @@ public final class SegmentInfos implemen
    * @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, CodecProvider codecs)
+  public static Map<String,String> readCurrentUserData(Directory directory)
     throws CorruptIndexException, IOException {
-    SegmentInfos sis = new SegmentInfos(codecs);
+    SegmentInfos sis = new SegmentInfos();
     sis.read(directory);
     return sis.getUserData();
   }
@@ -791,10 +789,10 @@ public final class SegmentInfos implemen
    *  method if changes have been made to this {@link SegmentInfos} instance
    *  </p>  
    **/
-  final void prepareCommit(Directory dir) throws IOException {
+  final void prepareCommit(Directory dir, Codec codec) throws IOException {
     if (pendingSegnOutput != null)
       throw new IllegalStateException("prepareCommit was already called");
-    write(dir);
+    write(dir, codec);
   }
   
   private final long writeGlobalFieldMap(FieldNumberBiMap map, Directory dir, String name) throws IOException {
@@ -865,12 +863,12 @@ public final class SegmentInfos implemen
     return files;
   }
 
-  final void finishCommit(Directory dir) throws IOException {
+  final void finishCommit(Directory dir, Codec codec) throws IOException {
     if (pendingSegnOutput == null)
       throw new IllegalStateException("prepareCommit was not called");
     boolean success = false;
     try {
-      SegmentInfosWriter infosWriter = codecs.getSegmentInfosWriter();
+      SegmentInfosWriter infosWriter = codec.segmentInfosFormat().getSegmentInfosWriter();
       infosWriter.finishCommit(pendingSegnOutput);
       pendingSegnOutput = null;
       success = true;
@@ -941,9 +939,10 @@ public final class SegmentInfos implemen
    *  method if changes have been made to this {@link SegmentInfos} instance
    *  </p>  
    **/
-  final void commit(Directory dir) throws IOException {
-    prepareCommit(dir);
-    finishCommit(dir);
+  // nocommit: move this prepare+finish into SIwriter so codec controls the logic?
+  final void commit(Directory dir, Codec codec) throws IOException {
+    prepareCommit(dir, codec);
+    finishCommit(dir, codec);
   }
 
   public String toString(Directory directory) {
@@ -1103,6 +1102,14 @@ public final class SegmentInfos implemen
     this.addAll(infos);
   }
   
+  /**
+   * Returns the codec used to decode this SegmentInfos from disk 
+   * @lucene.internal
+   */
+  Codec codecFormat() {
+    return codecFormat;
+  }
+  
   /** Returns an <b>unmodifiable</b> {@link Iterator} of contained segments in order. */
   // @Override (comment out until Java 6)
   public Iterator<SegmentInfo> iterator() {

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java Mon Oct 31 17:59:59 2011
@@ -20,7 +20,6 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.FieldsWriter;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.ArrayUtil;

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/Codec.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/Codec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/Codec.java Mon Oct 31 17:59:59 2011
@@ -18,9 +18,13 @@ package org.apache.lucene.index.codecs;
  */
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.codecs.lucene3x.Lucene3xCodec;
+import org.apache.lucene.index.codecs.lucene40.Lucene40Codec;
 import org.apache.lucene.store.Directory;
 
 /**
@@ -51,4 +55,36 @@ public abstract class Codec {
   
   /** Encodes/decodes stored fields, term vectors, fieldinfos */
   public abstract FieldsFormat fieldsFormat();
+  
+  // nocommit: make abstract
+  public final SegmentInfosFormat segmentInfosFormat() {
+    return siFormat;
+  }
+  
+  // nocommit
+  private static final SegmentInfosFormat siFormat = new DefaultSegmentInfosFormat();
+  
+  /** looks up a codec by name */
+  public static Codec forName(String name) {
+    // TODO: Uwe fix this crap, this is just a stub!
+    return CORE_CODECS.get(name);
+  }
+  
+  @Deprecated
+  private static final Map<String,Codec> CORE_CODECS = new HashMap<String,Codec>();
+  static {
+    CORE_CODECS.put("Lucene40", new Lucene40Codec());
+    CORE_CODECS.put("Lucene3x", new Lucene3xCodec());
+  }
+  
+  private static Codec defaultCodec = Codec.forName("Lucene40");
+  
+  // nocommit: should we remove these? 
+  public static Codec getDefault() {
+    return defaultCodec;
+  }
+  
+  public static void setDefault(Codec codec) {
+    defaultCodec = codec;
+  }
 }

Added: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosFormat.java?rev=1195585&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosFormat.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosFormat.java Mon Oct 31 17:59:59 2011
@@ -0,0 +1,36 @@
+package org.apache.lucene.index.codecs;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * @lucene.experimental
+ */
+public class DefaultSegmentInfosFormat extends SegmentInfosFormat {
+  private static final SegmentInfosReader reader = new DefaultSegmentInfosReader();
+  private static final SegmentInfosWriter writer = new DefaultSegmentInfosWriter();
+  
+  @Override
+  public SegmentInfosReader getSegmentInfosReader() {
+    return reader;
+  }
+
+  @Override
+  public SegmentInfosWriter getSegmentInfosWriter() {
+    return writer;
+  }
+}

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java Mon Oct 31 17:59:59 2011
@@ -38,8 +38,7 @@ import org.apache.lucene.store.IndexInpu
 public class DefaultSegmentInfosReader extends SegmentInfosReader {
 
   @Override
-  public void read(Directory directory, String segmentsFileName, CodecProvider codecs,
-          SegmentInfos infos, IOContext context) throws IOException {
+  public void read(Directory directory, String segmentsFileName, SegmentInfos infos, IOContext context) throws IOException {
     IndexInput input = null;
     try {
       input = openInput(directory, segmentsFileName, context);
@@ -60,7 +59,7 @@ public class DefaultSegmentInfosReader e
         infos.setGlobalFieldMapVersion(input.readLong());
       }
       for (int i = input.readInt(); i > 0; i--) { // read segmentInfos
-        SegmentInfo si = new SegmentInfo(directory, format, input, codecs);
+        SegmentInfo si = new SegmentInfo(directory, format, input);
         if (si.getVersion() == null) {
           // Could be a 3.0 - try to open the doc stores - if it fails, it's a
           // 2.x segment, and an IndexFormatTooOldException will be thrown,

Added: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosFormat.java?rev=1195585&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosFormat.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosFormat.java Mon Oct 31 17:59:59 2011
@@ -0,0 +1,47 @@
+package org.apache.lucene.index.codecs;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Expert: Controls the format of the segments file.
+ * Note, this isn't a per-segment file, if you change the format, other versions
+ * of lucene won't be able to read it, yackedy schmackedy
+ * 
+ * @lucene.experimental
+ */
+// TODO: would be great to handle this situation better.
+// ideally a custom implementation could implement two-phase commit differently,
+// (e.g. atomic rename), and ideally all versions of lucene could still read it.
+// but this is just reflecting reality as it is today...
+//
+// also, perhaps the name should change (to cover all global files like .fnx?)
+// then again, maybe we can just remove that file...
+public abstract class SegmentInfosFormat {
+  public abstract SegmentInfosReader getSegmentInfosReader();
+  public abstract SegmentInfosWriter getSegmentInfosWriter();
+  
+  public static SegmentInfosFormat getDefault() {
+    return defaultFormat;
+  }
+  
+  public static void setDefault(SegmentInfosFormat format) {
+    defaultFormat = format;
+  }
+  
+  private static SegmentInfosFormat defaultFormat = new DefaultSegmentInfosFormat();
+}

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosReader.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosReader.java Mon Oct 31 17:59:59 2011
@@ -37,5 +37,5 @@ public abstract class SegmentInfosReader
    * @param infos empty instance to be populated with data
    * @throws IOException
    */
-  public abstract void read(Directory directory, String segmentsFileName, CodecProvider codecs, SegmentInfos infos, IOContext context) throws IOException;
+  public abstract void read(Directory directory, String segmentsFileName, SegmentInfos infos, IOContext context) throws IOException;
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/IndexDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/IndexDocValues.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/IndexDocValues.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/IndexDocValues.java Mon Oct 31 17:59:59 2011
@@ -24,8 +24,7 @@ import org.apache.lucene.document.IndexD
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.codecs.PostingsFormat;
-import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.DocValuesFormat;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -40,13 +39,11 @@ import org.apache.lucene.util.BytesRef;
  * performance {@link IndexDocValues} should be consumed per-segment just like
  * IndexReader.
  * <p>
- * {@link IndexDocValues} are fully integrated into the {@link PostingsFormat} API.
- * Custom implementations can be exposed on a per field basis via
- * {@link CodecProvider}.
+ * {@link IndexDocValues} are fully integrated into the {@link DocValuesFormat} API.
  * 
  * @see ValueType for limitations and default implementation documentation
  * @see IndexDocValuesField for adding values to the index
- * @see PostingsFormat#docsConsumer(org.apache.lucene.index.PerDocWriteState) for
+ * @see DocValuesFormat#docsConsumer(org.apache.lucene.index.PerDocWriteState) for
  *      customization
  * @lucene.experimental
  */

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Mon Oct 31 17:59:59 2011
@@ -37,7 +37,6 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.search.similarities.DefaultSimilarityProvider;
 import org.apache.lucene.search.similarities.SimilarityProvider;
 import org.apache.lucene.store.Directory;
@@ -126,22 +125,6 @@ public class IndexSearcher implements Cl
   public IndexSearcher(Directory path, boolean readOnly) throws CorruptIndexException, IOException {
     this(IndexReader.open(path, readOnly), true, null);
   }
-  
-  /** Creates a searcher searching the index in the named
-   *  directory.  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 underlying IndexReader.
-   * @param path directory where IndexReader will be opened
-   * @param readOnly if true, the underlying IndexReader
-   * will be opened readOnly
-   * @param codecProvider {@link CodecProvider} used for reading the index
-   * @throws CorruptIndexException if the index is corrupt
-   * @throws IOException if there is a low-level IO error
-   */
-  public IndexSearcher(Directory path, boolean readOnly, CodecProvider codecProvider) throws CorruptIndexException, IOException {
-    this(IndexReader.open(path, readOnly, codecProvider), true, null);
-  }
 
   /** Creates a searcher searching the provided index. */
   public IndexSearcher(IndexReader r) {

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java Mon Oct 31 17:59:59 2011
@@ -27,7 +27,7 @@ import org.apache.lucene.analysis.MockAn
 import org.apache.lucene.document.IndexDocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexWriter; // javadoc
-import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.values.ValueType;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.store.Directory;
@@ -53,7 +53,7 @@ public class RandomIndexWriter implement
   private final int fixedBytesLength;
   private final long docValuesFieldPrefix;
   private volatile boolean doDocValues;
-  private CodecProvider codecProvider;
+  private final Codec codec; // sugar
 
   // Randomly calls Thread.yield so we mixup thread scheduling
   private static final class MockIndexWriter extends IndexWriter {
@@ -96,9 +96,10 @@ public class RandomIndexWriter implement
     this.r = r;
     w = new MockIndexWriter(r, dir, c);
     flushAt = _TestUtil.nextInt(r, 10, 1000);
+    codec = w.getConfig().getCodec();
     if (LuceneTestCase.VERBOSE) {
       System.out.println("RIW config=" + w.getConfig());
-      System.out.println("codec default=" + w.getConfig().getCodecProvider().getDefaultCodec().getName());
+      System.out.println("codec default=" + codec.getName());
       w.setInfoStream(System.out);
     }
     /* TODO: find some what to make that random...
@@ -109,7 +110,6 @@ public class RandomIndexWriter implement
      */
     fixedBytesLength = 37; 
     docValuesFieldPrefix = r.nextLong();
-    codecProvider =  w.getConfig().getCodecProvider();
     switchDoDocValues();
   } 
 
@@ -171,7 +171,7 @@ public class RandomIndexWriter implement
     ValueType[] values = ValueType.values();
     ValueType type = values[random.nextInt(values.length)];
     String name = "random_" + type.name() + "" + docValuesFieldPrefix;
-    if ("Lucene3x".equals(codecProvider.getDefaultCodec().getName()) || doc.getField(name) != null)
+    if ("Lucene3x".equals(codec.getName()) || doc.getField(name) != null)
         return;
     IndexDocValuesField docValuesField = new IndexDocValuesField(name);
     switch (type) {
@@ -367,7 +367,7 @@ public class RandomIndexWriter implement
     // If we are writing with PreFlexRW, force a full
     // IndexReader.open so terms are sorted in codepoint
     // order during searching:
-    if (!applyDeletions || !w.defaultCodec.getName().equals("Lucene3x") && r.nextBoolean()) {
+    if (!applyDeletions || !codec.getName().equals("Lucene3x") && r.nextBoolean()) {
       if (LuceneTestCase.VERBOSE) {
         System.out.println("RIW.getReader: use NRT reader");
       }
@@ -382,7 +382,7 @@ public class RandomIndexWriter implement
       w.commit();
       switchDoDocValues();
       if (r.nextBoolean()) {
-        return IndexReader.open(w.getDirectory(), new KeepOnlyLastCommitDeletionPolicy(), r.nextBoolean(), _TestUtil.nextInt(r, 1, 10), w.getConfig().getCodecProvider());
+        return IndexReader.open(w.getDirectory(), new KeepOnlyLastCommitDeletionPolicy(), r.nextBoolean(), _TestUtil.nextInt(r, 1, 10));
       } else {
         return w.getReader(applyDeletions);
       }

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java Mon Oct 31 17:59:59 2011
@@ -33,7 +33,6 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.ThrottledIndexOutput;
 import org.apache.lucene.util._TestUtil;
@@ -486,14 +485,8 @@ public class MockDirectoryWrapper extend
       if (LuceneTestCase.VERBOSE) {
         System.out.println("\nNOTE: MockDirectoryWrapper: now run CheckIndex");
       } 
-      if (codecProvider != null) {
-        if (IndexReader.indexExists(this, codecProvider)) {
-          _TestUtil.checkIndex(this, codecProvider);
-        }
-      } else {
-        if (IndexReader.indexExists(this)) {
-          _TestUtil.checkIndex(this);
-        }
+      if (IndexReader.indexExists(this)) {
+        _TestUtil.checkIndex(this);
       }
     }
     delegate.close();
@@ -523,13 +516,6 @@ public class MockDirectoryWrapper extend
   public synchronized void removeIndexInput(IndexInput in, String name) {
     removeOpenFile(in, name);
   }
-  
-  private CodecProvider codecProvider;
-
-  // We pass this CodecProvider to checkIndex when dir is closed...
-  public void setCodecProvider(CodecProvider cp) {
-    codecProvider = cp;
-  }
 
   boolean open = true;
   

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java Mon Oct 31 17:59:59 2011
@@ -40,11 +40,10 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.*;
 import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.CoreCodecProvider;
 import org.apache.lucene.index.codecs.PostingsFormat;
-import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.lucene3x.Lucene3xCodec;
 import org.apache.lucene.index.codecs.lucene3x.Lucene3xPostingsFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40Codec;
 import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
 import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
 import org.apache.lucene.index.codecs.mocksep.MockSepPostingsFormat;
@@ -142,8 +141,6 @@ public abstract class LuceneTestCase ext
   // tests)
   /** Gets the postingsFormat to run tests with. */
   public static final String TEST_POSTINGSFORMAT = System.getProperty("tests.postingsformat", "random");
-  /** Gets the codecprovider to run tests with */
-  public static final String TEST_CODECPROVIDER = System.getProperty("tests.codecprovider", "random");
   /** Gets the locale to run tests with */
   public static final String TEST_LOCALE = System.getProperty("tests.locale", "random");
   /** Gets the timezone to run tests with */
@@ -213,8 +210,8 @@ public abstract class LuceneTestCase ext
   }
   private List<UncaughtExceptionEntry> uncaughtExceptions = Collections.synchronizedList(new ArrayList<UncaughtExceptionEntry>());
 
-  // default codec provider
-  private static CodecProvider savedCodecProvider;
+  // default codec
+  private static Codec savedCodec;
   
   private static SimilarityProvider similarityProvider;
 
@@ -253,8 +250,14 @@ public abstract class LuceneTestCase ext
       System.setProperty("solr.directoryFactory", "org.apache.solr.core.MockDirectoryFactory");
     }
     
-    savedCodecProvider = CodecProvider.getDefault();
-    final CodecProvider cp;
+    savedCodec = Codec.getDefault();
+    final Codec codec;
+    if ("Lucene3x".equals(TEST_POSTINGSFORMAT) || ("random".equals(TEST_POSTINGSFORMAT) && random.nextInt(4) == 0)) { // preflex-only setup
+      codec = new PreFlexRWCodec();
+    } else {
+      codec = new Lucene40Codec();
+    }
+    /* nocommit: we should randomize via the provider interface!! (via all codecs available in classpath)
     if ("random".equals(TEST_CODECPROVIDER)) {
       // TODO: kinda jaky that Lucene3x will eventually be a real codec, deal with this later.
       if ("Lucene3x".equals(TEST_POSTINGSFORMAT) || ("random".equals(TEST_POSTINGSFORMAT) && random.nextInt(4) == 0)) { // preflex-only setup
@@ -286,18 +289,10 @@ public abstract class LuceneTestCase ext
           }
         };
       }
-    } else {
-      // someone specified their own codecprovider by class
-      try {
-        Class<? extends CodecProvider> cpClazz = Class.forName(TEST_CODECPROVIDER).asSubclass(CodecProvider.class);
-        cp = cpClazz.newInstance();
-      } catch (Exception e) {
-        System.err.println("Could not instantiate CodecProvider: " + TEST_CODECPROVIDER);
-        throw new RuntimeException(e);
-      }
     }
+    */
 
-    CodecProvider.setDefault(cp);
+    Codec.setDefault(codec);
     
     savedLocale = Locale.getDefault();
     
@@ -347,8 +342,8 @@ public abstract class LuceneTestCase ext
       }
     }
     
-    String codecDescription = CodecProvider.getDefault().getDefaultCodec().toString();
-    CodecProvider.setDefault(savedCodecProvider);
+    String codecDescription = Codec.getDefault().toString();
+    Codec.setDefault(savedCodec);
     Locale.setDefault(savedLocale);
     TimeZone.setDefault(savedTimeZone);
     System.clearProperty("solr.solr.home");

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java?rev=1195585&r1=1195584&r2=1195585&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java Mon Oct 31 17:59:59 2011
@@ -45,13 +45,7 @@ import org.apache.lucene.index.MergePoli
 import org.apache.lucene.index.MergeScheduler;
 import org.apache.lucene.index.TieredMergePolicy;
 import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.CoreCodecProvider;
-import org.apache.lucene.index.codecs.DefaultDocValuesFormat;
-import org.apache.lucene.index.codecs.DefaultFieldsFormat;
-import org.apache.lucene.index.codecs.DocValuesFormat;
-import org.apache.lucene.index.codecs.FieldsFormat;
 import org.apache.lucene.index.codecs.PostingsFormat;
-import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.lucene40.Lucene40Codec;
 import org.apache.lucene.index.codecs.perfield.PerFieldPostingsFormat;
 import org.apache.lucene.search.FieldDoc;
@@ -152,17 +146,10 @@ public class _TestUtil {
    *  issues are hit, a RuntimeException is thrown; else,
    *  true is returned. */
   public static CheckIndex.Status checkIndex(Directory dir) throws IOException {
-    return checkIndex(dir, CodecProvider.getDefault());
-  }
-  
-  /** This runs the CheckIndex tool on the index in.  If any
-   *  issues are hit, a RuntimeException is thrown; else,
-   *  true is returned. */
-  public static CheckIndex.Status checkIndex(Directory dir, CodecProvider codecs) throws IOException {
     ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
     CheckIndex checker = new CheckIndex(dir);
     checker.setInfoStream(new PrintStream(bos), false);
-    CheckIndex.Status indexStatus = checker.checkIndex(null, codecs);
+    CheckIndex.Status indexStatus = checker.checkIndex(null);
     if (indexStatus == null || indexStatus.clean == false) {
       System.out.println("CheckIndex failed");
       System.out.println(bos.toString());
@@ -359,33 +346,13 @@ public class _TestUtil {
     return new String(buffer, 0, i);
   }
 
-  public static CodecProvider alwaysCodec(final Codec c) {
-    return new CoreCodecProvider() {
-      @Override
-      public Codec lookup(String name) {
-        //System.out.println(name);
-        // can't do this until we fix PreFlexRW to not
-        //impersonate PreFlex:
-        if (name.equals(c.getName())) {
-          return c;
-        } else {
-          return super.lookup(name);
-        }
-      }
-
-      @Override
-      public Codec getDefaultCodec() {
-        return c;
-      }
-    };
-  }
   
   /** Return a CodecProvider that can read any of the
    *  default codecs and formats, but always writes in the specified
    *  format. */
   // nocommit rename to .alwaysPostingsFormat?
-  public static CodecProvider alwaysFormat(final PostingsFormat format) {
-    return alwaysCodec(new Lucene40Codec() {
+  public static Codec alwaysFormat(final PostingsFormat format) {
+    return new Lucene40Codec() {
 
       @Override
       public PostingsFormat getPostingsFormat(String formatName) {
@@ -401,38 +368,12 @@ public class _TestUtil {
         return format.name;
       }
       
-    });
-  }
-
-  /** Returns a CodecProvider that can only read and write
-   *  the provided PostingsFormat, and uses default format
-   *  for DocValues and Fields. */
-  public static CodecProvider onlyFormat(final PostingsFormat postingsFormat) {
-
-    final Codec onlyCodec = new Codec("Only") {
-      private final FieldsFormat fieldsFormat = new DefaultFieldsFormat();
-      private final DocValuesFormat docValuesFormat = new DefaultDocValuesFormat();
-
-      @Override
-      public PostingsFormat postingsFormat() {
-        return postingsFormat;
-      }
-
-      @Override
-      public DocValuesFormat docValuesFormat() {
-        return docValuesFormat;
-      }                
-
-      @Override
-      public FieldsFormat fieldsFormat() {
-        return fieldsFormat;
-      }                
     };
-    return alwaysCodec(onlyCodec);
   }
-  
+
+  // nocommit, stupid
   public static String getPostingsFormat(String field) {
-    PostingsFormat p = CodecProvider.getDefault().getDefaultCodec().postingsFormat();
+    PostingsFormat p = Codec.getDefault().postingsFormat();
     if (p instanceof PerFieldPostingsFormat) {
       return ((PerFieldPostingsFormat)p).getPostingsFormatForField(field);
     } else {