You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2011/10/29 21:02:53 UTC

svn commit: r1194974 - in /lucene/dev/branches/lucene2621/lucene/src: java/org/apache/lucene/index/ java/org/apache/lucene/index/codecs/ java/org/apache/lucene/index/codecs/lucene40/ java/org/apache/lucene/index/codecs/pulsing/ test-framework/org/apach...

Author: mikemccand
Date: Sat Oct 29 19:02:52 2011
New Revision: 1194974

URL: http://svn.apache.org/viewvc?rev=1194974&view=rev
Log:
get TestExternalCodecs to compile; add PostingsBaseFormat so PulsingPostingsFormat can wrap others

Added:
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsBaseFormat.java   (with props)
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsBaseFormat.java   (with props)
Modified:
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentReadState.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/CoreCodecProvider.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsWriterBase.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsFormat.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodec.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/TestExternalCodecs.java
    lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestPerFieldCodecSupport.java

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentReadState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentReadState.java?rev=1194974&r1=1194973&r2=1194974&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentReadState.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/SegmentReadState.java Sat Oct 29 19:02:52 2011
@@ -57,12 +57,12 @@ public class SegmentReadState {
   }
 
   public SegmentReadState(SegmentReadState other,
-                          int newFormatID) {
+                          int newFormatId) {
     this.dir = other.dir;
     this.segmentInfo = other.segmentInfo;
     this.fieldInfos = other.fieldInfos;
     this.context = other.context;
     this.termsIndexDivisor = other.termsIndexDivisor;
-    this.formatId = newFormatID;
+    this.formatId = newFormatId;
   }
-}
\ No newline at end of file
+}

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/CoreCodecProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/CoreCodecProvider.java?rev=1194974&r1=1194973&r2=1194974&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/CoreCodecProvider.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/CoreCodecProvider.java Sat Oct 29 19:02:52 2011
@@ -48,7 +48,8 @@ public class CoreCodecProvider extends C
   public Codec lookup(String name) {
     return CORE_CODECS.get(name);
   }
-  
+
+  // nocommit should we make this an unmodifiable map?
   /** Lucene's core codecs
    *  @lucene.internal
    */

Added: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsBaseFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsBaseFormat.java?rev=1194974&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsBaseFormat.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsBaseFormat.java Sat Oct 29 19:02:52 2011
@@ -0,0 +1,51 @@
+package org.apache.lucene.index.codecs;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.lucene.index.PerDocWriteState;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.store.Directory;
+
+/** 
+ * Provides a {@link PostingsReaderBase} and {@link
+ * PostingsWriterBase}.
+ *
+ * @lucene.experimental */
+
+// nocommit better name?
+public abstract class PostingsBaseFormat {
+
+  /** Unique name that's used to retrieve this codec when
+   *  reading the index */
+  public final String name;
+  
+  protected PostingsBaseFormat(String name) {
+    this.name = name;
+  }
+
+  public abstract PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException;
+
+  public abstract PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException;
+
+  public abstract void files(Directory dir, SegmentInfo segmentInfo, int codecID, Set<String> files) throws IOException;
+}

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java?rev=1194974&r1=1194973&r2=1194974&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java Sat Oct 29 19:02:52 2011
@@ -36,6 +36,7 @@ import org.apache.lucene.util.Bits;
  *  time. 
  *  @lucene.experimental */
 
