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/11/04 16:43:43 UTC

svn commit: r1197603 [2/5] - in /lucene/dev/trunk: ./ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/lucene/ dev-tools/idea/lucene/contrib/ dev-tools/idea/lucene/contrib/misc/ lucene/ lucene/contrib/ lucene/contrib/memory/src/test/org/apache/l...

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java Fri Nov  4 15:43:35 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/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java Fri Nov  4 15:43:35 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;
@@ -125,8 +126,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();

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Fri Nov  4 15:43:35 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/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java Fri Nov  4 15:43:35 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/trunk/lucene/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfo.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfo.java Fri Nov  4 15:43:35 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/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java Fri Nov  4 15:43:35 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;
@@ -698,9 +660,7 @@ public final class FieldInfos implements
 
     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;
@@ -781,8 +741,7 @@ 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()) {
@@ -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/trunk/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java Fri Nov  4 15:43:35 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/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java Fri Nov  4 15:43:35 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/trunk/lucene/src/java/org/apache/lucene/index/IndexFileNameFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileNameFilter.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileNameFilter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileNameFilter.java Fri Nov  4 15:43:35 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/trunk/lucene/src/java/org/apache/lucene/index/IndexFileNames.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileNames.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileNames.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileNames.java Fri Nov  4 15:43:35 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/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexReader.java Fri Nov  4 15:43:35 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/trunk/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexUpgrader.java Fri Nov  4 15:43:35 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/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java Fri Nov  4 15:43:35 2011
@@ -39,8 +39,7 @@ import org.apache.lucene.index.Documents
 import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.index.PayloadProcessorProvider.DirPayloadProcessor;
-import org.apache.lucene.index.SegmentCodecs.SegmentCodecsBuilder;
-import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.Codec;
 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,7 +801,7 @@ public class IndexWriter implements Clos
       infoStream.println("IW " + messageID + " [" + new Date() + "; " + Thread.currentThread().getName() + "]: " + message);
   }
 
-  CodecProvider codecs;
+  final Codec codec; // for writing new segments
 
   /**
    * Constructs a new IndexWriter per the settings given in <code>conf</code>.
@@ -837,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);
@@ -862,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
@@ -870,7 +869,7 @@ public class IndexWriter implements Clos
         // searching.  In this case we write the next
         // segments_N file with no segments:
         try {
-          segmentInfos.read(directory, codecs);
+          segmentInfos.read(directory);
           segmentInfos.clear();
         } catch (IOException e) {
           // Likely this means it's a fresh directory
@@ -881,7 +880,7 @@ public class IndexWriter implements Clos
         changeCount++;
         segmentInfos.changed();
       } else {
-        segmentInfos.read(directory, codecs);
+        segmentInfos.read(directory);
 
         IndexCommit commit = conf.getIndexCommit();
         if (commit != null) {
@@ -892,8 +891,8 @@ 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);
-          oldInfos.read(directory, commit.getSegmentsFileName(), codecs);
+          SegmentInfos oldInfos = new SegmentInfos();
+          oldInfos.read(directory, commit.getSegmentsFileName());
           segmentInfos.replace(oldInfos);
           changeCount++;
           segmentInfos.changed();
@@ -906,7 +905,7 @@ public class IndexWriter implements Clos
 
       // start with previous field numbers, but new FieldInfos
       globalFieldNumberMap = segmentInfos.getOrLoadGlobalFieldNumberMap(directory);
-      docWriter = new DocumentsWriter(config, directory, this, globalFieldNumberMap, bufferedDeletesStream);
+      docWriter = new DocumentsWriter(codec, config, directory, this, globalFieldNumberMap, bufferedDeletesStream);
       docWriter.setInfoStream(infoStream);
 
       // Default deleter (for backwards compatibility) is
@@ -914,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) {
@@ -2149,6 +2147,7 @@ public class IndexWriter implements Clos
    */
   public synchronized void deleteAll() throws IOException {
     ensureOpen();
+    boolean success = false;
     try {
 
       // Abort any running merges
@@ -2170,10 +2169,11 @@ public class IndexWriter implements Clos
       // Mark that the index has changed
       ++changeCount;
       segmentInfos.changed();
+      success = true;
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "deleteAll");
     } finally {
-      if (infoStream != null) {
+      if (!success && infoStream != null) {
         message("hit exception during deleteAll");
       }
     }
@@ -2476,8 +2476,8 @@ public class IndexWriter implements Clos
         if (infoStream != null) {
           message("addIndexes: process directory " + dir);
         }
-        SegmentInfos sis = new SegmentInfos(codecs); // read infos from dir
-        sis.read(dir, codecs);
+        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>();
         for (SegmentInfo info : sis) {
@@ -2567,7 +2567,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,
-                                               globalFieldNumberMap.newFieldInfos(SegmentCodecsBuilder.create(codecs)), context);
+                                               new FieldInfos(globalFieldNumberMap), codec, context);
 
       for (IndexReader reader : readers)      // add new indexes
         merger.add(reader);
