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

svn commit: r1189367 [2/3] - in /lucene/dev/branches/lucene2621: lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/ lucene/src/java/org/apache/lucene/index/ lucene/src/java/org/apache/lucene/index/codecs/ lucene/src/java/org/apache/...

Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xFields.java (from r1189253, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xFields.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xFields.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java&r1=1189253&r2=1189367&rev=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xFields.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.preflex;
+package org.apache.lucene.index.codecs.lucene3x;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -51,7 +51,7 @@ import org.apache.lucene.util.UnicodeUti
  * @deprecated (4.0)
  */
 @Deprecated
-public class PreFlexFields extends FieldsProducer {
+public class Lucene3xFields extends FieldsProducer {
   
   private static final boolean DEBUG_SURROGATES = false;
 
@@ -68,7 +68,7 @@ public class PreFlexFields extends Field
   private final IOContext context;
   private Directory cfsReader;
 
-  public PreFlexFields(Directory dir, FieldInfos fieldInfos, SegmentInfo info, IOContext context, int indexDivisor)
+  public Lucene3xFields(Directory dir, FieldInfos fieldInfos, SegmentInfo info, IOContext context, int indexDivisor)
     throws IOException {
 
     si = info;
@@ -95,7 +95,7 @@ public class PreFlexFields extends Field
 
       // make sure that all index files have been read or are kept open
       // so that if an index update removes them we'll still have them
-      freqStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", PreFlexPostingsFormat.FREQ_EXTENSION), context);
+      freqStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.FREQ_EXTENSION), context);
       boolean anyProx = false;
       for (FieldInfo fi : fieldInfos) {
         if (fi.isIndexed) {
@@ -108,7 +108,7 @@ public class PreFlexFields extends Field
       }
 
       if (anyProx) {
-        proxStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", PreFlexPostingsFormat.PROX_EXTENSION), context);
+        proxStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.PROX_EXTENSION), context);
       } else {
         proxStream = null;
       }
@@ -136,16 +136,16 @@ public class PreFlexFields extends Field
   }
 
   static void files(Directory dir, SegmentInfo info, Collection<String> files) throws IOException {
-    files.add(IndexFileNames.segmentFileName(info.name, "", PreFlexPostingsFormat.TERMS_EXTENSION));
-    files.add(IndexFileNames.segmentFileName(info.name, "", PreFlexPostingsFormat.TERMS_INDEX_EXTENSION));
-    files.add(IndexFileNames.segmentFileName(info.name, "", PreFlexPostingsFormat.FREQ_EXTENSION));
+    files.add(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.TERMS_EXTENSION));
+    files.add(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION));
+    files.add(IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.FREQ_EXTENSION));
     if (info.getHasProx()) {
       // LUCENE-1739: for certain versions of 2.9-dev,
       // hasProx would be incorrectly computed during
       // indexing as true, and then stored into the segments
       // file, when it should have been false.  So we do the
       // extra check, here:
-      final String prx = IndexFileNames.segmentFileName(info.name, "", PreFlexPostingsFormat.PROX_EXTENSION);
+      final String prx = IndexFileNames.segmentFileName(info.name, "", Lucene3xPostingsFormat.PROX_EXTENSION);
       if (dir.fileExists(prx)) {
         files.add(prx);
       }

Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xPostingsFormat.java (from r1189253, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexPostingsFormat.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xPostingsFormat.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xPostingsFormat.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexPostingsFormat.java&r1=1189253&r2=1189367&rev=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xPostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.preflex;
+package org.apache.lucene.index.codecs.lucene3x;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -40,7 +40,7 @@ import org.apache.lucene.index.codecs.Pe
  * @lucene.experimental
  */
 @Deprecated
-public class PreFlexPostingsFormat extends PostingsFormat {
+public class Lucene3xPostingsFormat extends PostingsFormat {
 
   /** Extension of terms file */
   public static final String TERMS_EXTENSION = "tis";
@@ -54,7 +54,7 @@ public class PreFlexPostingsFormat exten
   /** Extension of prox postings file */
   public static final String PROX_EXTENSION = "prx";
 
-  public PreFlexPostingsFormat() {
+  public Lucene3xPostingsFormat() {
     super("PreFlex");
   }
   
@@ -65,13 +65,13 @@ public class PreFlexPostingsFormat exten
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    return new PreFlexFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor);
+    return new Lucene3xFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor);
   }
 
   @Override
   public void files(Directory dir, SegmentInfo info, int id, Set<String> files) throws IOException {
     // preflex fields have no codec ID - we ignore it here
-    PreFlexFields.files(dir, info, files);
+    Lucene3xFields.files(dir, info, files);
   }
 
   @Override

Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermDocs.java (from r1189253, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermDocs.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermDocs.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java&r1=1189253&r2=1189367&rev=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermDocs.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.preflex;
+package org.apache.lucene.index.codecs.lucene3x;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -23,7 +23,7 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.codecs.standard.DefaultSkipListReader;
+import org.apache.lucene.index.codecs.lucene40.DefaultSkipListReader;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Bits;
 

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermEnum.java?rev=1189367&r1=1189253&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermEnum.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermEnum.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.preflex;
+package org.apache.lucene.index.codecs.lucene3x;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermPositions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermPositions.java?rev=1189367&r1=1189253&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermPositions.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/SegmentTermPositions.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.preflex;
+package org.apache.lucene.index.codecs.lucene3x;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermBuffer.java?rev=1189367&r1=1189253&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermBuffer.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermBuffer.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.preflex;
+package org.apache.lucene.index.codecs.lucene3x;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermInfo.java?rev=1189367&r1=1189253&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermInfo.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermInfo.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.preflex;
+package org.apache.lucene.index.codecs.lucene3x;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermInfosReader.java?rev=1189367&r1=1189253&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermInfosReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/TermInfosReader.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.preflex;
+package org.apache.lucene.index.codecs.lucene3x;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -110,7 +110,7 @@ public final class TermInfosReader {
       segment = seg;
       fieldInfos = fis;
 
-      origEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", PreFlexPostingsFormat.TERMS_EXTENSION),
+      origEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", Lucene3xPostingsFormat.TERMS_EXTENSION),
                                                          context), fieldInfos, false);
       size = origEnum.size;
 
@@ -118,7 +118,7 @@ public final class TermInfosReader {
       if (indexDivisor != -1) {
         // Load terms index
         totalIndexInterval = origEnum.indexInterval * indexDivisor;
-        final SegmentTermEnum indexEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", PreFlexPostingsFormat.TERMS_INDEX_EXTENSION),
+        final SegmentTermEnum indexEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION),
                                                                                   context), fieldInfos, true);
 
         try {

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/DefaultSkipListReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/DefaultSkipListReader.java?rev=1189367&r1=1189253&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/DefaultSkipListReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/DefaultSkipListReader.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.standard;
+package org.apache.lucene.index.codecs.lucene40;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/DefaultSkipListWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/DefaultSkipListWriter.java?rev=1189367&r1=1189253&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/DefaultSkipListWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/DefaultSkipListWriter.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.standard;
+package org.apache.lucene.index.codecs.lucene40;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more

Added: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java?rev=1189367&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java Wed Oct 26 18:33:48 2011
@@ -0,0 +1,76 @@
+package org.apache.lucene.index.codecs.lucene40;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.index.codecs.DefaultFieldsFormat;
+import org.apache.lucene.index.codecs.FieldsFormat;
+import org.apache.lucene.index.codecs.PostingsFormat;
+import org.apache.lucene.index.codecs.memory.MemoryPostingsFormat;
+import org.apache.lucene.index.codecs.perfield.PerFieldCodec;
+import org.apache.lucene.index.codecs.pulsing.PulsingPostingsFormat;
+import org.apache.lucene.index.codecs.simpletext.SimpleTextPostingsFormat;
+
+/**
+ * Implements the Lucene 4.0 index format, with configurable per-field postings formats
+ * and using {@link DefaultFieldsFormat}
+ * @lucene.experimental
+ */
+// TODO: which postings formats will we actually support for backwards compatibility?
+public class Lucene40Codec extends PerFieldCodec {
+  private final FieldsFormat fieldsFormat = new DefaultFieldsFormat();
+
+  public Lucene40Codec() {
+    this(Collections.<String,String>emptyMap());
+  }
+  
+  public Lucene40Codec(Map<String,String> perFieldMap) {
+    this("Lucene40", perFieldMap);
+  }
+  
+  public Lucene40Codec(String defaultFormat, Map<String,String> perFieldMap) {
+    super("Lucene40", defaultFormat, perFieldMap);
+  }
+  
+  @Override
+  public FieldsFormat fieldsFormat() {
+    return fieldsFormat;
+  }
+
+  @Override
+  public PostingsFormat lookup(String name) {
+    final PostingsFormat codec = CORE_FORMATS.get(name);
+    if (codec == null) {
+      throw new IllegalArgumentException("required format '" + name + "' not found; known formats: " + CORE_FORMATS.keySet());
+    }
+    return codec;
+  }
+  
+  // postings formats
+  private static final Map<String,PostingsFormat> CORE_FORMATS = new HashMap<String,PostingsFormat>();
+  static {
+    CORE_FORMATS.put("Lucene40", new Lucene40PostingsFormat());
+    CORE_FORMATS.put("Pulsing", new PulsingPostingsFormat());
+    CORE_FORMATS.put("SimpleText", new SimpleTextPostingsFormat());
+    CORE_FORMATS.put("Memory", new MemoryPostingsFormat());
+  }
+}

Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsFormat.java (from r1189253, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsFormat.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsFormat.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsFormat.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsFormat.java&r1=1189253&r2=1189367&rev=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.standard;
+package org.apache.lucene.index.codecs.lucene40;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -39,17 +39,17 @@ import org.apache.lucene.store.Directory
 
 /** Default codec. 
  *  @lucene.experimental */
-public class StandardPostingsFormat extends PostingsFormat {
+public class Lucene40PostingsFormat extends PostingsFormat {
 
   private final int minBlockSize;
   private final int maxBlockSize;
 
-  public StandardPostingsFormat() {
+  public Lucene40PostingsFormat() {
     this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
   }
 
-  public StandardPostingsFormat(int minBlockSize, int maxBlockSize) {
-    super("Standard");
+  public Lucene40PostingsFormat(int minBlockSize, int maxBlockSize) {
+    super("Lucene40");
     this.minBlockSize = minBlockSize;
     assert minBlockSize > 1;
     this.maxBlockSize = maxBlockSize;
@@ -57,7 +57,7 @@ public class StandardPostingsFormat exte
 
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    PostingsWriterBase docs = new StandardPostingsWriter(state);
+    PostingsWriterBase docs = new Lucene40PostingsWriter(state);
 
     // TODO: should we make the terms index more easily
     // pluggable?  Ie so that this codec would record which
@@ -79,7 +79,7 @@ public class StandardPostingsFormat exte
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    PostingsReaderBase postings = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
+    PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.formatId);
 
     boolean success = false;
     try {
@@ -89,7 +89,7 @@ public class StandardPostingsFormat exte
                                                     state.segmentInfo.name,
                                                     postings,
                                                     state.context,
-                                                    state.codecId,
+                                                    state.formatId,
                                                     state.termsIndexDivisor);
       success = true;
       return ret;
@@ -108,7 +108,7 @@ public class StandardPostingsFormat exte
 
   @Override
   public void files(Directory dir, SegmentInfo segmentInfo, int codecID, Set<String> files) throws IOException {
-    StandardPostingsReader.files(dir, segmentInfo, codecID, files);
+    Lucene40PostingsReader.files(dir, segmentInfo, codecID, files);
     BlockTreeTermsReader.files(dir, segmentInfo, codecID, files);
     DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files);
   }

Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsReader.java (from r1189253, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsReader.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsReader.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java&r1=1189253&r2=1189367&rev=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsReader.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.standard;
+package org.apache.lucene.index.codecs.lucene40;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -42,7 +42,7 @@ import org.apache.lucene.util.CodecUtil;
  *  postings format. 
  *  @lucene.experimental */
 
-public class StandardPostingsReader extends PostingsReaderBase {
+public class Lucene40PostingsReader extends PostingsReaderBase {
 
   private final IndexInput freqIn;
   private final IndexInput proxIn;
@@ -54,14 +54,14 @@ public class StandardPostingsReader exte
 
   // private String segment;
 
-  public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext ioContext, int codecId) throws IOException {
-    freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardPostingsFormat.FREQ_EXTENSION),
+  public Lucene40PostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext ioContext, int codecId) throws IOException {
+    freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, Lucene40PostingsFormat.FREQ_EXTENSION),
                            ioContext);
     // this.segment = segmentInfo.name;
     if (segmentInfo.getHasProx()) {
       boolean success = false;
       try {
-        proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardPostingsFormat.PROX_EXTENSION),
+        proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, Lucene40PostingsFormat.PROX_EXTENSION),
                                ioContext);
         success = true;
       } finally {
@@ -75,9 +75,9 @@ public class StandardPostingsReader exte
   }
 
   public static void files(Directory dir, SegmentInfo segmentInfo, int codecID, Collection<String> files) throws IOException {
-    files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, StandardPostingsFormat.FREQ_EXTENSION));
+    files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, Lucene40PostingsFormat.FREQ_EXTENSION));
     if (segmentInfo.getHasProx()) {
-      files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, StandardPostingsFormat.PROX_EXTENSION));
+      files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, Lucene40PostingsFormat.PROX_EXTENSION));
     }
   }
 
