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/28 22:26:11 UTC

svn commit: r1190548 - in /lucene/dev/branches/lucene2621/lucene/src: java/org/apache/lucene/index/codecs/lucene40/ test-framework/org/apache/lucene/index/codecs/mockintblock/ test-framework/org/apache/lucene/index/codecs/mockrandom/ test-framework/org...

Author: rmuir
Date: Fri Oct 28 20:26:10 2011
New Revision: 1190548

URL: http://svn.apache.org/viewvc?rev=1190548&view=rev
Log:
LUCENE-2490: Cleanup Lucene4xCodec and mocksep stuff in src/test

Added:
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepDocValuesFormat.java   (with props)
Modified:
    lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomPostingsFormat.java
    lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepPostingsFormat.java

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=1190548&r1=1190547&r2=1190548&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 Fri Oct 28 20:26:10 2011
@@ -17,8 +17,6 @@ package org.apache.lucene.index.codecs.l
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -41,41 +39,23 @@ import org.apache.lucene.index.codecs.si
  *
  * @lucene.experimental
  */
-// TODO: which postings formats will we actually support for backwards compatibility?
 public class Lucene40Codec extends Codec {
   private final FieldsFormat fieldsFormat = new DefaultFieldsFormat();
   private final DocValuesFormat docValuesFormat = new DefaultDocValuesFormat();
-  private final PostingsFormat postingsFormat;
-  private final String defaultPostingsFormat;
+  private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
+    @Override
+    protected String getPostingsFormatForField(FieldInfo field) {
+      return Lucene40Codec.this.getPostingsFormatForField(field);
+    }
+
+    @Override
+    protected PostingsFormat getPostingsFormat(String formatName) {
+      return Lucene40Codec.this.getPostingsFormat(formatName);
+    }
+  };
 
   public Lucene40Codec() {
-    this(Collections.<String,String>emptyMap());
-  }
-  
-  public Lucene40Codec(Map<String,String> perFieldMap) {
-    this("Lucene40", perFieldMap);
-  }
-  
- public Lucene40Codec(final String defaultPostingsFormat, final Map<String,String> perFieldMap) {
     super("Lucene40");
-    this.defaultPostingsFormat = defaultPostingsFormat;
-
-    postingsFormat = new PerFieldPostingsFormat() {
-
-      @Override
-      protected String getPostingsFormatForField(FieldInfo field) {
-        String format = perFieldMap.get(field.name);
-        if (format == null) {
-          format = defaultPostingsFormat;
-        }
-        return format;
-      }
-
-      @Override
-      protected PostingsFormat getPostingsFormat(String formatName) {
-        return CORE_FORMATS.get(formatName);
-      }
-    };
   }
   
   @Override
@@ -93,8 +73,27 @@ public class Lucene40Codec extends Codec
     return postingsFormat;
   }
   
-  // postings formats
-  private static final Map<String,PostingsFormat> CORE_FORMATS = new HashMap<String,PostingsFormat>();
+  /** Looks up a postings format by name, by default. 
+   * 
+   * The default looks up from {@link #CORE_FORMATS}.
+   */
+  public PostingsFormat getPostingsFormat(String formatName) {
+    return CORE_FORMATS.get(formatName);
+  }
+  
+  /** Returns the postings format that should be used for writing 
+   *  new segments of <code>field</code>.
+   *  
+   *  The default implementation always returns "Lucene40"
+   */
+  protected String getPostingsFormatForField(FieldInfo field) {
+    return "Lucene40";
+  }
+  
+  /** Lucene's core postings formats.
+   *  @lucene.internal
+   */
+  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());

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=1190548&r1=1190547&r2=1190548&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 Fri Oct 28 20:26:10 2011
@@ -207,16 +207,5 @@ public class MockFixedIntBlockPostingsFo
     SepPostingsReader.files(segmentInfo, formatId, files);
     BlockTermsReader.files(dir, segmentInfo, formatId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, formatId, files);
-    SepDocValuesConsumer.files(dir, segmentInfo, formatId, files);
-  }
-
-  @Override
-  public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
-    return new SepDocValuesConsumer(state);
-  }
-
-  @Override
-  public PerDocValues docsProducer(SegmentReadState state) throws IOException {
-    return new SepDocValuesProducer(state);
   }
 }

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=1190548&r1=1190547&r2=1190548&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 Fri Oct 28 20:26:10 2011
@@ -230,16 +230,5 @@ public class MockVariableIntBlockPosting
     SepPostingsReader.files(segmentInfo, formatId, files);
     BlockTermsReader.files(dir, segmentInfo, formatId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, formatId, files);