@@ -2575,7 +2575,7 @@ public class IndexWriter implements Clos
 
       final FieldInfos fieldInfos = merger.fieldInfos();
       SegmentInfo info = new SegmentInfo(mergedName, docCount, directory,
-                                         false, merger.getSegmentCodecs(),
+                                         false, merger.getCodec(),
                                          fieldInfos);
       setDiagnostics(info, "addIndexes(IndexReader...)");
 
@@ -2591,7 +2591,7 @@ public class IndexWriter implements Clos
 
       // Now create the compound file if needed
       if (useCompoundFile) {
-        merger.createCompoundFile(mergedName + ".cfs", info, context);
+        merger.createCompoundFile(IndexFileNames.segmentFileName(mergedName, "", IndexFileNames.COMPOUND_FILE_EXTENSION), info, context);
 
         // delete new non cfs files directly: they were never
         // registered with IFD
@@ -2916,7 +2916,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;
@@ -3459,7 +3459,7 @@ public class IndexWriter implements Clos
     // Bind a new segment name here so even with
     // ConcurrentMergePolicy we keep deterministic segment
     // names.
-    merge.info = new SegmentInfo(newSegmentName(), 0, directory, false, null, globalFieldNumberMap.newFieldInfos(SegmentCodecsBuilder.create(codecs)));
+    merge.info = new SegmentInfo(newSegmentName(), 0, directory, false, null, new FieldInfos(globalFieldNumberMap));
 
     // Lock order: IW -> BD
     final BufferedDeletesStream.ApplyDeletesResult result = bufferedDeletesStream.applyDeletes(readerPool, merge.segments);
@@ -3633,7 +3633,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(), context);
+                                             payloadProcessorProvider, merge.info.getFieldInfos(), codec, context);
 
     if (infoStream != null) {
       message("merging " + merge.segString(directory) + " mergeVectors=" + merge.info.getFieldInfos().hasVectors());
@@ -3679,10 +3679,10 @@ public class IndexWriter implements Clos
       mergedDocCount = merge.info.docCount = merger.merge();
 
       // Record which codec was used to write the segment
-      merge.info.setSegmentCodecs(merger.getSegmentCodecs());
+      merge.info.setCodec(merger.getCodec());
 
       if (infoStream != null) {
-        message("merge segmentCodecs=" + merger.getSegmentCodecs());
+        message("merge codecs=" + merger.getCodec());
         message("merge store matchedCount=" + merger.getMatchedSubReaderCount() + " vs " + merge.readers.size());
       }
       anyNonBulkMerges |= merger.getAnyNonBulkMerges();
@@ -3975,7 +3975,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/trunk/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java Fri Nov  4 15:43:35 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/trunk/lucene/src/java/org/apache/lucene/index/PerDocWriteState.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PerDocWriteState.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PerDocWriteState.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PerDocWriteState.java Fri Nov  4 15:43:35 2011
@@ -34,41 +34,37 @@ public class PerDocWriteState {
   public final String segmentName;
   public final FieldInfos fieldInfos;
   public final Counter bytesUsed;
-  public final SegmentCodecs segmentCodecs;
-  public final int codecId;
+  public final String segmentSuffix;
   public final IOContext context;
 
-  PerDocWriteState(PrintStream infoStream, Directory directory,
+  public PerDocWriteState(PrintStream infoStream, Directory directory,
       String segmentName, FieldInfos fieldInfos, Counter bytesUsed,
-      int codecId, IOContext context) {
+      String segmentSuffix, IOContext context) {
     this.infoStream = infoStream;
     this.directory = directory;
     this.segmentName = segmentName;
     this.fieldInfos = fieldInfos;
-    this.segmentCodecs = fieldInfos.buildSegmentCodecs(false);
-    this.codecId = codecId;
+    this.segmentSuffix = segmentSuffix;
     this.bytesUsed = bytesUsed;
     this.context = context;
   }
 
-  PerDocWriteState(SegmentWriteState state) {
+  public PerDocWriteState(SegmentWriteState state) {
     infoStream = state.infoStream;
     directory = state.directory;
-    segmentCodecs = state.segmentCodecs;
     segmentName = state.segmentName;
     fieldInfos = state.fieldInfos;
-    codecId = state.codecId;
+    segmentSuffix = state.segmentSuffix;
     bytesUsed = Counter.newCounter();
     context = state.context;
   }
 
-  PerDocWriteState(PerDocWriteState state, int codecId) {
+  public PerDocWriteState(PerDocWriteState state, String segmentSuffix) {
     this.infoStream = state.infoStream;
     this.directory = state.directory;
     this.segmentName = state.segmentName;
     this.fieldInfos = state.fieldInfos;
-    this.segmentCodecs = state.segmentCodecs;
-    this.codecId = codecId;
+    this.segmentSuffix = segmentSuffix;
     this.bytesUsed = state.bytesUsed;
     this.context = state.context;
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java Fri Nov  4 15:43:35 2011
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.index.codecs.FieldsReader;
 import org.apache.lucene.index.codecs.PerDocValues;
@@ -68,7 +69,7 @@ final class SegmentCoreReaders {
     }
     
     segment = si.name;
-    final SegmentCodecs segmentCodecs = si.getSegmentCodecs();
+    final Codec codec = si.getCodec();
     this.context = context;
     this.dir = dir;
     
@@ -85,12 +86,12 @@ final class SegmentCoreReaders {
       fieldInfos = si.getFieldInfos();
       
       this.termsIndexDivisor = termsIndexDivisor;
-      final Codec codec = segmentCodecs.codec();
+      final PostingsFormat format = codec.postingsFormat();
       final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si, fieldInfos, context, termsIndexDivisor);
       // Ask codec for its Fields
-      fields = codec.fieldsProducer(segmentReadState);
+      fields = format.fieldsProducer(segmentReadState);
       assert fields != null;
-      perDocProducer = codec.docsProducer(segmentReadState);
+      perDocProducer = codec.docValuesFormat().docsProducer(segmentReadState);
       success = true;
     } finally {
       if (!success) {
@@ -165,7 +166,7 @@ final class SegmentCoreReaders {
       }
       
       final String storesSegment = si.getDocStoreSegment();
-      fieldsReaderOrig = si.getSegmentCodecs().provider.fieldsReader(storeDir, storesSegment, fieldInfos, context,
+      fieldsReaderOrig = si.getCodec().fieldsFormat().fieldsReader(storeDir, storesSegment, fieldInfos, context,
           si.getDocStoreOffset(), si.docCount);
       
       // Verify two sources of "maxDoc" agree:

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java Fri Nov  4 15:43:35 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;
@@ -97,7 +96,7 @@ public final class SegmentInfo implement
   
   private FieldInfos fieldInfos;
 
-  private SegmentCodecs segmentCodecs;
+  private Codec codec;
 
   private Map<String,String> diagnostics;
 
@@ -116,7 +115,7 @@ public final class SegmentInfo implement
   private long fieldInfosVersion;
   
   public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile,
-                     SegmentCodecs segmentCodecs, FieldInfos fieldInfos) {
+                     Codec codec, FieldInfos fieldInfos) {
     this.name = name;
     this.docCount = docCount;
     this.dir = dir;
@@ -124,7 +123,7 @@ public final class SegmentInfo implement
     this.isCompoundFile = isCompoundFile;
     this.docStoreOffset = -1;
     this.docStoreSegment = name;
-    this.segmentCodecs = segmentCodecs;
+    this.codec = codec;
     delCount = 0;
     version = Constants.LUCENE_MAIN_VERSION;
     this.fieldInfos = fieldInfos;
@@ -156,7 +155,7 @@ public final class SegmentInfo implement
     }
     isCompoundFile = src.isCompoundFile;
     delCount = src.delCount;
-    segmentCodecs = src.segmentCodecs;
+    codec = src.codec;
   }
 
   void setDiagnostics(Map<String, String> diagnostics) {
@@ -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,13 +220,13 @@ public final class SegmentInfo implement
 
     hasProx = input.readByte();
 
+    
     // System.out.println(Thread.currentThread().getName() + ": si.read hasProx=" + hasProx + " seg=" + name);
+    // note: if the codec is not available: Codec.forName will throw an exception.
     if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
-      segmentCodecs = new SegmentCodecs(codecs, input);
+      codec = Codec.forName(input.readString());
     } else {
-      // codec ID on FieldInfo is 0 so it will simply use the first codec available
-      // TODO what todo if preflex is not available in the provider? register it or fail?
-      segmentCodecs = new SegmentCodecs(codecs, new Codec[] { codecs.lookup("PreFlex")});
+      codec = Codec.forName("Lucene3x");
     }
     diagnostics = input.readStringStringMap();
 
@@ -350,7 +349,7 @@ public final class SegmentInfo implement
 
   @Override
   public Object clone() {
-    final SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, segmentCodecs,
+    final SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, codec,
         fieldInfos == null ? null : (FieldInfos) fieldInfos.clone());
     si.docStoreOffset = docStoreOffset;
     si.docStoreSegment = docStoreSegment;
@@ -573,7 +572,7 @@ public final class SegmentInfo implement
     output.writeByte((byte) (isCompoundFile ? YES : NO));
     output.writeInt(delCount);
     output.writeByte((byte) (hasProx));
-    segmentCodecs.write(output);
+    output.writeString(codec.getName());
     output.writeStringStringMap(diagnostics);
     output.writeByte((byte) (hasVectors));
   }
@@ -583,16 +582,16 @@ public final class SegmentInfo implement
   }
 
   /** Can only be called once. */
-  public void setSegmentCodecs(SegmentCodecs segmentCodecs) {
-    assert this.segmentCodecs == null;
-    if (segmentCodecs == null) {
+  public void setCodec(Codec codec) {
+    assert this.codec == null;
+    if (codec == null) {
       throw new IllegalArgumentException("segmentCodecs must be non-null");
     }
-    this.segmentCodecs = segmentCodecs;
+    this.codec = codec;
   }
 
-  SegmentCodecs getSegmentCodecs() {
-    return segmentCodecs;
+  Codec getCodec() {
+    return codec;
   }
 
   private void addIfExists(Set<String> files, String fileName) throws IOException {
@@ -628,7 +627,7 @@ public final class SegmentInfo implement
       for(String ext : IndexFileNames.NON_STORE_INDEX_EXTENSIONS) {
         addIfExists(fileSet, IndexFileNames.segmentFileName(name, "", ext));
       }
-      segmentCodecs.files(dir, this, fileSet);
+      codec.files(dir, this, fileSet);
     }
 
     if (docStoreOffset != -1) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java Fri Nov  4 15:43:35 2011
@@ -32,10 +32,11 @@ 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;
+import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -83,8 +84,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 CodecProvider codecs;
 
   private int format;
   
@@ -95,20 +94,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() {
-    this(CodecProvider.getDefault());
-  }
-  
-  public SegmentInfos(CodecProvider codecs) {
-    this.codecs = codecs;
-  }
-
   public void setFormat(int format) {
     this.format = format;
   }
@@ -197,7 +190,7 @@ public final class SegmentInfos implemen
      * since this file might belong to more than one segment (global map) and
      * could otherwise easily be confused with a per-segment file.
      */
-    return IndexFileNames.segmentFileName(""+ version, "", IndexFileNames.GLOBAL_FIELD_NUM_MAP_EXTENSION);
+    return IndexFileNames.segmentFileName("_"+ version, "", IndexFileNames.GLOBAL_FIELD_NUM_MAP_EXTENSION);
   }
 
   /**
@@ -241,9 +234,7 @@ public final class SegmentInfos implemen
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public final void read(Directory directory, String segmentFileName, 
-                         CodecProvider codecs) throws CorruptIndexException, IOException {
-    this.codecs = codecs;
+  public final void read(Directory directory, String segmentFileName) throws CorruptIndexException, IOException {
     boolean success = false;
 
     // Clear any previous segments:
@@ -253,12 +244,40 @@ public final class SegmentInfos implemen
 
     lastGeneration = generation;
 
+    // TODO: scary to have default impl reopen the file... but to make it a bit more flexible,
+    // maybe we could use a plain indexinput here... could default impl rewind/wrap with checksumII,
+    // and any checksumming is then up to implementation?
+    ChecksumIndexInput input = null;
     try {
-      SegmentInfosReader infosReader = codecs.getSegmentInfosReader();
-      infosReader.read(directory, segmentFileName, codecs, this, IOContext.READ);
+      input = new ChecksumIndexInput(directory.openInput(segmentFileName, IOContext.READ));
+      final int format = input.readInt();
+      setFormat(format);
+    
+      // check that it is a format we can understand
+      if (format > DefaultSegmentInfosWriter.FORMAT_MINIMUM)
+        throw new IndexFormatTooOldException(segmentFileName, format,
+          DefaultSegmentInfosWriter.FORMAT_MINIMUM, DefaultSegmentInfosWriter.FORMAT_CURRENT);
+      if (format < DefaultSegmentInfosWriter.FORMAT_CURRENT)
+        throw new IndexFormatTooNewException(segmentFileName, format,
+          DefaultSegmentInfosWriter.FORMAT_MINIMUM, DefaultSegmentInfosWriter.FORMAT_CURRENT);
+
+      if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
+        codecFormat = Codec.forName(input.readString());
+      } else {
+        codecFormat = Codec.forName("Lucene3x");
+      }
+      SegmentInfosReader infosReader = codecFormat.segmentInfosFormat().getSegmentInfosReader();
+      infosReader.read(directory, segmentFileName, input, this, IOContext.READ);
+      final long checksumNow = input.getChecksum();
+      final long checksumThen = input.readLong();
+      if (checksumNow != checksumThen)
+        throw new CorruptIndexException("checksum mismatch in segments file");
       success = true;
     }
     finally {
+      if (input != null) {
+        input.close();
+      }
       if (!success) {
         // Clear any segment infos we had loaded so we
         // have a clean slate on retry:
@@ -267,25 +286,14 @@ public final class SegmentInfos implemen
     }
   }
 
-  /**
-   * This version of read uses the retry logic (for lock-less
-   * commits) to find the right segments file to load.
-   * @throws CorruptIndexException if the index is corrupt
-   * @throws IOException if there is a low-level IO error
-   */
   public final void read(Directory directory) throws CorruptIndexException, IOException {
-    read(directory, CodecProvider.getDefault());
-  }
-  
-  public final void read(Directory directory, final CodecProvider codecs) throws CorruptIndexException, IOException {
     generation = lastGeneration = -1;
-    this.codecs = codecs;
 
     new FindSegmentsFile(directory) {
 
       @Override
       protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
-        read(directory, segmentFileName, codecs);
+        read(directory, segmentFileName);
         return null;
       }
     }.run();
@@ -297,7 +305,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;
@@ -322,8 +330,8 @@ public final class SegmentInfos implemen
     boolean success = false;
 
     try {
-      SegmentInfosWriter infosWriter = codecs.getSegmentInfosWriter();
-      segnOutput = infosWriter.writeInfos(directory, segmentFileName, this, IOContext.DEFAULT);
+      SegmentInfosWriter infosWriter = codec.segmentInfosFormat().getSegmentInfosWriter();
+      segnOutput = infosWriter.writeInfos(directory, segmentFileName, codec.getName(), this, IOContext.DEFAULT);
       infosWriter.prepareCommit(segnOutput);
       pendingSegnOutput = segnOutput;
       success = true;
@@ -380,7 +388,7 @@ public final class SegmentInfos implemen
       sis.cachedUnmodifiableList = null;
       sis.cachedUnmodifiableSet = null;
       for(final SegmentInfo info : this) {
-        assert info.getSegmentCodecs() != null;
+        assert info.getCodec() != null;
         // dont directly access segments, use add method!!!
         sis.add((SegmentInfo) info.clone());
       }
@@ -409,7 +417,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
@@ -417,8 +425,8 @@ 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);
-    sis.read(directory, codecs);
+    SegmentInfos sis = new SegmentInfos();
+    sis.read(directory);
     return sis.version;
   }
 
@@ -427,10 +435,10 @@ 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);
-    sis.read(directory, codecs);
+    SegmentInfos sis = new SegmentInfos();
+    sis.read(directory);
     return sis.getUserData();
   }
 
@@ -808,10 +816,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 {
@@ -882,12 +890,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;
@@ -958,9 +966,9 @@ 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);
+  final void commit(Directory dir, Codec codec) throws IOException {
+    prepareCommit(dir, codec);
+    finishCommit(dir, codec);
   }
 
   public String toString(Directory directory) {
@@ -1106,7 +1114,7 @@ public final class SegmentInfos implemen
     if (cloneChildren) {
       final List<SegmentInfo> list = new ArrayList<SegmentInfo>(size());
       for(final SegmentInfo info : this) {
-        assert info.getSegmentCodecs() != null;
+        assert info.getCodec() != null;
         list.add((SegmentInfo) info.clone());
       }
       return list;
@@ -1120,6 +1128,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/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Fri Nov  4 15:43:35 2011
@@ -73,7 +73,7 @@ final class SegmentMerger {
   
   private IOContext context;
 
-  SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos, IOContext context) {
+  SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos, Codec codec, IOContext context) {
     this.payloadProcessorProvider = payloadProcessorProvider;
     directory = dir;
     segment = name;
@@ -89,6 +89,7 @@ final class SegmentMerger {
       };
     }
     this.termIndexInterval = termIndexInterval;
+    this.codec = codec;
     this.context = context;
   }
 
@@ -254,12 +255,11 @@ final class SegmentMerger {
         fieldInfos.addOrUpdate(reader.getFieldNames(FieldOption.DOC_VALUES), false);
       }
     }
-    final SegmentCodecs codecInfo = fieldInfos.buildSegmentCodecs(false);
 
     int docCount = 0;
 
     setMatchingSegmentReaders();
-    final FieldsWriter fieldsWriter = codecInfo.provider.fieldsWriter(directory, segment, context);
+    final FieldsWriter fieldsWriter = codec.fieldsFormat().fieldsWriter(directory, segment, context);
     try {
       int idx = 0;
       for (MergeState.IndexReaderAndLiveDocs reader : readers) {
@@ -293,7 +293,7 @@ final class SegmentMerger {
       // entering the index.  See LUCENE-1282 for
       // details.
       throw new RuntimeException("mergeFields produced an invalid result: docCount is " + docCount + " but fdx file size is " + fdxFileLength + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
-    segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codecInfo, null, context);
+    segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codec, null, context);
 
     return docCount;
   }
@@ -493,9 +493,9 @@ final class SegmentMerger {
     }
   }
 
-  SegmentCodecs getSegmentCodecs() {
+  Codec getCodec() {
     assert segmentWriteState != null;
-    return segmentWriteState.segmentCodecs;
+    return segmentWriteState.codec;
   }
 
   private final void mergeTerms() throws CorruptIndexException, IOException {
@@ -566,8 +566,8 @@ final class SegmentMerger {
         mergeState.dirPayloadProcessor[i] = payloadProcessorProvider.getDirProcessor(reader.reader.directory());
       }
     }
-    codec = segmentWriteState.segmentCodecs.codec();
-    final FieldsConsumer consumer = codec.fieldsConsumer(segmentWriteState);
+
+    final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState);
     boolean success = false;
     try {
       consumer.merge(mergeState,
@@ -584,7 +584,7 @@ final class SegmentMerger {
   }
 
   private void mergePerDoc() throws IOException {
-      final PerDocConsumer docsConsumer = codec
+      final PerDocConsumer docsConsumer = codec.docValuesFormat()
           .docsConsumer(new PerDocWriteState(segmentWriteState));
       // TODO: remove this check when 3.x indexes are no longer supported
       // (3.x indexes don't have docvalues)

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReadState.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReadState.java?rev=1197603&r1=1197602&r2=1197603&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReadState.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReadState.java Fri Nov  4 15:43:35 2011
@@ -35,11 +35,11 @@ public class SegmentReadState {
   // that must do so), then it should negate this value to
   // get the app's terms divisor:
   public int termsIndexDivisor;
-  public final int codecId;
+  public final String segmentSuffix;
 
   public SegmentReadState(Directory dir, SegmentInfo info,
       FieldInfos fieldInfos, IOContext context, int termsIndexDivisor) {
-    this(dir, info, fieldInfos,  context, termsIndexDivisor, -1);
+    this(dir, info, fieldInfos,  context, termsIndexDivisor, "");
   }
   
   public SegmentReadState(Directory dir,
@@ -47,12 +47,22 @@ public class SegmentReadState {
                           FieldInfos fieldInfos,
                           IOContext context,
                           int termsIndexDivisor,
-                          int codecId) {
+                          String segmentSuffix) {
     this.dir = dir;
     this.segmentInfo = info;
     this.fieldInfos = fieldInfos;
     this.context = context;
     this.termsIndexDivisor = termsIndexDivisor;
-    this.codecId = codecId;
+    this.segmentSuffix = segmentSuffix;
   }
-}
\ No newline at end of file
+
+  public SegmentReadState(SegmentReadState other,
+                          String newSegmentSuffix) {
+    this.dir = other.dir;
+    this.segmentInfo = other.segmentInfo;
+    this.fieldInfos = other.fieldInfos;
+    this.context = other.context;
+    this.termsIndexDivisor = other.termsIndexDivisor;
+    this.segmentSuffix = newSegmentSuffix;
+  }
+}