+// nocommit better name?
 public abstract class PostingsReaderBase implements Closeable {
 
   public abstract void init(IndexInput termsIn) throws IOException;

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsWriterBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsWriterBase.java?rev=1194974&r1=1194973&r2=1194974&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsWriterBase.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/PostingsWriterBase.java Sat Oct 29 19:02:52 2011
@@ -27,6 +27,7 @@ import org.apache.lucene.index.FieldInfo
  * @lucene.experimental
  */
 
+// nocommit better name?
 public abstract class PostingsWriterBase extends PostingsConsumer implements Closeable {
 
   public abstract void start(IndexOutput termsOut) throws IOException;

Modified: 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=1194974&r1=1194973&r2=1194974&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java Sat Oct 29 19:02:52 2011
@@ -38,6 +38,8 @@ import org.apache.lucene.index.codecs.si
  *
  * @lucene.experimental
  */
+// nocommit maybe rename to Lucene4XCodec?  This is supposed
+// to "last" through all 4.x releases...?  or maybe not?
 public class Lucene40Codec extends Codec {
   private final FieldsFormat fieldsFormat = new DefaultFieldsFormat();
   private final DocValuesFormat docValuesFormat = new DefaultDocValuesFormat();
@@ -95,7 +97,7 @@ public class Lucene40Codec extends Codec
   public 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("Pulsing", new PulsingPostingsFormat(new Lucene40PostingsBaseFormat(), 1));
     CORE_FORMATS.put("SimpleText", new SimpleTextPostingsFormat());
     CORE_FORMATS.put("Memory", new MemoryPostingsFormat());
   }

Added: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsBaseFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsBaseFormat.java?rev=1194974&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsBaseFormat.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsBaseFormat.java Sat Oct 29 19:02:52 2011
@@ -0,0 +1,62 @@
+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.Set;
+
+import org.apache.lucene.index.PerDocWriteState;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.codecs.PostingsBaseFormat;
+import org.apache.lucene.index.codecs.PostingsReaderBase;
+import org.apache.lucene.index.codecs.PostingsWriterBase;
+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;
+
+/** 
+ * Provides a {@link PostingsReaderBase} and {@link
+ * PostingsWriterBase}.
+ *
+ * @lucene.experimental */
+
+public final class Lucene40PostingsBaseFormat extends PostingsBaseFormat {
+
+  // nocommit static INSTANCE?
+  public Lucene40PostingsBaseFormat() {
+    super("Lucene40");
+  }
+
+  @Override
+  public PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException {
+    return new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.formatId);
+  }
+
+  @Override
+  public PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException {
+    return new Lucene40PostingsWriter(state);
+  }
+  
+  @Override
+  public void files(Directory dir, SegmentInfo segmentInfo, int formatID, Set<String> files) throws IOException {
+    Lucene40PostingsReader.files(dir, segmentInfo, formatID, files);
+  }
+}

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=1194974&r1=1194973&r2=1194974&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 Sat Oct 29 19:02:52 2011
@@ -27,12 +27,10 @@ import org.apache.lucene.index.codecs.Po
 import org.apache.lucene.index.codecs.PostingsWriterBase;
 import org.apache.lucene.index.codecs.BlockTreeTermsReader;
 import org.apache.lucene.index.codecs.BlockTreeTermsWriter;
+import org.apache.lucene.index.codecs.PostingsBaseFormat;
 import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
-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 postings format "inlines" the postings for terms that have
@@ -48,23 +46,21 @@ public class PulsingPostingsFormat exten
   private final int freqCutoff;
   private final int minBlockSize;
   private final int maxBlockSize;
+  private final PostingsBaseFormat wrappedPostingsBaseFormat;
 
-  public PulsingPostingsFormat() {
-    this(1);
-  }
-  
-  public PulsingPostingsFormat(int freqCutoff) {
-    this(freqCutoff, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
+  public PulsingPostingsFormat(PostingsBaseFormat wrappedPostingsBaseFormat, int freqCutoff) {
+    this(wrappedPostingsBaseFormat, freqCutoff, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
   }
 
   /** Terms with freq <= freqCutoff are inlined into terms
    *  dict. */
-  public PulsingPostingsFormat(int freqCutoff, int minBlockSize, int maxBlockSize) {
+  public PulsingPostingsFormat(PostingsBaseFormat wrappedPostingsBaseFormat, int freqCutoff, int minBlockSize, int maxBlockSize) {
     super("Pulsing");
     this.freqCutoff = freqCutoff;
     this.minBlockSize = minBlockSize;
     assert minBlockSize > 1;
     this.maxBlockSize = maxBlockSize;
+    this.wrappedPostingsBaseFormat = wrappedPostingsBaseFormat;
   }
 
   @Override
@@ -74,10 +70,7 @@ public class PulsingPostingsFormat exten
 
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    // We wrap StandardPostingsWriter, but any PostingsWriterBase
-    // will work:
-
-    PostingsWriterBase docsWriter = new Lucene40PostingsWriter(state);
+    PostingsWriterBase docsWriter = wrappedPostingsBaseFormat.postingsWriterBase(state);
 
     // Terms that have <= freqCutoff number of docs are
     // "pulsed" (inlined):
@@ -99,9 +92,7 @@ public class PulsingPostingsFormat exten
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
 
-    // We wrap StandardPostingsReader, but any StandardPostingsReader
-    // will work:
-    PostingsReaderBase docsReader = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.formatId);
+    PostingsReaderBase docsReader = wrappedPostingsBaseFormat.postingsReaderBase(state);
     PostingsReaderBase pulsingReader = new PulsingPostingsReader(docsReader);
 
     boolean success = false;
@@ -127,7 +118,7 @@ public class PulsingPostingsFormat exten
 
   @Override
   public void files(Directory dir, SegmentInfo segmentInfo, int codecID, Set<String> files) throws IOException {
-    Lucene40PostingsReader.files(dir, segmentInfo, codecID, files);
+    wrappedPostingsBaseFormat.files(dir, segmentInfo, codecID, files);
     BlockTreeTermsReader.files(dir, segmentInfo, codecID, files);
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodec.java?rev=1194974&r1=1194973&r2=1194974&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/RandomCodec.java Sat Oct 29 19:02:52 2011
@@ -27,6 +27,7 @@ import java.util.Random;
 
 import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsBaseFormat;
 import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
 import org.apache.lucene.index.codecs.memory.MemoryPostingsFormat;
 import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
@@ -91,7 +92,7 @@ public class RandomCodec extends Lucene4
     // block via CL:
     minItemsPerBlock = _TestUtil.nextInt(random, 2, 100);
     maxItemsPerBlock = 2*(Math.max(1, minItemsPerBlock-1)) + random.nextInt(100);
-    register(new PulsingPostingsFormat( 1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock));
+    register(new PulsingPostingsFormat(new Lucene40PostingsBaseFormat(), 1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock));
     register(new MockSepPostingsFormat());
     register(new MockFixedIntBlockPostingsFormat(_TestUtil.nextInt(random, 1, 2000)));
     register(new MockVariableIntBlockPostingsFormat( _TestUtil.nextInt(random, 1, 127)));

Modified: lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/TestExternalCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/TestExternalCodecs.java?rev=1194974&r1=1194973&r2=1194974&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/TestExternalCodecs.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/TestExternalCodecs.java Sat Oct 29 19:02:52 2011
@@ -17,16 +17,21 @@ package org.apache.lucene;
  * limitations under the License.
  */
 
-import org.apache.lucene.util.*;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.index.*;
-import org.apache.lucene.document.*;
-import org.apache.lucene.search.*;
+import java.io.*;
+import java.util.*;
+
 import org.apache.lucene.analysis.*;
+import org.apache.lucene.document.*;
+import org.apache.lucene.index.*;
 import org.apache.lucene.index.codecs.*;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsBaseFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
+import org.apache.lucene.index.codecs.perfield.PerFieldPostingsFormat;
+import org.apache.lucene.index.codecs.pulsing.PulsingPostingsFormat;
+import org.apache.lucene.search.*;
 import org.apache.lucene.store.*;
-import java.util.*;
-import java.io.*;
+import org.apache.lucene.util.*;
+import org.apache.lucene.util.Bits;
 
 /* Intentionally outside of oal.index to verify fully
    external codecs work fine */
@@ -70,11 +75,12 @@ public class TestExternalCodecs extends 
   // TODO
   //   - good improvement would be to write through to disk,
   //     and then load into ram from disk
-  public static class RAMOnlyCodec extends PostingsFormat {
-    
-    public RAMOnlyCodec() {
-      super("RamOnly");
+  public static class RAMOnlyPostingsFormat extends PostingsFormat {
+
+    public RAMOnlyPostingsFormat() {
+      super("RAMOnlyPostingsFormat");
     }
+    
     // Postings state:
     static class RAMPostings extends FieldsProducer {
       final Map<String,RAMField> fieldToTerms = new TreeMap<String,RAMField>();
@@ -127,7 +133,7 @@ public class TestExternalCodecs extends 
 
       @Override
       public TermsEnum iterator() {
-        return new RAMTermsEnum(RAMOnlyCodec.RAMField.this);
+        return new RAMTermsEnum(RAMOnlyPostingsFormat.RAMField.this);
       }
 
       @Override
@@ -502,13 +508,39 @@ public class TestExternalCodecs extends 
     }
   }
 
+  private static class CustomPerFieldPostingsFormat extends PerFieldPostingsFormat {
+    private final PostingsFormat ramFormat = new RAMOnlyPostingsFormat();
+    private final PostingsFormat defaultFormat = new Lucene40PostingsFormat();
+    private final PostingsFormat pulsingFormat = new PulsingPostingsFormat(new Lucene40PostingsBaseFormat(), 1);
+
+    @Override
+    public String getPostingsFormatForField(String field) {
+      if (field.equals("field2") || field.equals("id")) {
+        return "Pulsing";
+      } else if (field.equals("field1")) {
+        return "Default";
+      } else {
+        return "RAM";
+      }
+    }
+
+    @Override
+    public PostingsFormat getPostingsFormat(String formatName) {
+      if (formatName.equals("Pulsing")) {
+        return pulsingFormat;
+      } else if (formatName.equals("Default")) {
+        return defaultFormat;
+      } else {
+        assertEquals("RAM", formatName);
+        return ramFormat;
+      }
+    }
+  }
+
   // tests storing "id" and "field2" fields as pulsing codec,
   // whose term sort is backwards unicode code point, and
   // storing "field1" as a custom entirely-in-RAM codec
   public void testPerFieldCodec() throws Exception {
-    CodecProvider provider = new CoreCodecProvider();
-    provider.register(new RAMOnlyCodec());
-    provider.setDefaultFieldCodec("RamOnly");
     
     final int NUM_DOCS = atLeast(173);
     MockDirectoryWrapper dir = newDirectory();
@@ -516,7 +548,7 @@ public class TestExternalCodecs extends 
     IndexWriter w = new IndexWriter(
         dir,
         newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).
-            setCodecProvider(provider).
+        setCodecProvider(_TestUtil.alwaysFormat(new CustomPerFieldPostingsFormat())).
             setMergePolicy(newLogMergePolicy(3))
     );
     w.setInfoStream(VERBOSE ? System.out : null);
@@ -525,11 +557,9 @@ public class TestExternalCodecs extends 
     doc.add(newField("field1", "this field uses the standard codec as the test", TextField.TYPE_UNSTORED));
     // uses pulsing codec:
     Field field2 = newField("field2", "this field uses the pulsing codec as the test", TextField.TYPE_UNSTORED);
-    provider.setFieldCodec(field2.name(), "Pulsing");
     doc.add(field2);
     
     Field idField = newField("id", "", StringField.TYPE_UNSTORED);
-    provider.setFieldCodec(idField.name(), "Pulsing");
 
     doc.add(idField);
     for(int i=0;i<NUM_DOCS;i++) {

Modified: lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestPerFieldCodecSupport.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestPerFieldCodecSupport.java?rev=1194974&r1=1194973&r2=1194974&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestPerFieldCodecSupport.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestPerFieldCodecSupport.java Sat Oct 29 19:02:52 2011
@@ -34,7 +34,6 @@ import org.apache.lucene.index.codecs.lu
 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.perfield.SegmentFormats;
 import org.apache.lucene.index.codecs.pulsing.PulsingPostingsFormat;
 import org.apache.lucene.index.codecs.simpletext.SimpleTextPostingsFormat;
 import org.apache.lucene.search.IndexSearcher;