You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by yo...@apache.org on 2011/11/05 13:47:05 UTC

svn commit: r1197949 [2/7] - in /lucene/dev/branches/solrcloud: ./ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/lucene/ dev-tools/idea/lucene/contrib/ dev-tools/idea/lucene/contrib/misc/ dev-tools/maven/ dev-tools/maven/lucene/contrib/misc/ ...

Modified: lucene/dev/branches/solrcloud/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingTermsIndexReader.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingTermsIndexReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/AppendingTermsIndexReader.java Sat Nov  5 12:46:53 2011
@@ -31,9 +31,9 @@ import org.apache.lucene.util.CodecUtil;
 public class AppendingTermsIndexReader extends FixedGapTermsIndexReader {
 
   public AppendingTermsIndexReader(Directory dir, FieldInfos fieldInfos,
-          String segment, int indexDivisor, Comparator<BytesRef> termComp, int codecId, IOContext context)
+          String segment, int indexDivisor, Comparator<BytesRef> termComp, String segmentSuffix, IOContext context)
           throws IOException {
-    super(dir, fieldInfos, segment, indexDivisor, termComp, codecId, context);
+    super(dir, fieldInfos, segment, indexDivisor, termComp, segmentSuffix, context);
   }
   
   @Override

Modified: lucene/dev/branches/solrcloud/lucene/contrib/misc/src/java/org/apache/lucene/store/DirectIOLinuxDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/contrib/misc/src/java/org/apache/lucene/store/DirectIOLinuxDirectory.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/contrib/misc/src/java/org/apache/lucene/store/DirectIOLinuxDirectory.java (original)
+++ lucene/dev/branches/solrcloud/lucene/contrib/misc/src/java/org/apache/lucene/store/DirectIOLinuxDirectory.java Sat Nov  5 12:46:53 2011
@@ -245,6 +245,7 @@ public class DirectIOLinuxDirectory exte
 
     public DirectIOLinuxIndexInput(File path, int bufferSize) throws IOException {
       // TODO make use of IOContext
+      super("DirectIOLinuxIndexInput(path=\"" + path.getPath() + "\")");
       FileDescriptor fd = NativePosixUtil.open_direct(path.toString(), true);
       fis = new FileInputStream(fd);
       channel = fis.getChannel();
@@ -259,6 +260,7 @@ public class DirectIOLinuxDirectory exte
 
     // for clone
     public DirectIOLinuxIndexInput(DirectIOLinuxIndexInput other) throws IOException {
+      super(other.toString());
       this.fis = null;
       channel = other.channel;
       this.bufferSize = other.bufferSize;
@@ -308,7 +310,7 @@ public class DirectIOLinuxDirectory exte
       try {
         return channel.size();
       } catch (IOException ioe) {
-        throw new RuntimeException(ioe);
+        throw new RuntimeException("IOException during length(): " + this, ioe);
       }
     }
 
@@ -331,9 +333,14 @@ public class DirectIOLinuxDirectory exte
       bufferPos = 0;
       assert (filePos & ALIGN_NOT_MASK) == filePos : "filePos=" + filePos + " anded=" + (filePos & ALIGN_NOT_MASK);
       //System.out.println("X refill filePos=" + filePos);
-      int n = channel.read(buffer, filePos);
+      int n;
+      try {
+        n = channel.read(buffer, filePos);
+      } catch (IOException ioe) {
+        throw new IOException(ioe.getMessage() + ": " + this, ioe);
+      }
       if (n < 0) {
-        throw new IOException("eof");
+        throw new IOException("eof: " + this);
       }
       buffer.rewind();
     }
@@ -365,7 +372,7 @@ public class DirectIOLinuxDirectory exte
       try {
         return new DirectIOLinuxIndexInput(this);
       } catch (IOException ioe) {
-        throw new RuntimeException(ioe);
+        throw new RuntimeException("IOException during clone: " + this, ioe);
       }
     }
   }

Modified: lucene/dev/branches/solrcloud/lucene/contrib/misc/src/java/org/apache/lucene/store/WindowsDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/contrib/misc/src/java/org/apache/lucene/store/WindowsDirectory.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/contrib/misc/src/java/org/apache/lucene/store/WindowsDirectory.java (original)
+++ lucene/dev/branches/solrcloud/lucene/contrib/misc/src/java/org/apache/lucene/store/WindowsDirectory.java Sat Nov  5 12:46:53 2011
@@ -19,6 +19,7 @@ package org.apache.lucene.store;
 
 import java.io.File;
 import java.io.IOException;
+import java.io.EOFException;
 
 import org.apache.lucene.store.Directory; // javadoc
 import org.apache.lucene.store.NativeFSLockFactory; // javadoc