-    SepDocValuesConsumer.files(dir, segmentInfo, formatId, files);
-  }
-
-  @Override
-  public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
-    return new SepDocValuesConsumer(state);
-  }
-
-  @Override
-  public PerDocValues docsProducer(SegmentReadState state) throws IOException {
-    return new SepDocValuesProducer(state);
   }
 }

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=1190548&r1=1190547&r2=1190548&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 Fri Oct 28 20:26:10 2011
@@ -26,7 +26,6 @@ import java.util.Set;
 
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.PerDocWriteState;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
@@ -35,14 +34,10 @@ import org.apache.lucene.index.codecs.Bl
 import org.apache.lucene.index.codecs.BlockTermsReader;
 import org.apache.lucene.index.codecs.BlockTermsWriter;
 import org.apache.lucene.index.codecs.PostingsFormat;
-import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
-import org.apache.lucene.index.codecs.PerDocConsumer;
-import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
-import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.index.codecs.PostingsReaderBase;
 import org.apache.lucene.index.codecs.PostingsWriterBase;
 import org.apache.lucene.index.codecs.TermStats;
@@ -60,8 +55,6 @@ import org.apache.lucene.index.codecs.pu
 import org.apache.lucene.index.codecs.sep.IntIndexInput;
 import org.apache.lucene.index.codecs.sep.IntIndexOutput;
 import org.apache.lucene.index.codecs.sep.IntStreamFactory;
-import org.apache.lucene.index.codecs.sep.SepDocValuesConsumer;
-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.store.Directory;
@@ -77,13 +70,11 @@ import org.apache.lucene.util._TestUtil;
  */
 
 public class MockRandomPostingsFormat extends PostingsFormat {
-  private final boolean useSepDocValues;
   private final Random seedRandom;
   private final String SEED_EXT = "sd";
   
   public MockRandomPostingsFormat(Random random) {
     super("MockRandom");
-    this.useSepDocValues = random.nextBoolean();
     this.seedRandom = new Random(random.nextLong());
   }
 
@@ -426,11 +417,6 @@ public class MockRandomPostingsFormat ex
     BlockTreeTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
     VariableGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
-    if (useSepDocValues) {
-      SepDocValuesConsumer.files(dir, segmentInfo, codecId, files);
-    } else {
-      DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files);
-    }
     // hackish!
     Iterator<String> it = files.iterator();
     while(it.hasNext()) {
@@ -441,23 +427,4 @@ public class MockRandomPostingsFormat ex
     }
     //System.out.println("MockRandom.files return " + files);
   }
-
-  // can we make this more evil?
-  @Override
-  public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
-    if (useSepDocValues) {
-      return new SepDocValuesConsumer(state);
-    } else {
-      return new DefaultDocValuesConsumer(state);
-    }
-  }
-
-  @Override
-  public PerDocValues docsProducer(SegmentReadState state) throws IOException {
-    if (useSepDocValues) {
-      return new SepDocValuesProducer(state);
-    } else {
-      return new DefaultDocValuesProducer(state);
-    }
-  }
 }

Added: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepDocValuesFormat.java?rev=1190548&view=auto
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepDocValuesFormat.java (added)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepDocValuesFormat.java Fri Oct 28 20:26:10 2011
@@ -0,0 +1,54 @@
+package org.apache.lucene.index.codecs.mocksep;
+
+/**
+ * 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.codecs.DocValuesFormat;
+import org.apache.lucene.index.codecs.PerDocConsumer;
+import org.apache.lucene.index.codecs.PerDocValues;
+import org.apache.lucene.index.codecs.sep.SepDocValuesConsumer;
+import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
+import org.apache.lucene.store.Directory;
+
+/**
+ * Separate-file docvalues implementation
+ * @lucene.experimental
+ */
+// TODO: we could move this out of src/test ?
+public class MockSepDocValuesFormat extends DocValuesFormat {
+
+  @Override
+  public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
+    return new SepDocValuesConsumer(state);
+  }
+
+  @Override
+  public PerDocValues docsProducer(SegmentReadState state) throws IOException {
+    return new SepDocValuesProducer(state);
+  }
+
+  @Override
+  public void files(Directory dir, SegmentInfo info, int formatId, Set<String> files) throws IOException {
+    SepDocValuesConsumer.files(dir, info, formatId, 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=1190548&r1=1190547&r2=1190548&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 Fri Oct 28 20:26:10 2011
@@ -139,16 +139,5 @@ public class MockSepPostingsFormat exten
     SepPostingsReader.files(segmentInfo, formatId, files);
     BlockTermsReader.files(dir, segmentInfo, formatId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, formatId, files);
-    SepDocValuesConsumer.files(dir, segmentInfo, formatId, files);
-  }
-
-  @Override
-  public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
-    return new SepDocValuesConsumer(state);
-  }
-
-  @Override
-  public PerDocValues docsProducer(SegmentReadState state) throws IOException {
-    return new SepDocValuesProducer(state);
   }
 }