@@ -85,8 +85,8 @@ public class StandardPostingsReader exte
   public void init(IndexInput termsIn) throws IOException {
 
     // Make sure we are talking to the matching past writer
-    CodecUtil.checkHeader(termsIn, StandardPostingsWriter.CODEC,
-      StandardPostingsWriter.VERSION_START, StandardPostingsWriter.VERSION_START);
+    CodecUtil.checkHeader(termsIn, Lucene40PostingsWriter.CODEC,
+      Lucene40PostingsWriter.VERSION_START, Lucene40PostingsWriter.VERSION_START);
 
     skipInterval = termsIn.readInt();
     maxSkipLevels = termsIn.readInt();

Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsWriter.java (from r1189253, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsWriter.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsWriter.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java&r1=1189253&r2=1189367&rev=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsWriter.java Wed Oct 26 18:33:48 2011
@@ -1,4 +1,4 @@
-package org.apache.lucene.index.codecs.standard;
+package org.apache.lucene.index.codecs.lucene40;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -38,8 +38,8 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CodecUtil;
 
 /** @lucene.experimental */
-public final class StandardPostingsWriter extends PostingsWriterBase {
-  final static String CODEC = "StandardPostingsWriter";
+public final class Lucene40PostingsWriter extends PostingsWriterBase {
+  final static String CODEC = "Lucene40PostingsWriter";
 
   //private static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
   
@@ -81,21 +81,21 @@ public final class StandardPostingsWrite
 
   // private String segment;
 
-  public StandardPostingsWriter(SegmentWriteState state) throws IOException {
+  public Lucene40PostingsWriter(SegmentWriteState state) throws IOException {
     this(state, DEFAULT_SKIP_INTERVAL);
   }
   
-  public StandardPostingsWriter(SegmentWriteState state, int skipInterval) throws IOException {
+  public Lucene40PostingsWriter(SegmentWriteState state, int skipInterval) throws IOException {
     super();
     this.skipInterval = skipInterval;
     this.skipMinimum = skipInterval; /* set to the same for now */
     // this.segment = state.segmentName;
-    String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardPostingsFormat.FREQ_EXTENSION);
+    String fileName = IndexFileNames.segmentFileName(state.segmentName, state.formatId, Lucene40PostingsFormat.FREQ_EXTENSION);
     freqOut = state.directory.createOutput(fileName, state.context);
     if (state.fieldInfos.hasProx()) {
       // At least one field does not omit TF, so create the
       // prox file
-      fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardPostingsFormat.PROX_EXTENSION);
+      fileName = IndexFileNames.segmentFileName(state.segmentName, state.formatId, Lucene40PostingsFormat.PROX_EXTENSION);
       proxOut = state.directory.createOutput(fileName, state.context);
     } else {
       // Every field omits TF so we will write no prox file

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryPostingsFormat.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryPostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -248,7 +248,7 @@ public class MemoryPostingsFormat extend
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
 
-    final String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, EXTENSION);
+    final String fileName = IndexFileNames.segmentFileName(state.segmentName, state.formatId, EXTENSION);
     final IndexOutput out = state.directory.createOutput(fileName, state.context);
     
     return new FieldsConsumer() {
@@ -740,7 +740,7 @@ public class MemoryPostingsFormat extend
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.codecId, EXTENSION);
+    final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.formatId, EXTENSION);
     final IndexInput in = state.dir.openInput(fileName, IOContext.READONCE);
 
     final SortedMap<String,TermsReader> fields = new TreeMap<String,TermsReader>();

Added: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldCodec.java?rev=1189367&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldCodec.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldCodec.java Wed Oct 26 18:33:48 2011
@@ -0,0 +1,87 @@
+package org.apache.lucene.index.codecs.perfield;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.DefaultFieldsFormat;
+import org.apache.lucene.index.codecs.FieldsFormat;
+import org.apache.lucene.index.codecs.PostingsFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
+import org.apache.lucene.index.codecs.memory.MemoryPostingsFormat;
+import org.apache.lucene.index.codecs.pulsing.PulsingPostingsFormat;
+import org.apache.lucene.index.codecs.simpletext.SimpleTextPostingsFormat;
+
+public abstract class PerFieldCodec extends Codec {
+  private final PostingsFormat postingsFormat;
+
+  private final String defaultPostingsFormat;
+  private final Map<String,String> perFieldMap;
+  
+  public PerFieldCodec(String name, String defaultPostingsFormat, Map<String,String> perFieldMap) {
+    super(name);
+    this.defaultPostingsFormat = defaultPostingsFormat;
+    this.perFieldMap = perFieldMap;
+    this.postingsFormat = new PerFieldPostingsFormat(this);
+  }
+
+  @Override
+  public PostingsFormat postingsFormat() {
+    return postingsFormat;
+  }
+  
+  /**
+   * Returns the {@link PostingsFormat} name for the given field or the default format if
+   * not set.
+   * 
+   * @param name
+   *          the fields name
+   * @return the {@link PostingsFormat} name for the given field or the default format if
+   *         not set.
+   */
+  public String getPostingsFormat(String name) {
+    final String format;
+    if ((format = perFieldMap.get(name)) == null) {
+      return defaultPostingsFormat;
+    }
+    return format;
+  }
+
+  /**
+   * Returns <code>true</code> if this Codec has a Format registered for this
+   * field.
+   */
+  public boolean hasPostingsFormatFormat(String name) {
+    return perFieldMap.containsKey(name);
+  }
+
+  /**
+   * Returns the default {@link PostingsFormat} for this {@link Codec}
+   * 
+   * @return the default {@link PostingsFormat} for this {@link Codec}
+   */
+  public String getDefaultPostingsFormat() {
+    return defaultPostingsFormat;
+  }
+  
+  public abstract PostingsFormat lookup(String name);
+}

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldPostingsFormat.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldPostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -49,20 +49,20 @@ import org.apache.lucene.store.IOContext
 import org.apache.lucene.util.IOUtils;
 
 /**
- * Enables native per field codec support. This class selects the codec used to
- * write a field depending on the provided {@link SegmentCodecs}. For each field
- * seen it resolves the codec based on the {@link FieldInfo#codecId} which is
- * only valid during a segment merge. See {@link SegmentCodecs} javadoc for
+ * Enables per field format support. This class selects the format used to
+ * write a field depending on the provided {@link SegmentFormats}. For each field
+ * seen it resolves the format based on the {@link FieldInfo#formatId} which is
+ * only valid during a segment merge. See {@link SegmentFormats} javadoc for
  * details.
  * 
  * @lucene.internal
  */
 final class PerFieldPostingsFormat extends PostingsFormat {
-  private final SegmentCodecs segmentCodecs;
+  private final SegmentFormats segmentFormats;
 
-  PerFieldPostingsFormat(SegmentCodecs segmentCodecs) {
+  PerFieldPostingsFormat(SegmentFormats segmentFormats) {
     super("PerField");
-    this.segmentCodecs = segmentCodecs;
+    this.segmentFormats = segmentFormats;
   }
 
   @Override
@@ -75,12 +75,12 @@ final class PerFieldPostingsFormat exten
     private final ArrayList<FieldsConsumer> consumers = new ArrayList<FieldsConsumer>();
 
     public FieldsWriter(SegmentWriteState state) throws IOException {
-      assert segmentCodecs == state.segmentCodecs;
-      final PostingsFormat[] codecs = segmentCodecs.codecs;
-      for (int i = 0; i < codecs.length; i++) {
+      assert segmentFormats == state.segmentFormats;
+      final PostingsFormat[] formats = segmentFormats.formats;
+      for (int i = 0; i < formats.length; i++) {
         boolean success = false;
         try {
-          consumers.add(codecs[i].fieldsConsumer(new SegmentWriteState(state, i)));
+          consumers.add(formats[i].fieldsConsumer(new SegmentWriteState(state, i)));
           success = true;
         } finally {
           if (!success) {
@@ -92,8 +92,8 @@ final class PerFieldPostingsFormat exten
 
     @Override
     public TermsConsumer addField(FieldInfo field) throws IOException {
-      assert field.getCodecId() != FieldInfo.UNASSIGNED_CODEC_ID;
-      final FieldsConsumer fields = consumers.get(field.getCodecId());
+      assert field.getFormatId() != FieldInfo.UNASSIGNED_FORMAT_ID;
+      final FieldsConsumer fields = consumers.get(field.getFormatId());
       return fields.addField(field);
     }
 
@@ -117,13 +117,13 @@ final class PerFieldPostingsFormat exten
         for (FieldInfo fi : fieldInfos) {
           if (fi.isIndexed) { 
             fields.add(fi.name);
-            assert fi.getCodecId() != FieldInfo.UNASSIGNED_CODEC_ID;
-            PostingsFormat codec = segmentCodecs.codecs[fi.getCodecId()];
-            if (!producers.containsKey(codec)) {
-              producers.put(codec, codec.fieldsProducer(new SegmentReadState(dir,
-                                                                             si, fieldInfos, context, indexDivisor, fi.getCodecId())));
+            assert fi.getFormatId() != FieldInfo.UNASSIGNED_FORMAT_ID;
+            PostingsFormat format = segmentFormats.formats[fi.getFormatId()];
+            if (!producers.containsKey(format)) {
+              producers.put(format, format.fieldsProducer(new SegmentReadState(dir,
+                                                                             si, fieldInfos, context, indexDivisor, fi.getFormatId())));
             }
-            codecs.put(fi.name, producers.get(codec));
+            codecs.put(fi.name, producers.get(format));
           }
         }
         success = true;
@@ -193,16 +193,16 @@ final class PerFieldPostingsFormat exten
   }
 
   @Override
-  public void files(Directory dir, SegmentInfo info, int codecId, Set<String> files)
+  public void files(Directory dir, SegmentInfo info, int formatId, Set<String> files)
       throws IOException {
-    // ignore codecid since segmentCodec will assign it per codec
-    segmentCodecs.files(dir, info, files);
+    // ignore formatid since segmentFormat will assign it per codec
+    segmentFormats.files(dir, info, files);
   }
 
   @Override
   public void getExtensions(Set<String> extensions) {
-    for (PostingsFormat codec : segmentCodecs.codecs) {
-      codec.getExtensions(extensions);
+    for (PostingsFormat format : segmentFormats.formats) {
+      format.getExtensions(extensions);
     }
   }
 
@@ -227,11 +227,11 @@ final class PerFieldPostingsFormat exten
       try {
         for (FieldInfo fi : fieldInfos) {
           if (fi.hasDocValues()) { 
-            assert fi.getCodecId() != FieldInfo.UNASSIGNED_CODEC_ID;
-            PostingsFormat codec = segmentCodecs.codecs[fi.getCodecId()];
+            assert fi.getFormatId() != FieldInfo.UNASSIGNED_FORMAT_ID;
+            PostingsFormat codec = segmentFormats.formats[fi.getFormatId()];
             if (!producers.containsKey(codec)) {
               producers.put(codec, codec.docsProducer(new SegmentReadState(dir,
-                si, fieldInfos, context, indexDivisor, fi.getCodecId())));
+                si, fieldInfos, context, indexDivisor, fi.getFormatId())));
             }
             codecs.put(fi.name, producers.get(codec));
           }
@@ -264,14 +264,14 @@ final class PerFieldPostingsFormat exten
   
   private final class PerDocConsumers extends PerDocConsumer {
     private final PerDocConsumer[] consumers;
-    private final PostingsFormat[] codecs;
+    private final PostingsFormat[] formats;
     private final PerDocWriteState state;
 
     public PerDocConsumers(PerDocWriteState state) throws IOException {
-      assert segmentCodecs == state.segmentCodecs;
+      assert segmentFormats == state.segmentFormats;
       this.state = state;
-      codecs = segmentCodecs.codecs;
-      consumers = new PerDocConsumer[codecs.length];
+      formats = segmentFormats.formats;
+      consumers = new PerDocConsumer[formats.length];
     }
 
     public void close() throws IOException {
@@ -280,13 +280,13 @@ final class PerFieldPostingsFormat exten
 
     @Override
     public DocValuesConsumer addValuesField(FieldInfo field) throws IOException {
-      final int codecId = field.getCodecId();
-      assert codecId != FieldInfo.UNASSIGNED_CODEC_ID;
-      PerDocConsumer perDoc = consumers[codecId];
+      final int formatId = field.getFormatId();
+      assert formatId != FieldInfo.UNASSIGNED_FORMAT_ID;
+      PerDocConsumer perDoc = consumers[formatId];
       if (perDoc == null) {
-        perDoc = codecs[codecId].docsConsumer(new PerDocWriteState(state, codecId));
+        perDoc = formats[formatId].docsConsumer(new PerDocWriteState(state, formatId));
         assert perDoc != null;
-        consumers[codecId] = perDoc;
+        consumers[formatId] = perDoc;
       }
       return perDoc.addValuesField(field);
     }

Copied: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentFormats.java (from r1189253, lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentCodecs.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentFormats.java?p2=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentFormats.java&p1=lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentCodecs.java&r1=1189253&r2=1189367&rev=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentCodecs.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/perfield/SegmentFormats.java Wed Oct 26 18:33:48 2011
@@ -28,76 +28,75 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.codecs.PostingsFormat;
-import org.apache.lucene.index.codecs.CodecProvider;
-import org.apache.lucene.index.codecs.preflex.PreFlexPostingsFormat;
+import org.apache.lucene.index.codecs.lucene3x.Lucene3xPostingsFormat;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 
 /**
- * SegmentCodecs maintains an ordered list of distinct codecs used within a
- * segment. Within a segment on codec is used to write multiple fields while
- * each field could be written by a different codec. To enable codecs per field
- * within a single segment we need to record the distinct codecs and map them to
+ * SegmentFormats maintains an ordered list of distinct formats used within a
+ * segment. Within a segment a format is used to write multiple fields while
+ * each field could be written by a different format. To enable formats per field
+ * within a single segment we need to record the distinct formats and map them to
  * each field present in the segment. SegmentCodecs is created together with
  * {@link SegmentWriteState} for each flush and is maintained in the
  * corresponding {@link SegmentInfo} until it is committed.
  * <p>
- * During indexing {@link FieldInfos} uses {@link SegmentCodecsBuilder} to incrementally
- * build the {@link SegmentCodecs} mapping. Once a segment is flushed
- * DocumentsWriter creates a {@link SegmentCodecs} instance from
- * {@link FieldInfos#buildSegmentCodecs(boolean)} The {@link FieldInfo#codecId}
- * assigned by {@link SegmentCodecsBuilder} refers to the codecs ordinal
- * maintained inside {@link SegmentCodecs}. This ord is later used to get the
- * right codec when the segment is opened in a reader.The {@link PostingsFormat} returned
- * from {@link SegmentCodecs#codec()} in turn uses {@link SegmentCodecs}
- * internal structure to select and initialize the right codec for a fields when
+ * During indexing {@link FieldInfos} uses {@link SegmentFormatsBuilder} to incrementally
+ * build the {@link SegmentFormats} mapping. Once a segment is flushed
+ * DocumentsWriter creates a {@link SegmentFormats} instance from
+ * {@link FieldInfos#buildSegmentCodecs(boolean)} The {@link FieldInfo#formatId}
+ * assigned by {@link SegmentFormatsBuilder} refers to the formats ordinal
+ * maintained inside {@link SegmentFormats}. This ord is later used to get the
+ * right format when the segment is opened in a reader.The {@link PostingsFormat} returned
+ * from {@link SegmentFormats#format()} in turn uses {@link SegmentFormats}
+ * internal structure to select and initialize the right format for a fields when
  * it is written.
  * <p>
- * Once a flush succeeded the {@link SegmentCodecs} is maintained inside the
+ * Once a flush succeeded the {@link SegmentFormats} is maintained inside the
  * {@link SegmentInfo} for the flushed segment it was created for.
- * {@link SegmentInfo} writes the name of each codec in {@link SegmentCodecs}
+ * {@link SegmentInfo} writes the name of each format in {@link SegmentFormats}
  * for each segment and maintains the order. Later if a segment is opened by a
- * reader this mapping is deserialized and used to create the codec per field.
+ * reader this mapping is deserialized and used to create the format per field.
  * 
  * 
  * @lucene.internal
  */
-public final class SegmentCodecs implements Cloneable {
+public final class SegmentFormats implements Cloneable {
   /**
-   * internal structure to map codecs to fields - don't modify this from outside
+   * internal structure to map formats to fields - don't modify this from outside
    * of this class!
    */
-  public final PostingsFormat[] codecs;
-  public final CodecProvider provider;
-  private final PostingsFormat codec;
+  public final PostingsFormat[] formats;
+  public final PerFieldCodec provider;
+  private final PostingsFormat format;
   
-  public SegmentCodecs(CodecProvider provider, IndexInput input) throws IOException {
+  public SegmentFormats(PerFieldCodec provider, IndexInput input) throws IOException {
     this(provider, read(input, provider));
   }
   
-  public SegmentCodecs(CodecProvider provider, PostingsFormat... codecs) {
+  public SegmentFormats(PerFieldCodec provider, PostingsFormat... formats) {
     this.provider = provider;
-    this.codecs = codecs;
-    if (codecs.length == 1 && codecs[0] instanceof PreFlexPostingsFormat) {
-      this.codec = codecs[0]; // hack for backwards break... don't wrap the codec in preflex
+    this.formats = formats;
+    if (formats.length == 1 && formats[0] instanceof Lucene3xPostingsFormat) {
+      this.format = formats[0]; // hack for backwards break... don't wrap the codec in preflex
     } else {
-      this.codec = new PerFieldPostingsFormat(this);
+      this.format = new PerFieldPostingsFormat(this);
     }
   }
 
-  public PostingsFormat codec() {
-    return codec;
+  public PostingsFormat format() {
+    return format;
   }
 
   public void write(IndexOutput out) throws IOException {
-    out.writeVInt(codecs.length);
-    for (PostingsFormat codec : codecs) {
-      out.writeString(codec.name);
+    out.writeVInt(formats.length);
+    for (PostingsFormat format : formats) {
+      out.writeString(format.name);
     }
   }
 
-  private static PostingsFormat[] read(IndexInput in, CodecProvider provider) throws IOException {
+  private static PostingsFormat[] read(IndexInput in, PerFieldCodec provider) throws IOException {
     final int size = in.readVInt();
     final ArrayList<PostingsFormat> list = new ArrayList<PostingsFormat>();
     for (int i = 0; i < size; i++) {
@@ -110,7 +109,7 @@ public final class SegmentCodecs impleme
 
   public void files(Directory dir, SegmentInfo info, Set<String> files)
       throws IOException {
-    final PostingsFormat[] codecArray = codecs;
+    final PostingsFormat[] codecArray = formats;
     for (int i = 0; i < codecArray.length; i++) {
       codecArray[i].files(dir, info, i, files);
     }      
@@ -119,52 +118,52 @@ public final class SegmentCodecs impleme
 
   @Override
   public String toString() {
-    return "SegmentCodecs [codecs=" + Arrays.toString(codecs) + ", provider=" + provider + "]";
+    return "SegmentFormats [formats=" + Arrays.toString(formats) + ", provider=" + provider + "]";
   }
   
   /**
-   * Used in {@link FieldInfos} to incrementally build the codec ID mapping for
+   * Used in {@link FieldInfos} to incrementally build the format ID mapping for
    * {@link FieldInfo} instances.
    * <p>
    * Note: this class is not thread-safe
    * </p>
-   * @see FieldInfo#getCodecId()
+   * @see FieldInfo#getFormatId()
    */
-  public final static class SegmentCodecsBuilder {
-    private final Map<PostingsFormat, Integer> codecRegistry = new IdentityHashMap<PostingsFormat, Integer>();
-    private final ArrayList<PostingsFormat> codecs = new ArrayList<PostingsFormat>();
-    private final CodecProvider provider;
+  public final static class SegmentFormatsBuilder {
+    private final Map<PostingsFormat, Integer> formatRegistry = new IdentityHashMap<PostingsFormat, Integer>();
+    private final ArrayList<PostingsFormat> formats = new ArrayList<PostingsFormat>();
+    private final PerFieldCodec provider;
 
-    private SegmentCodecsBuilder(CodecProvider provider) {
+    private SegmentFormatsBuilder(PerFieldCodec provider) {
       this.provider = provider;
     }
     
-    public static SegmentCodecsBuilder create(CodecProvider provider) {
-      return new SegmentCodecsBuilder(provider);
+    public static SegmentFormatsBuilder create(PerFieldCodec provider) {
+      return new SegmentFormatsBuilder(provider);
     }
     
-    public SegmentCodecsBuilder tryAddAndSet(FieldInfo fi) {
-      if (fi.getCodecId() == FieldInfo.UNASSIGNED_CODEC_ID) {
+    public SegmentFormatsBuilder tryAddAndSet(FieldInfo fi) {
+      if (fi.getFormatId() == FieldInfo.UNASSIGNED_FORMAT_ID) {
         final PostingsFormat fieldCodec = provider.lookup(provider
-            .getFieldCodec(fi.name));
-        Integer ord = codecRegistry.get(fieldCodec);
+            .getPostingsFormat(fi.name));
+        Integer ord = formatRegistry.get(fieldCodec);
         if (ord == null) {
-          ord = Integer.valueOf(codecs.size());
-          codecRegistry.put(fieldCodec, ord);
-          codecs.add(fieldCodec);
+          ord = Integer.valueOf(formats.size());
+          formatRegistry.put(fieldCodec, ord);
+          formats.add(fieldCodec);
         }
-        fi.setCodecId(ord.intValue());
+        fi.setFormatId(ord.intValue());
       }
       return this;
     }
     
-    public SegmentCodecs build() {
-      return new SegmentCodecs(provider, codecs.toArray(PostingsFormat.EMPTY));
+    public SegmentFormats build() {
+      return new SegmentFormats(provider, formats.toArray(PostingsFormat.EMPTY));
     }
     
-    public SegmentCodecsBuilder clear() {
-      codecRegistry.clear();
-      codecs.clear();
+    public SegmentFormatsBuilder clear() {
+      formatRegistry.clear();
+      formats.clear();
       return this;
     }
   }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsFormat.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -35,9 +35,9 @@ import org.apache.lucene.index.codecs.Fi
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.index.codecs.PerDocConsumer;
 import org.apache.lucene.index.codecs.PerDocValues;
-import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
-import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
-import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsReader;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsWriter;
 import org.apache.lucene.store.Directory;
 
 /** This codec "inlines" the postings for terms that have
@@ -82,7 +82,7 @@ public class PulsingPostingsFormat exten
     // We wrap StandardPostingsWriter, but any PostingsWriterBase
     // will work:
 
-    PostingsWriterBase docsWriter = new StandardPostingsWriter(state);
+    PostingsWriterBase docsWriter = new Lucene40PostingsWriter(state);
 
     // Terms that have <= freqCutoff number of docs are
     // "pulsed" (inlined):
@@ -106,7 +106,7 @@ public class PulsingPostingsFormat exten
 
     // We wrap StandardPostingsReader, but any StandardPostingsReader
     // will work:
-    PostingsReaderBase docsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
+    PostingsReaderBase docsReader = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.formatId);
     PostingsReaderBase pulsingReader = new PulsingPostingsReader(docsReader);
 
     boolean success = false;
@@ -115,7 +115,7 @@ public class PulsingPostingsFormat exten
                                                     state.dir, state.fieldInfos, state.segmentInfo.name,
                                                     pulsingReader,
                                                     state.context,
-                                                    state.codecId,
+                                                    state.formatId,
                                                     state.termsIndexDivisor);
       success = true;
       return ret;
@@ -132,14 +132,14 @@ public class PulsingPostingsFormat exten
 
   @Override
   public void files(Directory dir, SegmentInfo segmentInfo, int codecID, Set<String> files) throws IOException {
-    StandardPostingsReader.files(dir, segmentInfo, codecID, files);
+    Lucene40PostingsReader.files(dir, segmentInfo, codecID, files);
     BlockTreeTermsReader.files(dir, segmentInfo, codecID, files);
     DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files);
   }
 
   @Override
   public void getExtensions(Set<String> extensions) {
-    StandardPostingsFormat.getStandardExtensions(extensions);
+    Lucene40PostingsFormat.getStandardExtensions(extensions);
     DefaultDocValuesConsumer.getExtensions(extensions);
   }
 

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesConsumer.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesConsumer.java Wed Oct 26 18:33:48 2011
@@ -47,12 +47,12 @@ public class SepDocValuesConsumer extend
   }
 
   @SuppressWarnings("fallthrough")
-  public static void files(Directory dir, SegmentInfo segmentInfo, int codecId,
+  public static void files(Directory dir, SegmentInfo segmentInfo, int formatId,
       Set<String> files) throws IOException {
     FieldInfos fieldInfos = segmentInfo.getFieldInfos();
     for (FieldInfo fieldInfo : fieldInfos) {
-      if (fieldInfo.getCodecId() == codecId && fieldInfo.hasDocValues()) {
-        String filename = docValuesId(segmentInfo.name, codecId, fieldInfo.number);
+      if (fieldInfo.getFormatId() == formatId && fieldInfo.hasDocValues()) {
+        String filename = docValuesId(segmentInfo.name, formatId, fieldInfo.number);
         switch (fieldInfo.getDocValues()) {
           case BYTES_FIXED_DEREF:
           case BYTES_VAR_DEREF:

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesProducer.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesProducer.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepDocValuesProducer.java Wed Oct 26 18:33:48 2011
@@ -39,7 +39,7 @@ public class SepDocValuesProducer extend
    * {@link IndexDocValues} instances for this segment and codec.
    */
   public SepDocValuesProducer(SegmentReadState state) throws IOException {
-    docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.docCount, state.dir, state.codecId, state.context);
+    docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.docCount, state.dir, state.formatId, state.context);
   }
   
   @Override

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java Wed Oct 26 18:33:48 2011
@@ -116,27 +116,27 @@ public final class SepPostingsWriter ext
     try {
       this.skipInterval = skipInterval;
       this.skipMinimum = skipInterval; /* set to the same for now */
-      final String docFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, DOC_EXTENSION);
+      final String docFileName = IndexFileNames.segmentFileName(state.segmentName, state.formatId, DOC_EXTENSION);
       docOut = factory.createOutput(state.directory, docFileName, state.context);
       docIndex = docOut.index();
       
       if (state.fieldInfos.hasFreq()) {
-        final String frqFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, FREQ_EXTENSION);
+        final String frqFileName = IndexFileNames.segmentFileName(state.segmentName, state.formatId, FREQ_EXTENSION);
         freqOut = factory.createOutput(state.directory, frqFileName, state.context);
         freqIndex = freqOut.index();
       }
 
       if (state.fieldInfos.hasProx()) {      
-        final String posFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, POS_EXTENSION);
+        final String posFileName = IndexFileNames.segmentFileName(state.segmentName, state.formatId, POS_EXTENSION);
         posOut = factory.createOutput(state.directory, posFileName, state.context);
         posIndex = posOut.index();
         
         // TODO: -- only if at least one field stores payloads?
-        final String payloadFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, PAYLOAD_EXTENSION);
+        final String payloadFileName = IndexFileNames.segmentFileName(state.segmentName, state.formatId, PAYLOAD_EXTENSION);
         payloadOut = state.directory.createOutput(payloadFileName, state.context);
       }
       
-      final String skipFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, SKIP_EXTENSION);
+      final String skipFileName = IndexFileNames.segmentFileName(state.segmentName, state.formatId, SKIP_EXTENSION);
       skipOut = state.directory.createOutput(skipFileName, state.context);
       
       totalNumDocs = state.numDocs;

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java Wed Oct 26 18:33:48 2011
@@ -60,7 +60,7 @@ class SimpleTextFieldsReader extends Fie
   final static BytesRef PAYLOAD = SimpleTextFieldsWriter.PAYLOAD;
 
   public SimpleTextFieldsReader(SegmentReadState state) throws IOException {
-    in = state.dir.openInput(SimpleTextPostingsFormat.getPostingsFileName(state.segmentInfo.name, state.codecId), state.context);
+    in = state.dir.openInput(SimpleTextPostingsFormat.getPostingsFileName(state.segmentInfo.name, state.formatId), state.context);
    
     fieldInfos = state.fieldInfos;
   }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java Wed Oct 26 18:33:48 2011
@@ -47,7 +47,7 @@ class SimpleTextFieldsWriter extends Fie
   final static BytesRef PAYLOAD = new BytesRef("        payload ");
 
   public SimpleTextFieldsWriter(SegmentWriteState state) throws IOException {
-    final String fileName = SimpleTextPostingsFormat.getPostingsFileName(state.segmentName, state.codecId);
+    final String fileName = SimpleTextPostingsFormat.getPostingsFileName(state.segmentName, state.formatId);
     out = state.directory.createOutput(fileName, state.context);
   }
 

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodecProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodecProvider.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodecProvider.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodecProvider.java Wed Oct 26 18:33:48 2011
@@ -27,11 +27,11 @@ import java.util.Random;
 
 import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.lucene3x.Lucene3xPostingsFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
 import org.apache.lucene.index.codecs.memory.MemoryPostingsFormat;
-import org.apache.lucene.index.codecs.preflex.PreFlexPostingsFormat;
 import org.apache.lucene.index.codecs.pulsing.PulsingPostingsFormat;
 import org.apache.lucene.index.codecs.simpletext.SimpleTextPostingsFormat;
-import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
 import org.apache.lucene.util._TestUtil;
 
 /**
@@ -54,8 +54,8 @@ public class RandomCodecProvider extends
     // block via CL:
     int minItemsPerBlock = _TestUtil.nextInt(random, 2, 100);
     int maxItemsPerBlock = 2*(Math.max(2, minItemsPerBlock-1)) + random.nextInt(100);
-    register(new StandardPostingsFormat(minItemsPerBlock, maxItemsPerBlock));
-    register(new PreFlexPostingsFormat());
+    register(new Lucene40PostingsFormat(minItemsPerBlock, maxItemsPerBlock));
+    register(new Lucene3xPostingsFormat());
     // TODO: make it possible to specify min/max iterms per
     // block via CL:
     minItemsPerBlock = _TestUtil.nextInt(random, 2, 100);

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -34,9 +34,9 @@ import org.apache.lucene.index.codecs.se
 import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
 import org.apache.lucene.index.codecs.sep.SepPostingsReader;
 import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
-import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
 import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexInput;
 import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexOutput;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.index.codecs.PerDocConsumer;
@@ -187,7 +187,7 @@ public class MockFixedIntBlockPostingsFo
                                                 state.segmentInfo.name,
                                                 postingsReader,
                                                 state.context,
-                                                StandardPostingsFormat.TERMS_CACHE_SIZE,
+                                                Lucene40PostingsFormat.TERMS_CACHE_SIZE,
                                                 state.codecId);
       success = true;
       return ret;

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -34,9 +34,9 @@ import org.apache.lucene.index.codecs.se
 import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
 import org.apache.lucene.index.codecs.sep.SepPostingsReader;
 import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
-import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
 import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexInput;
 import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexOutput;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.index.codecs.PerDocConsumer;
@@ -210,7 +210,7 @@ public class MockVariableIntBlockPosting
                                                 state.segmentInfo.name,
                                                 postingsReader,
                                                 state.context,
-                                                StandardPostingsFormat.TERMS_CACHE_SIZE,
+                                                Lucene40PostingsFormat.TERMS_CACHE_SIZE,
                                                 state.codecId);
       success = true;
       return ret;

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomPostingsFormat.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomPostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -50,6 +50,8 @@ import org.apache.lucene.index.codecs.Te
 import org.apache.lucene.index.codecs.TermsIndexWriterBase;
 import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
 import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsReader;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsWriter;
 import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
 import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
 import org.apache.lucene.index.codecs.mocksep.MockSingleIntFactory;
@@ -62,8 +64,6 @@ import org.apache.lucene.index.codecs.se
 import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
 import org.apache.lucene.index.codecs.sep.SepPostingsReader;
 import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
-import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
-import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -164,7 +164,7 @@ public class MockRandomPostingsFormat ex
       if (LuceneTestCase.VERBOSE) {
         System.out.println("MockRandomCodec: writing Standard postings");
       }
-      postingsWriter = new StandardPostingsWriter(state, skipInterval);
+      postingsWriter = new Lucene40PostingsWriter(state, skipInterval);
     }
 
     if (random.nextBoolean()) {
@@ -303,7 +303,7 @@ public class MockRandomPostingsFormat ex
       if (LuceneTestCase.VERBOSE) {
         System.out.println("MockRandomCodec: reading Standard postings");
       }
-      postingsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
+      postingsReader = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
     }
 
     if (random.nextBoolean()) {
@@ -421,7 +421,7 @@ public class MockRandomPostingsFormat ex
     final String seedFileName = IndexFileNames.segmentFileName(segmentInfo.name, codecId, SEED_EXT);    
     files.add(seedFileName);
     SepPostingsReader.files(segmentInfo, codecId, files);
-    StandardPostingsReader.files(dir, segmentInfo, codecId, files);
+    Lucene40PostingsReader.files(dir, segmentInfo, codecId, files);
     BlockTermsReader.files(dir, segmentInfo, codecId, files);
     BlockTreeTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepPostingsFormat.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepPostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -37,7 +37,7 @@ import org.apache.lucene.index.codecs.Bl
 import org.apache.lucene.index.codecs.BlockTermsWriter;
 import org.apache.lucene.index.codecs.TermsIndexReaderBase;
 import org.apache.lucene.index.codecs.TermsIndexWriterBase;
-import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
 import org.apache.lucene.index.codecs.sep.SepDocValuesConsumer;
 import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
 import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
@@ -119,7 +119,7 @@ public class MockSepPostingsFormat exten
                                                 state.segmentInfo.name,
                                                 postingsReader,
                                                 state.context,
-                                                StandardPostingsFormat.TERMS_CACHE_SIZE,
+                                                Lucene40PostingsFormat.TERMS_CACHE_SIZE,
                                                 state.codecId);
       success = true;
       return ret;

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java Wed Oct 26 18:33:48 2011
@@ -29,9 +29,9 @@ import org.apache.lucene.index.codecs.Fi
 import org.apache.lucene.index.codecs.PostingsConsumer;
 import org.apache.lucene.index.codecs.TermStats;
 import org.apache.lucene.index.codecs.TermsConsumer;
-import org.apache.lucene.index.codecs.preflex.PreFlexPostingsFormat;
-import org.apache.lucene.index.codecs.preflex.TermInfo;
-import org.apache.lucene.index.codecs.standard.DefaultSkipListWriter;
+import org.apache.lucene.index.codecs.lucene3x.Lucene3xPostingsFormat;
+import org.apache.lucene.index.codecs.lucene3x.TermInfo;
+import org.apache.lucene.index.codecs.lucene40.DefaultSkipListWriter;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
@@ -50,12 +50,12 @@ class PreFlexFieldsWriter extends Fields
                                    state.fieldInfos,
                                    state.termIndexInterval);
 
-    final String freqFile = IndexFileNames.segmentFileName(state.segmentName, "", PreFlexPostingsFormat.FREQ_EXTENSION);
+    final String freqFile = IndexFileNames.segmentFileName(state.segmentName, "", Lucene3xPostingsFormat.FREQ_EXTENSION);
     freqOut = state.directory.createOutput(freqFile, state.context);
     totalNumDocs = state.numDocs;
 
     if (state.fieldInfos.hasProx()) {
-      final String proxFile = IndexFileNames.segmentFileName(state.segmentName, "", PreFlexPostingsFormat.PROX_EXTENSION);
+      final String proxFile = IndexFileNames.segmentFileName(state.segmentName, "", Lucene3xPostingsFormat.PROX_EXTENSION);
       proxOut = state.directory.createOutput(proxFile, state.context);
     } else {
       proxOut = null;

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexRWPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexRWPostingsFormat.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexRWPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexRWPostingsFormat.java Wed Oct 26 18:33:48 2011
@@ -21,8 +21,8 @@ import java.io.IOException;
 
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.codecs.preflex.PreFlexPostingsFormat;
-import org.apache.lucene.index.codecs.preflex.PreFlexFields;
+import org.apache.lucene.index.codecs.lucene3x.Lucene3xFields;
+import org.apache.lucene.index.codecs.lucene3x.Lucene3xPostingsFormat;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.util.LuceneTestCase;
@@ -32,7 +32,7 @@ import org.apache.lucene.util.LuceneTest
  *
  * @lucene.experimental
  */
-public class PreFlexRWPostingsFormat extends PreFlexPostingsFormat {
+public class PreFlexRWPostingsFormat extends Lucene3xPostingsFormat {
 
   public PreFlexRWPostingsFormat() {
     // NOTE: we impersonate the PreFlex codec so that it can
@@ -50,7 +50,7 @@ public class PreFlexRWPostingsFormat ext
     // Whenever IW opens readers, eg for merging, we have to
     // keep terms order in UTF16:
 
-    return new PreFlexFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor) {
+    return new Lucene3xFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor) {
       @Override
       protected boolean sortTermsByUnicode() {
         // We carefully peek into stack track above us: if

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java Wed Oct 26 18:33:48 2011
@@ -23,8 +23,8 @@ import java.io.IOException;
 
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.codecs.preflex.PreFlexPostingsFormat;
-import org.apache.lucene.index.codecs.preflex.TermInfo;
+import org.apache.lucene.index.codecs.lucene3x.Lucene3xPostingsFormat;
+import org.apache.lucene.index.codecs.lucene3x.TermInfo;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
@@ -106,8 +106,8 @@ final class TermInfosWriter implements C
 
         try {
           directory.deleteFile(IndexFileNames.segmentFileName(segment, "",
-              (isIndex ? PreFlexPostingsFormat.TERMS_INDEX_EXTENSION
-                  : PreFlexPostingsFormat.TERMS_EXTENSION)));
+              (isIndex ? Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION
+                  : Lucene3xPostingsFormat.TERMS_EXTENSION)));
         } catch (IOException ignored) {
         }
       }
@@ -125,8 +125,8 @@ final class TermInfosWriter implements C
     fieldInfos = fis;
     isIndex = isi;
     output = directory.createOutput(IndexFileNames.segmentFileName(segment, "",
-        (isIndex ? PreFlexPostingsFormat.TERMS_INDEX_EXTENSION
-            : PreFlexPostingsFormat.TERMS_EXTENSION)), IOContext.DEFAULT);
+        (isIndex ? Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION
+            : Lucene3xPostingsFormat.TERMS_EXTENSION)), IOContext.DEFAULT);
     boolean success = false;
     try {
     output.writeInt(FORMAT_CURRENT);              // write format
@@ -147,8 +147,8 @@ final class TermInfosWriter implements C
 
         try {
           directory.deleteFile(IndexFileNames.segmentFileName(segment, "",
-              (isIndex ? PreFlexPostingsFormat.TERMS_INDEX_EXTENSION
-                  : PreFlexPostingsFormat.TERMS_EXTENSION)));
+              (isIndex ? Lucene3xPostingsFormat.TERMS_INDEX_EXTENSION
+                  : Lucene3xPostingsFormat.TERMS_EXTENSION)));
         } catch (IOException ignored) {
         }
       }

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java Wed Oct 26 18:33:48 2011
@@ -41,11 +41,11 @@ import org.apache.lucene.document.FieldT
 import org.apache.lucene.index.*;
 import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.lucene3x.Lucene3xPostingsFormat;
 import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
 import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
 import org.apache.lucene.index.codecs.mocksep.MockSepPostingsFormat;
 import org.apache.lucene.index.codecs.mockrandom.MockRandomPostingsFormat;
-import org.apache.lucene.index.codecs.preflex.PreFlexPostingsFormat;
 import org.apache.lucene.index.codecs.preflexrw.PreFlexRWPostingsFormat;
 import org.apache.lucene.index.codecs.pulsing.PulsingPostingsFormat;
 import org.apache.lucene.search.BooleanQuery;
@@ -293,7 +293,7 @@ public abstract class LuceneTestCase ext
       if (preFlex != null) {
         cp.unregister(preFlex);
       }
-      cp.register(new PreFlexPostingsFormat());
+      cp.register(new Lucene3xPostingsFormat());
     }
     cp.unregister(cp.lookup("MockSep"));
     cp.unregister(cp.lookup("MockFixedIntBlock"));

Modified: lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1189367&r1=1189366&r2=1189367&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java Wed Oct 26 18:33:48 2011
@@ -30,10 +30,10 @@ import org.apache.lucene.document.String
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
 import org.apache.lucene.index.codecs.mocksep.MockSepPostingsFormat;
 import org.apache.lucene.index.codecs.pulsing.PulsingPostingsFormat;
 import org.apache.lucene.index.codecs.simpletext.SimpleTextPostingsFormat;
-import org.apache.lucene.index.codecs.standard.StandardPostingsFormat;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.store.AlreadyClosedException;
@@ -1039,7 +1039,7 @@ public class TestAddIndexes extends Luce
 
   public static class MockCodecProvider extends CodecProvider {
     public MockCodecProvider() {
-      StandardPostingsFormat standardCodec = new StandardPostingsFormat();
+      Lucene40PostingsFormat standardCodec = new Lucene40PostingsFormat();
       SimpleTextPostingsFormat simpleTextCodec = new SimpleTextPostingsFormat();
       MockSepPostingsFormat mockSepCodec = new MockSepPostingsFormat();
       register(standardCodec);
@@ -1146,7 +1146,7 @@ public class TestAddIndexes extends Luce
       IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT,
           new MockAnalyzer(random));
       CodecProvider provider = new CodecProvider();
-      provider.register(new StandardPostingsFormat());
+      provider.register(new Lucene40PostingsFormat());
       conf.setCodecProvider(provider);
       IndexWriter w = new IndexWriter(toAdd, conf);
       Document doc = new Document();