@@ -80,7 +81,7 @@ public class WindowsDirectory extends FS
     boolean isOpen;
     
     public WindowsIndexInput(File file, int bufferSize) throws IOException {
-      super(bufferSize);
+      super("WindowsIndexInput(path=\"" + file.getPath() + "\")", bufferSize);
       fd = WindowsDirectory.open(file.getPath());
       length = WindowsDirectory.length(fd);
       isOpen = true;
@@ -88,8 +89,16 @@ public class WindowsDirectory extends FS
     
     @Override
     protected void readInternal(byte[] b, int offset, int length) throws IOException {
-      if (WindowsDirectory.read(fd, b, offset, length, getFilePointer()) != length)
-        throw new IOException("Read past EOF");
+      int bytesRead;
+      try {
+        bytesRead = WindowsDirectory.read(fd, b, offset, length, getFilePointer());
+      } catch (IOException ioe) {
+        throw new IOException(ioe.getMessage() + ": " + this, ioe);
+      }
+
+      if (bytesRead != length) {
+        throw new EOFException("Read past EOF: " + this);
+      }
     }
 
     @Override

Modified: lucene/dev/branches/solrcloud/lucene/contrib/misc/src/test/org/apache/lucene/index/TestNRTManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/contrib/misc/src/test/org/apache/lucene/index/TestNRTManager.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/contrib/misc/src/test/org/apache/lucene/index/TestNRTManager.java (original)
+++ lucene/dev/branches/solrcloud/lucene/contrib/misc/src/test/org/apache/lucene/index/TestNRTManager.java Sat Nov  5 12:46:53 2011
@@ -27,7 +27,9 @@ import org.apache.lucene.search.Searcher
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.NRTCachingDirectory;
+import org.apache.lucene.util.LuceneTestCase.UseNoMemoryExpensiveCodec;
 
+@UseNoMemoryExpensiveCodec
 public class TestNRTManager extends ThreadedIndexingAndSearchingTestCase {
 
   private final ThreadLocal<Long> lastGens = new ThreadLocal<Long>();

Modified: lucene/dev/branches/solrcloud/lucene/contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java (original)
+++ lucene/dev/branches/solrcloud/lucene/contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java Sat Nov  5 12:46:53 2011
@@ -34,11 +34,6 @@ import org.apache.lucene.index.MultiFiel
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.TermsEnum.SeekStatus;
-import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.CodecProvider;
-import org.apache.lucene.index.codecs.DefaultSegmentInfosReader;
-import org.apache.lucene.index.codecs.SegmentInfosReader;
-import org.apache.lucene.index.codecs.SegmentInfosWriter;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
@@ -50,30 +45,7 @@ import org.apache.lucene.util.Version;
 
 public class TestAppendingCodec extends LuceneTestCase {
   
-  static class AppendingCodecProvider extends CodecProvider {
-    Codec appending = new AppendingCodec();
-    SegmentInfosWriter infosWriter = new AppendingSegmentInfosWriter();
-    SegmentInfosReader infosReader = new DefaultSegmentInfosReader();
-    public AppendingCodecProvider() {
-      setDefaultFieldCodec(appending.name);
-    }
-    @Override
-    public Codec lookup(String name) {
-      return appending;
-    }
-   
-    @Override
-    public SegmentInfosReader getSegmentInfosReader() {
-      return infosReader;
-    }
-    @Override
-    public SegmentInfosWriter getSegmentInfosWriter() {
-      return infosWriter;
-    }
-    
-  }
-  
-  private static class AppendingIndexOutputWrapper extends IndexOutput {
+    private static class AppendingIndexOutputWrapper extends IndexOutput {
     IndexOutput wrapped;
     
     public AppendingIndexOutputWrapper(IndexOutput wrapped) {
@@ -137,7 +109,7 @@ public class TestAppendingCodec extends 
     Directory dir = new AppendingRAMDirectory(random, new RAMDirectory());
     IndexWriterConfig cfg = new IndexWriterConfig(Version.LUCENE_40, new MockAnalyzer(random));
     
-    cfg.setCodecProvider(new AppendingCodecProvider());
+    cfg.setCodec(new AppendingCodec());
     ((TieredMergePolicy)cfg.getMergePolicy()).setUseCompoundFile(false);
     IndexWriter writer = new IndexWriter(dir, cfg);
     Document doc = new Document();
@@ -151,7 +123,7 @@ public class TestAppendingCodec extends 
     writer.addDocument(doc);
     writer.optimize();
     writer.close();
-    IndexReader reader = IndexReader.open(dir, null, true, 1, new AppendingCodecProvider());
+    IndexReader reader = IndexReader.open(dir, null, true, 1);
     assertEquals(2, reader.numDocs());
     Document doc2 = reader.document(0);
     assertEquals(text, doc2.get("f"));

Modified: lucene/dev/branches/solrcloud/lucene/contrib/misc/src/test/org/apache/lucene/search/TestSearcherManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/contrib/misc/src/test/org/apache/lucene/search/TestSearcherManager.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/contrib/misc/src/test/org/apache/lucene/search/TestSearcherManager.java (original)
+++ lucene/dev/branches/solrcloud/lucene/contrib/misc/src/test/org/apache/lucene/search/TestSearcherManager.java Sat Nov  5 12:46:53 2011
@@ -34,9 +34,11 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.index.ThreadedIndexingAndSearchingTestCase;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase.UseNoMemoryExpensiveCodec;
 import org.apache.lucene.util.NamedThreadFactory;
 import org.apache.lucene.util._TestUtil;
 
+@UseNoMemoryExpensiveCodec
 public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
 
   boolean warmCalled;

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/CheckIndex.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/CheckIndex.java Sat Nov  5 12:46:53 2011
@@ -25,7 +25,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
 import java.io.File;
 import java.io.IOException;
@@ -143,8 +143,8 @@ public class CheckIndex {
       /** Name of the segment. */
       public String name;
 
-      /** CodecInfo used to read this segment. */
-      public SegmentCodecs codec;
+      /** Codec used to read this segment. */
+      public Codec codec;
 
       /** Document count (does not take deletions into account). */
       public int docCount;
@@ -322,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.
@@ -339,13 +335,13 @@ 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 {
-      sis.read(dir, codecs);
+      sis.read(dir);
     } catch (Throwable t) {
       msg("ERROR: could not read any segments file in directory");
       result.missingSegments = true;
@@ -377,6 +373,7 @@ public class CheckIndex {
 
     final int numSegments = sis.size();
     final String segmentsFileName = sis.getCurrentSegmentFileName();
+    // note: we only read the format byte (required preamble) here!
     IndexInput input = null;
     try {
       input = dir.openInput(segmentsFileName, IOContext.DEFAULT);
@@ -489,7 +486,7 @@ public class CheckIndex {
       SegmentReader reader = null;
 
       try {
-        final SegmentCodecs codec = info.getSegmentCodecs();
+        final Codec codec = info.getCodec();
         msg("    codec=" + codec);
         segInfoStat.codec = codec;
         msg("    compound=" + info.getUseCompoundFile());
@@ -1182,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;
@@ -1236,6 +1233,7 @@ public class CheckIndex {
   public static void main(String[] args) throws IOException, InterruptedException {
 
     boolean doFix = false;
+    Codec codec = Codec.getDefault(); // only used when fixing
     boolean verbose = false;
     List<String> onlySegments = new ArrayList<String>();
     String indexPath = null;
@@ -1244,6 +1242,13 @@ public class CheckIndex {
       if (args[i].equals("-fix")) {
         doFix = true;
         i++;
+      } else if (args[i].equals("-codec")) {
+        if (i == args.length-1) {
+          System.out.println("ERROR: missing name for -codec option");
+          System.exit(1);
+        }
+        codec = Codec.forName(args[i+1]);
+        i+=2;
       } else if (args[i].equals("-verbose")) {
         verbose = true;
         i++;
@@ -1269,6 +1274,7 @@ public class CheckIndex {
       System.out.println("\nUsage: java org.apache.lucene.index.CheckIndex pathToIndex [-fix] [-segment X] [-segment Y]\n" +
                          "\n" +
                          "  -fix: actually write a new segments_N file, removing any problematic segments\n" +
+                         "  -codec X: when fixing, codec to write the new segments_N file with\n" +
                          "  -verbose: print additional details\n" +
                          "  -segment X: only check the specified segments.  This can be specified multiple\n" + 
                          "              times, to check more than one segment, eg '-segment _2 -segment _a'.\n" +
@@ -1329,7 +1335,7 @@ public class CheckIndex {
           System.out.println("  " + (5-s) + "...");
         }
         System.out.println("Writing...");
-        checker.fixIndex(result);
+        checker.fixIndex(result, codec);
         System.out.println("OK");
         System.out.println("Wrote new segments file \"" + result.newSegments.getCurrentSegmentFileName() + "\"");
       }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DirectoryReader.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DirectoryReader.java Sat Nov  5 12:46:53 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,15 +75,13 @@ class DirectoryReader extends IndexReade
 //  }
   
   static IndexReader open(final Directory directory, final IndexDeletionPolicy deletionPolicy, final IndexCommit commit, final boolean readOnly,
-                          final int termInfosIndexDivisor, CodecProvider codecs) throws CorruptIndexException, IOException {
-    final CodecProvider codecProvider = codecs == null ? CodecProvider.getDefault()
-        : codecs;
+                          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(codecProvider);
-        infos.read(directory, segmentFileName, codecProvider);
-        return new DirectoryReader(directory, infos, deletionPolicy, readOnly, termInfosIndexDivisor, codecProvider);
+        SegmentInfos infos = new SegmentInfos();
+        infos.read(directory, segmentFileName);
+        return new DirectoryReader(directory, infos, deletionPolicy, readOnly, termInfosIndexDivisor);
       }
     }.run(commit);
   }
@@ -97,17 +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;
-    if (codecs == null) {
-      this.codecs = CodecProvider.getDefault();
-    } else {
-      this.codecs = codecs;
-    }
     readerFinishedListeners = new MapBackedSet<ReaderFinishedListener>(new ConcurrentHashMap<ReaderFinishedListener,Boolean>());
     applyAllDeletes = false;
 
@@ -141,17 +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();
-    if (codecs == null) {
-      this.codecs = CodecProvider.getDefault();
-    } else {
-      this.codecs = codecs;
-    }
     readerFinishedListeners = writer.getReaderFinishedListeners();
 
     // IndexWriter synchronizes externally before calling
@@ -200,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;
@@ -209,13 +193,6 @@ class DirectoryReader extends IndexReade
     this.readerFinishedListeners = readerFinishedListeners;
     applyAllDeletes = false;
 
-    if (codecs == null) {
-      this.codecs = CodecProvider.getDefault();
-    } else {
-      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>();
@@ -347,7 +324,7 @@ class DirectoryReader extends IndexReade
     starts[subReaders.length] = maxDoc;
 
     if (!readOnly) {
-      maxIndexVersion = SegmentInfos.readCurrentVersion(directory, codecs);
+      maxIndexVersion = SegmentInfos.readCurrentVersion(directory);
     }
   }
 
@@ -498,15 +475,15 @@ 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);
-        infos.read(directory, segmentFileName, codecs);
+        final SegmentInfos infos = new SegmentInfos();
+        infos.read(directory, segmentFileName);
         return doOpenIfChanged(infos, false, openReadOnly);
       }
     }.run(commit);
   }
 
   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. */
@@ -712,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;
@@ -743,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();
 
@@ -764,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 {
 
@@ -842,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);
     }
@@ -925,17 +902,12 @@ 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);
-    latest.read(dir, codecs);
+    SegmentInfos latest = new SegmentInfos();
+    latest.read(dir);
     final long currentGen = latest.getGeneration();
 
     commits.add(new ReaderCommit(latest, dir));
@@ -948,11 +920,11 @@ 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
-          sis.read(dir, fileName, codecs);
+          sis.read(dir, fileName);
         } catch (FileNotFoundException fnfe) {
           // LUCENE-948: on NFS (and maybe others), if
           // you have writers switching back and forth

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java Sat Nov  5 12:46:53 2011
@@ -25,7 +25,7 @@ import java.util.HashSet;
 import java.util.Map;
 
 import org.apache.lucene.index.DocumentsWriterPerThread.DocState;
-import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.DocValuesFormat;
 import org.apache.lucene.index.codecs.DocValuesConsumer;
 import org.apache.lucene.index.codecs.PerDocConsumer;
 import org.apache.lucene.index.values.PerDocFieldValues;
@@ -320,14 +320,13 @@ final class DocFieldProcessor extends Do
       docValuesConsumerAndDocID.docID = docState.docID;
       return docValuesConsumerAndDocID.docValuesConsumer;
     }
-    PerDocConsumer perDocConsumer = perDocConsumers.get(fieldInfo.getCodecId());
+
+    PerDocConsumer perDocConsumer = perDocConsumers.get(0);
     if (perDocConsumer == null) {
-      PerDocWriteState perDocWriteState = docState.docWriter.newPerDocWriteState(fieldInfo.getCodecId());
-      SegmentCodecs codecs = perDocWriteState.segmentCodecs;
-      assert codecs.codecs.length > fieldInfo.getCodecId();
-      Codec codec = codecs.codecs[fieldInfo.getCodecId()];
-      perDocConsumer = codec.docsConsumer(perDocWriteState);
-      perDocConsumers.put(Integer.valueOf(fieldInfo.getCodecId()), perDocConsumer);
+      PerDocWriteState perDocWriteState = docState.docWriter.newPerDocWriteState("");
+      DocValuesFormat dvFormat = docState.docWriter.codec.docValuesFormat();
+      perDocConsumer = dvFormat.docsConsumer(perDocWriteState);
+      perDocConsumers.put(0, perDocConsumer);
     }
     boolean success = false;
     DocValuesConsumer docValuesConsumer = null;

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java Sat Nov  5 12:46:53 2011
@@ -31,6 +31,7 @@ import org.apache.lucene.index.Documents
 import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
 import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
 import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
+import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.similarities.SimilarityProvider;
 import org.apache.lucene.store.AlreadyClosedException;
@@ -116,7 +117,14 @@ final class DocumentsWriter {
 
   // TODO: cut over to BytesRefHash in BufferedDeletes
   volatile DocumentsWriterDeleteQueue deleteQueue = new DocumentsWriterDeleteQueue();
-  private final Queue<FlushTicket> ticketQueue = new LinkedList<DocumentsWriter.FlushTicket>();
+  private final TicketQueue ticketQueue = new TicketQueue();
+  /*
+   * we preserve changes during a full flush since IW might not checkout before
+   * we release all changes. NRT Readers otherwise suddenly return true from
+   * isCurrent while there are actually changes currently committed. See also
+   * #anyChanges() & #flushAllThreads
+   */
+  private volatile boolean pendingChangesInCurrentFullFlush;
 
   private Collection<String> abortedFiles;               // List of files that were written before last abort()
 
@@ -125,8 +133,11 @@ final class DocumentsWriter {
   final DocumentsWriterPerThreadPool perThreadPool;
   final FlushPolicy flushPolicy;
   final DocumentsWriterFlushControl flushControl;
-  DocumentsWriter(IndexWriterConfig config, Directory directory, IndexWriter writer, FieldNumberBiMap globalFieldNumbers,
+  
+  final Codec codec;
+  DocumentsWriter(Codec codec, IndexWriterConfig config, Directory directory, IndexWriter writer, FieldNumberBiMap globalFieldNumbers,
       BufferedDeletesStream bufferedDeletesStream) throws IOException {
+    this.codec = codec;
     this.directory = directory;
     this.indexWriter = writer;
     this.similarityProvider = config.getSimilarityProvider();
@@ -166,6 +177,7 @@ final class DocumentsWriter {
   private void applyAllDeletes(DocumentsWriterDeleteQueue deleteQueue) throws IOException {
     if (deleteQueue != null && !flushControl.isFullFlush()) {
       synchronized (ticketQueue) {
+        ticketQueue.incTicketCount();// first inc the ticket count - freeze opens a window for #anyChanges to fail
         // Freeze and insert the delete flush ticket in the queue
         ticketQueue.add(new FlushTicket(deleteQueue.freezeGlobalBuffer(null), false));
         applyFlushTickets();
@@ -252,9 +264,22 @@ final class DocumentsWriter {
   }
 
   boolean anyChanges() {
-    return numDocsInRAM.get() != 0 || anyDeletions();
+    if (infoStream != null) {
+      message("docWriter: anyChanges? numDocsInRam=" + numDocsInRAM.get()
+          + " deletes=" + anyDeletions() + " hasTickets:"
+          + ticketQueue.hasTickets() + " pendingChangesInFullFlush: "
+          + pendingChangesInCurrentFullFlush);
+    }
+    /*
+     * changes are either in a DWPT or in the deleteQueue.
+     * yet if we currently flush deletes and / or dwpt there
+     * could be a window where all changes are in the ticket queue
+     * before they are published to the IW. ie we need to check if the 
+     * ticket queue has any tickets.
+     */
+    return numDocsInRAM.get() != 0 || anyDeletions() || ticketQueue.hasTickets() || pendingChangesInCurrentFullFlush;
   }
-
+  
   public int getBufferedDeleteTermsSize() {
     return deleteQueue.getBufferedDeleteTermsSize();
   }
@@ -279,7 +304,7 @@ final class DocumentsWriter {
     if (flushControl.anyStalledThreads() || flushControl.numQueuedFlushes() > 0) {
       // Help out flushing any queued DWPTs so we can un-stall:
       if (infoStream != null) {
-        message("DocumentsWriter has queued dwpt; will hijack this thread to flush pending segment(s)");
+        message("docWriter: DocumentsWriter has queued dwpt; will hijack this thread to flush pending segment(s)");
       }
       do {
         // Try pick up pending threads here if possible
@@ -413,7 +438,7 @@ final class DocumentsWriter {
           synchronized (ticketQueue) {
             // Each flush is assigned a ticket in the order they acquire the ticketQueue lock
             ticket =  new FlushTicket(flushingDWPT.prepareFlush(), true);
-            ticketQueue.add(ticket);
+            ticketQueue.incrementAndAdd(ticket);
           }
   
           // flush concurrently without locking
@@ -470,8 +495,11 @@ final class DocumentsWriter {
         // Keep publishing eligible flushed segments:
         final FlushTicket head = ticketQueue.peek();
         if (head != null && head.canPublish()) {
-          ticketQueue.poll();
-          finishFlush(head.segment, head.frozenDeletes);
+          try {
+            finishFlush(head.segment, head.frozenDeletes);
+          } finally {
+            ticketQueue.poll();
+          }
         } else {
           break;
         }
@@ -485,7 +513,7 @@ final class DocumentsWriter {
     if (newSegment == null) {
       assert bufferedDeletes != null;
       if (bufferedDeletes != null && bufferedDeletes.any()) {
-        indexWriter.bufferedDeletesStream.push(bufferedDeletes);
+        indexWriter.publishFrozenDeletes(bufferedDeletes);
         if (infoStream != null) {
           message("flush: push buffered deletes: " + bufferedDeletes);
         }
@@ -531,6 +559,7 @@ final class DocumentsWriter {
   
   // for asserts
   private volatile DocumentsWriterDeleteQueue currentFullFlushDelQueue = null;
+
   // for asserts
   private synchronized boolean setFlushingDeleteQueue(DocumentsWriterDeleteQueue session) {
     currentFullFlushDelQueue = session;
@@ -550,6 +579,7 @@ final class DocumentsWriter {
     }
     
     synchronized (this) {
+      pendingChangesInCurrentFullFlush = anyChanges();
       flushingDeleteQueue = deleteQueue;
       /* Cutover to a new delete queue.  This must be synced on the flush control
        * otherwise a new DWPT could sneak into the loop with an already flushing
@@ -569,11 +599,12 @@ final class DocumentsWriter {
       }
       // If a concurrent flush is still in flight wait for it
       flushControl.waitForFlush();  
-      if (!anythingFlushed) { // apply deletes if we did not flush any document
+      if (!anythingFlushed && flushingDeleteQueue.anyChanges()) { // apply deletes if we did not flush any document
         if (infoStream != null) {
          message(Thread.currentThread().getName() + ": flush naked frozen global deletes");
         }
         synchronized (ticketQueue) {
+          ticketQueue.incTicketCount(); // first inc the ticket count - freeze opens a window for #anyChanges to fail
           ticketQueue.add(new FlushTicket(flushingDeleteQueue.freezeGlobalBuffer(null), false));
         }
         applyFlushTickets();
@@ -586,16 +617,21 @@ final class DocumentsWriter {
   }
   
   final void finishFullFlush(boolean success) {
-    if (infoStream != null) {
-      message(Thread.currentThread().getName() + " finishFullFlush success=" + success);
-    }
-    assert setFlushingDeleteQueue(null);
-    if (success) {
-      // Release the flush lock
-      flushControl.finishFullFlush();
-    } else {
-      flushControl.abortFullFlushes();
+    try {
+      if (infoStream != null) {
+        message(Thread.currentThread().getName() + " finishFullFlush success=" + success);
+      }
+      assert setFlushingDeleteQueue(null);
+      if (success) {
+        // Release the flush lock
+        flushControl.finishFullFlush();
+      } else {
+        flushControl.abortFullFlushes();
+      }
+    } finally {
+      pendingChangesInCurrentFullFlush = false;
     }
+    
   }
 
   static final class FlushTicket {
@@ -614,6 +650,46 @@ final class DocumentsWriter {
     }
   }
   
+  static final class TicketQueue {
+    private final Queue<FlushTicket> queue = new LinkedList<FlushTicket>();
+    final AtomicInteger ticketCount = new AtomicInteger();
+    
+    void incTicketCount() {
+      ticketCount.incrementAndGet();
+    }
+    
+    public boolean hasTickets() {
+      assert ticketCount.get() >= 0;
+      return ticketCount.get() != 0;
+    }
+
+    void incrementAndAdd(FlushTicket ticket) {
+      incTicketCount();
+      add(ticket);
+    }
+    
+    void add(FlushTicket ticket) {
+      queue.add(ticket);
+    }
+    
+    FlushTicket peek() {
+      return queue.peek();
+    }
+    
+    FlushTicket poll() {
+      try {
+        return queue.poll();
+      } finally {
+        ticketCount.decrementAndGet();
+      }
+    }
+    
+    void clear() {
+      queue.clear();
+      ticketCount.set(0);
+    }
+  }
+  
   // use by IW during close to assert all DWPT are inactive after final flush
   boolean assertNoActiveDWPT() {
     Iterator<ThreadState> activePerThreadsIterator = perThreadPool.getAllPerThreadsIterator();

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Sat Nov  5 12:46:53 2011
@@ -26,7 +26,7 @@ import java.text.NumberFormat;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice;
-import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.search.similarities.SimilarityProvider;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FlushInfo;
@@ -152,7 +152,7 @@ public class DocumentsWriterPerThread {
   }
   private final static boolean INFO_VERBOSE = false;
   final DocumentsWriter parent;
-  final CodecProvider codecProvider;
+  final Codec codec;
   final IndexWriter writer;
   final Directory directory;
   final DocState docState;
@@ -183,7 +183,7 @@ public class DocumentsWriterPerThread {
     this.fieldInfos = fieldInfos;
     this.writer = parent.indexWriter;
     this.infoStream = parent.infoStream;
-    this.codecProvider = this.writer.codecs;
+    this.codec = parent.codec;
     this.docState = new DocState(this);
     this.docState.similarityProvider = parent.indexWriter.getConfig()
         .getSimilarityProvider();
@@ -405,8 +405,8 @@ public class DocumentsWriterPerThread {
     return numDocsInRAM;
   }
 
-  SegmentCodecs getCodec() {
-    return flushState.segmentCodecs;
+  Codec getCodec() {
+    return flushState.codec;
   }
 
   /** Reset after a flush */
@@ -443,7 +443,7 @@ public class DocumentsWriterPerThread {
     assert deleteSlice == null : "all deletes must be applied in prepareFlush";
     flushState = new SegmentWriteState(infoStream, directory, segment, fieldInfos,
         numDocsInRAM, writer.getConfig().getTermIndexInterval(),
-        fieldInfos.buildSegmentCodecs(true), pendingDeletes, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed())));
+        codec, pendingDeletes, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed())));
     final double startMBUsed = parent.flushControl.netBytes() / 1024. / 1024.;
     // Apply delete-by-docID now (delete-byDocID only
     // happens when an exception is hit processing that
@@ -474,12 +474,12 @@ public class DocumentsWriterPerThread {
     try {
       consumer.flush(flushState);
       pendingDeletes.terms.clear();
-      final SegmentInfo newSegment = new SegmentInfo(segment, flushState.numDocs, directory, false, flushState.segmentCodecs, fieldInfos.asReadOnly());
+      final SegmentInfo newSegment = new SegmentInfo(segment, flushState.numDocs, directory, false, flushState.codec, fieldInfos.asReadOnly());
       if (infoStream != null) {
         message("new segment has " + (flushState.liveDocs == null ? 0 : (flushState.numDocs - flushState.liveDocs.count())) + " deleted docs");
         message("new segment has " + (newSegment.getHasVectors() ? "vectors" : "no vectors"));
         message("flushedFiles=" + newSegment.files());
-        message("flushed codecs=" + newSegment.getSegmentCodecs());
+        message("flushed codec=" + newSegment.getCodec());
       }
       flushedDocCount += flushState.numDocs;
 
@@ -556,9 +556,9 @@ public class DocumentsWriterPerThread {
     bytesUsed.addAndGet(-(length *(INT_BLOCK_SIZE*RamUsageEstimator.NUM_BYTES_INT)));
   }
 
-  PerDocWriteState newPerDocWriteState(int codecId) {
+  PerDocWriteState newPerDocWriteState(String segmentSuffix) {
     assert segment != null;
-    return new PerDocWriteState(infoStream, directory, segment, fieldInfos, bytesUsed, codecId, IOContext.DEFAULT);
+    return new PerDocWriteState(infoStream, directory, segment, fieldInfos, bytesUsed, segmentSuffix, IOContext.DEFAULT);
   }
   
   void setInfoStream(PrintStream infoStream) {

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java Sat Nov  5 12:46:53 2011
@@ -20,8 +20,6 @@ import java.util.Iterator;
 import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
-import org.apache.lucene.index.SegmentCodecs.SegmentCodecsBuilder;
-import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.util.SetOnce;
 
 /**
@@ -128,7 +126,6 @@ public abstract class DocumentsWriterPer
 
   private final ThreadState[] perThreads;
   private volatile int numThreadStatesActive;
-  private CodecProvider codecProvider;
   private FieldNumberBiMap globalFieldMap;
   private final SetOnce<DocumentsWriter> documentsWriter = new SetOnce<DocumentsWriter>();
   
@@ -148,11 +145,9 @@ public abstract class DocumentsWriterPer
 
   public void initialize(DocumentsWriter documentsWriter, FieldNumberBiMap globalFieldMap, IndexWriterConfig config) {
     this.documentsWriter.set(documentsWriter); // thread pool is bound to DW
-    final CodecProvider codecs = config.getCodecProvider();
-    this.codecProvider = codecs;
     this.globalFieldMap = globalFieldMap;
     for (int i = 0; i < perThreads.length; i++) {
-      final FieldInfos infos = globalFieldMap.newFieldInfos(SegmentCodecsBuilder.create(codecs));
+      final FieldInfos infos = new FieldInfos(globalFieldMap);
       perThreads[i] = new ThreadState(new DocumentsWriterPerThread(documentsWriter.directory, documentsWriter, infos, documentsWriter.chain));
     }
   }
@@ -240,7 +235,7 @@ public abstract class DocumentsWriterPer
     assert threadState.isHeldByCurrentThread();
     final DocumentsWriterPerThread dwpt = threadState.perThread;
     if (!closed) {
-      final FieldInfos infos = globalFieldMap.newFieldInfos(SegmentCodecsBuilder.create(codecProvider));
+      final FieldInfos infos = new FieldInfos(globalFieldMap);
       final DocumentsWriterPerThread newDwpt = new DocumentsWriterPerThread(dwpt, infos);
       newDwpt.initialize();
       threadState.resetWriter(newDwpt);

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FieldInfo.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FieldInfo.java Sat Nov  5 12:46:53 2011
@@ -21,7 +21,6 @@ import org.apache.lucene.index.values.Va
 
 /** @lucene.experimental */
 public final class FieldInfo {
-  public static final int UNASSIGNED_CODEC_ID = -1;
   public final String name;
   public final int number;
 
@@ -38,7 +37,6 @@ public final class FieldInfo {
   public IndexOptions indexOptions;
 
   public boolean storePayloads; // whether this field stores payloads together with term positions
-  private int codecId = UNASSIGNED_CODEC_ID; // set inside SegmentCodecs#build() during segment flush - this is used to identify the codec used to write this field
 
   /**
    * Controls how much information is stored in the postings lists.
@@ -77,21 +75,11 @@ public final class FieldInfo {
     }
     assert indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS || !storePayloads;
   }
-
-  void setCodecId(int codecId) {
-    assert this.codecId == UNASSIGNED_CODEC_ID : "CodecId can only be set once.";
-    this.codecId = codecId;
-  }
-
-  public int getCodecId() {
-    return codecId;
-  }
   
   @Override
   public Object clone() {
     FieldInfo clone = new FieldInfo(name, isIndexed, number, storeTermVector, storePositionWithTermVector,
                          storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions, docValues);
-    clone.codecId = this.codecId;
     return clone;
   }
 

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FieldInfos.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FieldInfos.java Sat Nov  5 12:46:53 2011
@@ -29,9 +29,6 @@ import java.util.TreeMap;
 import java.util.Map.Entry;
 
 import org.apache.lucene.index.FieldInfo.IndexOptions;
-import org.apache.lucene.index.SegmentCodecs; // Required for Java 1.5 javadocs
-import org.apache.lucene.index.SegmentCodecs.SegmentCodecsBuilder;
-import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.values.ValueType;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -146,17 +143,6 @@ public final class FieldInfos implements
     }
     
     /**
-     * Returns a new {@link FieldInfos} instance with this as the global field
-     * map
-     * 
-     * @return a new {@link FieldInfos} instance with this as the global field
-     *         map
-     */
-    public FieldInfos newFieldInfos(SegmentCodecsBuilder segmentCodecsBuilder) {
-      return new FieldInfos(this, segmentCodecsBuilder);
-    }
-
-    /**
      * Returns <code>true</code> iff the last committed version differs from the
      * current version, otherwise <code>false</code>
      * 
@@ -198,7 +184,6 @@ public final class FieldInfos implements
   private final SortedMap<Integer,FieldInfo> byNumber = new TreeMap<Integer,FieldInfo>();
   private final HashMap<String,FieldInfo> byName = new HashMap<String,FieldInfo>();
   private final FieldNumberBiMap globalFieldNumbers;
-  private final SegmentCodecsBuilder segmentCodecsBuilder;
   
   // First used in 2.9; prior to 2.9 there was no format header
   public static final int FORMAT_START = -2;
@@ -230,16 +215,15 @@ public final class FieldInfos implements
 
   /**
    * Creates a new {@link FieldInfos} instance with a private
-   * {@link org.apache.lucene.index.FieldInfos.FieldNumberBiMap} and a default {@link SegmentCodecsBuilder}
-   * initialized with {@link CodecProvider#getDefault()}.
+   * {@link org.apache.lucene.index.FieldInfos.FieldNumberBiMap} 
    * <p>
    * Note: this ctor should not be used during indexing use
    * {@link FieldInfos#FieldInfos(FieldInfos)} or
-   * {@link FieldInfos#FieldInfos(FieldNumberBiMap,org.apache.lucene.index.SegmentCodecs.SegmentCodecsBuilder)}
+   * {@link FieldInfos#FieldInfos(FieldNumberBiMap)}
    * instead.
    */
   public FieldInfos() {
-    this(new FieldNumberBiMap(), SegmentCodecsBuilder.create(CodecProvider.getDefault()));
+    this(new FieldNumberBiMap());
   }
   
   /**
@@ -249,7 +233,7 @@ public final class FieldInfos implements
    * @see #isReadOnly()
    */
   FieldInfos(FieldInfos other) {
-    this(other.globalFieldNumbers, other.segmentCodecsBuilder);
+    this(other.globalFieldNumbers);
   }
   
   /**
@@ -257,9 +241,8 @@ public final class FieldInfos implements
    * If the {@link FieldNumberBiMap} is <code>null</code> this instance will be read-only.
    * @see #isReadOnly()
    */
-  FieldInfos(FieldNumberBiMap globalFieldNumbers, SegmentCodecsBuilder segmentCodecsBuilder) {
+  FieldInfos(FieldNumberBiMap globalFieldNumbers) {
     this.globalFieldNumbers = globalFieldNumbers;
-    this.segmentCodecsBuilder = segmentCodecsBuilder;
   }
 
   /**
@@ -273,7 +256,7 @@ public final class FieldInfos implements
    * @throws IOException
    */
   public FieldInfos(Directory d, String name) throws IOException {
-    this((FieldNumberBiMap)null, null); // use null here to make this FIs Read-Only
+    this((FieldNumberBiMap)null); // use null here to make this FIs Read-Only
     final IndexInput input = d.openInput(name, IOContext.READONCE);
     try {
       read(input, name);
@@ -309,7 +292,7 @@ public final class FieldInfos implements
    */
   @Override
   synchronized public Object clone() {
-    FieldInfos fis = new FieldInfos(globalFieldNumbers, segmentCodecsBuilder);
+    FieldInfos fis = new FieldInfos(globalFieldNumbers);
     fis.format = format;
     fis.hasFreq = hasFreq;
     fis.hasProx = hasProx;
@@ -468,7 +451,6 @@ public final class FieldInfos implements
     if (globalFieldNumbers == null) {
       throw new IllegalStateException("FieldInfos are read-only, create a new instance with a global field map to make modifications to FieldInfos");
     }
-    assert segmentCodecsBuilder != null : "SegmentCodecsBuilder is set to null but FieldInfos is not read-only";
     FieldInfo fi = fieldInfo(name);
     if (fi == null) {
       final int fieldNumber = nextFieldNumber(name, preferredFieldNumber);
@@ -477,9 +459,6 @@ public final class FieldInfos implements
       fi.update(isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions);
       fi.setDocValues(docValues);
     }
-    if ((fi.isIndexed || fi.hasDocValues()) && fi.getCodecId() == FieldInfo.UNASSIGNED_CODEC_ID) {
-      segmentCodecsBuilder.tryAddAndSet(fi);
-    }
     version++;
     return fi;
   }
@@ -569,22 +548,6 @@ public final class FieldInfos implements
     }
     return false;
   }
-  
-  /**
-   * Builds the {@link SegmentCodecs} mapping for this {@link FieldInfos} instance.
-   * @param clearBuilder <code>true</code> iff the internal {@link SegmentCodecsBuilder} must be cleared otherwise <code>false</code>
-   */
-  public SegmentCodecs buildSegmentCodecs(boolean clearBuilder) {
-    if (globalFieldNumbers == null) {
-      throw new IllegalStateException("FieldInfos are read-only no SegmentCodecs available");
-    }
-    assert segmentCodecsBuilder != null;
-    final SegmentCodecs segmentCodecs = segmentCodecsBuilder.build();
-    if (clearBuilder) {
-      segmentCodecsBuilder.clear();
-    }
-    return segmentCodecs;
-  }
 
   public void write(Directory d, String name) throws IOException {
     IndexOutput output = d.createOutput(name, IOContext.READONCE);
@@ -628,7 +591,6 @@ public final class FieldInfos implements
         bits |= OMIT_POSITIONS;
       output.writeString(fi.name);
       output.writeInt(fi.number);
-      output.writeInt(fi.getCodecId());
       output.writeByte(bits);
 
       final byte b;
@@ -688,19 +650,17 @@ public final class FieldInfos implements
     format = input.readVInt();
 
     if (format > FORMAT_MINIMUM) {
-      throw new IndexFormatTooOldException(fileName, format, FORMAT_MINIMUM, FORMAT_CURRENT);
+      throw new IndexFormatTooOldException(input, format, FORMAT_MINIMUM, FORMAT_CURRENT);
     }
     if (format < FORMAT_CURRENT) {
-      throw new IndexFormatTooNewException(fileName, format, FORMAT_MINIMUM, FORMAT_CURRENT);
+      throw new IndexFormatTooNewException(input, format, FORMAT_MINIMUM, FORMAT_CURRENT);
     }
 
     final int size = input.readVInt(); //read in the size
 
     for (int i = 0; i < size; i++) {
       String name = input.readString();
-      // if this is a previous format codec 0 will be preflex!
       final int fieldNumber = format <= FORMAT_FLEX? input.readInt():i;
-      final int codecId = format <= FORMAT_FLEX? input.readInt():0;
       byte bits = input.readByte();
       boolean isIndexed = (bits & IS_INDEXED) != 0;
       boolean storeTermVector = (bits & STORE_TERMVECTOR) != 0;
@@ -715,7 +675,7 @@ public final class FieldInfos implements
         if (format <= FORMAT_OMIT_POSITIONS) {
           indexOptions = IndexOptions.DOCS_AND_FREQS;
         } else {
-          throw new CorruptIndexException("Corrupt fieldinfos, OMIT_POSITIONS set but format=" + format);
+          throw new CorruptIndexException("Corrupt fieldinfos, OMIT_POSITIONS set but format=" + format + " (resource: " + input + ")");
         }
       } else {
         indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
@@ -781,12 +741,11 @@ public final class FieldInfos implements
           throw new IllegalStateException("unhandled indexValues type " + b);
         }
       }
-      final FieldInfo addInternal = addInternal(name, fieldNumber, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions, docValuesType);
-      addInternal.setCodecId(codecId);
+      addInternal(name, fieldNumber, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions, docValuesType);
     }
 
     if (input.getFilePointer() != input.length()) {
-      throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length());
+      throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")");
     }    
   }
   
@@ -804,7 +763,7 @@ public final class FieldInfos implements
     if (isReadOnly()) {
       return this;
     }
-    final FieldInfos roFis = new FieldInfos((FieldNumberBiMap)null, null);
+    final FieldInfos roFis = new FieldInfos((FieldNumberBiMap)null);
     for (FieldInfo fieldInfo : this) {
       FieldInfo clone = (FieldInfo) (fieldInfo).clone();
       roFis.putInternal(clone);
@@ -814,5 +773,14 @@ public final class FieldInfos implements
     }
     return roFis;
   }
-  
+
+  public boolean anyDocValuesFields() {
+    for (FieldInfo fi : this) {
+      if (fi.hasDocValues()) { 
+        return true;
+      }
+    }
+
+    return false;
+  }
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java Sat Nov  5 12:46:53 2011
@@ -57,7 +57,7 @@ final class FreqProxTermsWriter extends 
     // Sort by field name
     CollectionUtil.quickSort(allFields);
 
-    final FieldsConsumer consumer = state.segmentCodecs.codec().fieldsConsumer(state);
+    final FieldsConsumer consumer = state.codec.postingsFormat().fieldsConsumer(state);
 
     boolean success = false;
 

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java Sat Nov  5 12:46:53 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 import java.io.FileNotFoundException;
-import java.io.FilenameFilter;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.ArrayList;
@@ -29,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;
@@ -122,8 +120,6 @@ final class IndexFileDeleter {
     infoStream.println("IFD [" + new Date() + "; " + Thread.currentThread().getName() + "]: " + message);
   }
 
-  private final FilenameFilter indexFilenameFilter;
-
   // called only from assert
   private boolean locked() {
     return writer == null || Thread.holdsLock(writer);
@@ -138,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;
 
@@ -154,7 +150,6 @@ final class IndexFileDeleter {
     // First pass: walk the files and initialize our ref
     // counts:
     long currentGen = segmentInfos.getGeneration();
-    indexFilenameFilter = new IndexFileNameFilter(codecs);
 
     CommitPoint currentCommitPoint = null;
     String[] files = null;
@@ -167,7 +162,7 @@ final class IndexFileDeleter {
 
     for (String fileName : files) {
 
-      if ((indexFilenameFilter.accept(null, fileName)) && !fileName.endsWith("write.lock") && !fileName.equals(IndexFileNames.SEGMENTS_GEN)) {
+      if ((IndexFileNameFilter.INSTANCE.accept(null, fileName)) && !fileName.endsWith("write.lock") && !fileName.equals(IndexFileNames.SEGMENTS_GEN)) {
 
         // Add this file to refCounts with initial count 0:
         getRefCount(fileName);
@@ -180,9 +175,9 @@ 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, codecs);
+            sis.read(directory, fileName);
           } catch (FileNotFoundException e) {
             // LUCENE-948: on NFS (and maybe others), if
             // you have writers switching back and forth
@@ -253,9 +248,9 @@ 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, codecs);
+        sis.read(directory, currentSegmentsFile);
       } catch (IOException e) {
         throw new CorruptIndexException("failed to locate current segments_N file");
       }
@@ -373,7 +368,7 @@ final class IndexFileDeleter {
     for(int i=0;i<files.length;i++) {
       String fileName = files[i];
       if ((segmentName == null || fileName.startsWith(segmentPrefix1) || fileName.startsWith(segmentPrefix2)) &&
-          indexFilenameFilter.accept(null, fileName) &&
+          IndexFileNameFilter.INSTANCE.accept(null, fileName) &&
           !refCounts.containsKey(fileName) &&
           !fileName.equals(IndexFileNames.SEGMENTS_GEN)) {
         // Unreferenced file, so remove it

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileNameFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileNameFilter.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileNameFilter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileNameFilter.java Sat Nov  5 12:46:53 2011
@@ -20,50 +20,43 @@ package org.apache.lucene.index;
 import java.io.File;
 import java.io.FilenameFilter;
 import java.util.HashSet;
-import org.apache.lucene.index.codecs.CodecProvider;
+import java.util.regex.Pattern;
 
 /**
- * Filename filter that accept filenames and extensions only
- * created by Lucene.
+ * Filename filter that attempts to accept only filenames
+ * created by Lucene.  Note that this is a "best effort"
+ * process.  If a file is used in a Lucene index, it will
+ * always match the file; but if a file is not used in a
+ * Lucene index but is named in a similar way to Lucene's
+ * files then this filter may accept the file.
+ *
+ * <p>This does not accept <code>*-write.lock</code> files.
  *
  * @lucene.internal
  */
 
 public class IndexFileNameFilter implements FilenameFilter {
 
-  private final HashSet<String> extensions;
-
-  public IndexFileNameFilter(CodecProvider codecs) {
-    extensions = new HashSet<String>();
-    for (String ext : IndexFileNames.INDEX_EXTENSIONS) {
-      extensions.add(ext);
-    }
-    if (codecs != null) {
-      for(String ext : codecs.getAllExtensions()) {
-        extensions.add(ext);
-      }
-    }
+  public static final FilenameFilter INSTANCE = new IndexFileNameFilter();
+  
+  private IndexFileNameFilter() {
   }
 
+  // Approximate match for files that seem to be Lucene
+  // index files.  This can easily over-match, ie if some
+  // app names a file _foo_bar.go:
+  private final Pattern luceneFilePattern = Pattern.compile("^_[a-z0-9]+(_[a-z0-9]+)?\\.[a-z0-9]+$");
+
   /* (non-Javadoc)
    * @see java.io.FilenameFilter#accept(java.io.File, java.lang.String)
    */
   public boolean accept(File dir, String name) {
-    int i = name.lastIndexOf('.');
-    if (i != -1) {
-      String extension = name.substring(1+i);
-      if (extensions.contains(extension)) {
-        return true;
-      } else if (extension.startsWith("f") &&
-                 extension.matches("f\\d+")) {
-        return true;
-      } else if (extension.startsWith("s") &&
-                 extension.matches("s\\d+")) {
-        return true;
-      }
+    if (name.lastIndexOf('.') != -1) {
+      // Has an extension
+      return luceneFilePattern.matcher(name).matches();
     } else {
-      if (name.startsWith(IndexFileNames.SEGMENTS)) return true;
+      // No extension -- only segments_N file;
+      return name.startsWith(IndexFileNames.SEGMENTS);
     }
-    return false;
   }
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileNames.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileNames.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileNames.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFileNames.java Sat Nov  5 12:46:53 2011
@@ -19,7 +19,9 @@ package org.apache.lucene.index;
 
 import java.util.regex.Pattern;
 
-import org.apache.lucene.index.codecs.Codec;  // for javadocs
+import org.apache.lucene.index.codecs.PostingsFormat;  // for javadocs
+
+// TODO: put all files under codec and remove all the static extensions here
 
 /**
  * This class contains useful constants representing filenames and extensions
@@ -31,7 +33,7 @@ import org.apache.lucene.index.codecs.Co
  * {@link #segmentFileName(String, String, String) segmentFileName}).
  *
  * <p><b>NOTE</b>: extensions used by codecs are not
- * listed here.  You must interact with the {@link Codec}
+ * listed here.  You must interact with the {@link PostingsFormat}
  * directly.
  *
  * @lucene.internal
@@ -188,20 +190,20 @@ public final class IndexFileNames {
    * <b>NOTE:</b> .&lt;ext&gt; is added to the result file name only if
    * <code>ext</code> is not empty.
    * <p>
-   * <b>NOTE:</b> _&lt;name&gt; is added to the result file name only if
-   * <code>name</code> is not empty.
+   * <b>NOTE:</b> _&lt;segmentSuffix&gt; is added to the result file name only if
+   * it's not the empty string
    * <p>
    * <b>NOTE:</b> all custom files should be named using this method, or
    * otherwise some structures may fail to handle them properly (such as if they
    * are added to compound files).
    */
-  public static String segmentFileName(String segmentName, String name, String ext) {
-    if (ext.length() > 0 || name.length() > 0) {
+  public static String segmentFileName(String segmentName, String segmentSuffix, String ext) {
+    if (ext.length() > 0 || segmentSuffix.length() > 0) {
       assert !ext.startsWith(".");
-      StringBuilder sb = new StringBuilder(segmentName.length() + 2 + name.length() + ext.length());
+      StringBuilder sb = new StringBuilder(segmentName.length() + 2 + segmentSuffix.length() + ext.length());
       sb.append(segmentName);
-      if (name.length() > 0) {
-        sb.append('_').append(name);
+      if (segmentSuffix.length() > 0) {
+        sb.append('_').append(segmentSuffix);
       }
       if (ext.length() > 0) {
         sb.append('.').append(ext);
@@ -212,11 +214,6 @@ public final class IndexFileNames {
     }
   }
 
-  /** Sugar for passing "" + name instead */
-  public static String segmentFileName(String segmentName, int name, String ext) {
-    return segmentFileName(segmentName, ""+name, ext);
-  }
-
   /**
    * Returns true if the given filename ends with the given extension. One
    * should provide a <i>pure</i> extension, without '.'.

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFormatTooNewException.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFormatTooNewException.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFormatTooNewException.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFormatTooNewException.java Sat Nov  5 12:46:53 2011
@@ -17,15 +17,24 @@
 
 package org.apache.lucene.index;
 
+import org.apache.lucene.store.DataInput;
+
 /**
  * This exception is thrown when Lucene detects
  * an index that is newer than this Lucene version.
  */
 public class IndexFormatTooNewException extends CorruptIndexException {
 
-  public IndexFormatTooNewException(String filename, int version, int minVersion, int maxVersion) {
-    super("Format version is not supported" + (filename!=null ? (" in file '" + filename + "'") : "") +
-      ": " + version + " (needs to be between " + minVersion + " and " + maxVersion + ")");
+  /** @lucene.internal */
+  public IndexFormatTooNewException(String resourceDesc, int version, int minVersion, int maxVersion) {
+    super("Format version is not supported (resource: " + resourceDesc + "): "
+      + version + " (needs to be between " + minVersion + " and " + maxVersion + ")");
+    assert resourceDesc != null;
+  }
+
+  /** @lucene.internal */
+  public IndexFormatTooNewException(DataInput in, int version, int minVersion, int maxVersion) {
+    this(in.toString(), version, minVersion, maxVersion);
   }
 
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFormatTooOldException.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFormatTooOldException.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFormatTooOldException.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexFormatTooOldException.java Sat Nov  5 12:46:53 2011
@@ -17,21 +17,36 @@
 
 package org.apache.lucene.index;
 
+import org.apache.lucene.store.DataInput;
+
 /**
  * This exception is thrown when Lucene detects
  * an index that is too old for this Lucene version
  */
 public class IndexFormatTooOldException extends CorruptIndexException {
 
-  public IndexFormatTooOldException(String filename, String version) {
-    super("Format version is not supported" + (filename!=null ? (" in file '" + filename + "'") : "") +
-        ": " + version + ". This version of Lucene only supports indexes created with release 3.0 and later.");
+  /** @lucene.internal */
+  public IndexFormatTooOldException(String resourceDesc, String version) {
+    super("Format version is not supported (resource: " + resourceDesc + "): " +
+        version + ". This version of Lucene only supports indexes created with release 3.0 and later.");
+    assert resourceDesc != null;
+  }
+
+  /** @lucene.internal */
+  public IndexFormatTooOldException(DataInput in, String version) {
+    this(in.toString(), version);
   }
   
-  public IndexFormatTooOldException(String filename, int version, int minVersion, int maxVersion) {
-    super("Format version is not supported" + (filename!=null ? (" in file '" + filename + "'") : "") +
-        ": " + version + " (needs to be between " + minVersion + " and " + maxVersion +
+  /** @lucene.internal */
+  public IndexFormatTooOldException(String resourceDesc, int version, int minVersion, int maxVersion) {
+    super("Format version is not supported (resource: " + resourceDesc + "): " +
+        version + " (needs to be between " + minVersion + " and " + maxVersion +
     "). This version of Lucene only supports indexes created with release 3.0 and later.");
+    assert resourceDesc != null;
   }
 
+  /** @lucene.internal */
+  public IndexFormatTooOldException(DataInput in, int version, int minVersion, int maxVersion) {
+    this(in.toString(), version, minVersion, maxVersion);
+  }
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexReader.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexReader.java Sat Nov  5 12:46:53 2011
@@ -28,8 +28,7 @@ import java.util.concurrent.atomic.Atomi
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DocumentStoredFieldVisitor;
-import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.PostingsFormat;
 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, null);
+    return open(directory, null, null, true, DEFAULT_TERMS_INDEX_DIVISOR);
   }
 
   /** Returns an IndexReader reading the index in the given
@@ -326,9 +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, null);
+    return open(directory, null, null, readOnly, DEFAULT_TERMS_INDEX_DIVISOR);
   }
-
+  
   /**
    * Open a near real time IndexReader from the {@link org.apache.lucene.index.IndexWriter}.
    *
@@ -363,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, null);
+    return open(commit.getDirectory(), null, commit, readOnly, DEFAULT_TERMS_INDEX_DIVISOR);
   }
 
   /** Expert: returns an IndexReader reading the index in
@@ -381,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, null);
+    return open(directory, deletionPolicy, null, readOnly, DEFAULT_TERMS_INDEX_DIVISOR);
   }
 
   /** Expert: returns an IndexReader reading the index in
@@ -409,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, null);
+    return open(directory, deletionPolicy, null, readOnly, termInfosIndexDivisor);
   }
 
   /** Expert: returns an IndexReader reading the index in
@@ -429,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, null);
+    return open(commit.getDirectory(), deletionPolicy, commit, readOnly, DEFAULT_TERMS_INDEX_DIVISOR);
   }
 
   /** Expert: returns an IndexReader reading the index in
@@ -462,78 +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, null);
-  }
-
-  /** 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);
+    return open(commit.getDirectory(), deletionPolicy, commit, readOnly, termInfosIndexDivisor);
   }
 
-  /** 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 {
-    if (codecs == null) {
-      codecs = CodecProvider.getDefault();
-    }
-    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);
   }
 
   /**
@@ -767,20 +699,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
@@ -788,34 +706,14 @@ public abstract class IndexReader implem
    * the index.
    * 
    * @param directory where the index resides.
-   * @param codecs the {@link CodecProvider} holding all {@link Codec}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
@@ -824,15 +722,14 @@ public abstract class IndexReader implem
    * this index.
    * 
    * @param directory where the index resides.
-   * @param codecs the {@link CodecProvider} provider holding all {@link Codec}s required to open the index
    * @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, 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);
   }
 
   /**
@@ -985,22 +882,6 @@ public abstract class IndexReader implem
     }
   }
 
-  /**
-   * 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().read(directory, codecProvider);
-      return true;
-    } catch (IOException ioe) {
-      return false;
-    }
-  }
-
   /** Returns the number of documents in this index. */
   public abstract int numDocs();
 

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java?rev=1197949&r1=1197948&r2=1197949&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java Sat Nov  5 12:46:53 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);
       }