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 2013/07/25 19:46:31 UTC

svn commit: r1507075 - in /lucene/dev/branches/lucene5127/lucene: codecs/src/test/org/apache/lucene/codecs/blockterms/ test-framework/src/java/org/apache/lucene/codecs/lucene41vargap/ test-framework/src/java/org/apache/lucene/index/ test-framework/src/...

Author: rmuir
Date: Thu Jul 25 17:46:31 2013
New Revision: 1507075

URL: http://svn.apache.org/r1507075
Log:
LUCENE-5127: explicit var gap testing part 1

Added:
    lucene/dev/branches/lucene5127/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestVarGapFixedIntervalPostingsFormat.java   (with props)
    lucene/dev/branches/lucene5127/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41vargap/
    lucene/dev/branches/lucene5127/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41vargap/Lucene41VarGapFixedInterval.java   (with props)
    lucene/dev/branches/lucene5127/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41vargap/package.html   (with props)
Modified:
    lucene/dev/branches/lucene5127/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestFixedGapPostingsFormat.java
    lucene/dev/branches/lucene5127/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
    lucene/dev/branches/lucene5127/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat

Modified: lucene/dev/branches/lucene5127/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestFixedGapPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5127/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestFixedGapPostingsFormat.java?rev=1507075&r1=1507074&r2=1507075&view=diff
==============================================================================
--- lucene/dev/branches/lucene5127/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestFixedGapPostingsFormat.java (original)
+++ lucene/dev/branches/lucene5127/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestFixedGapPostingsFormat.java Thu Jul 25 17:46:31 2013
@@ -25,8 +25,6 @@ import org.apache.lucene.util._TestUtil;
 /**
  * Basic tests of a PF using FixedGap terms dictionary
  */
-// TODO: we should add an instantiation for VarGap too to TestFramework, and a test in this package
-// TODO: ensure both of these are also in rotation in RandomCodec
 public class TestFixedGapPostingsFormat extends BasePostingsFormatTestCase {
   private final Codec codec = _TestUtil.alwaysPostingsFormat(new Lucene41WithOrds(_TestUtil.nextInt(random(), 1, 1000)));
 

Added: lucene/dev/branches/lucene5127/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestVarGapFixedIntervalPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5127/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestVarGapFixedIntervalPostingsFormat.java?rev=1507075&view=auto
==============================================================================
--- lucene/dev/branches/lucene5127/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestVarGapFixedIntervalPostingsFormat.java (added)
+++ lucene/dev/branches/lucene5127/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestVarGapFixedIntervalPostingsFormat.java Thu Jul 25 17:46:31 2013
@@ -0,0 +1,36 @@
+package org.apache.lucene.codecs.blockterms;
+
+/*
+ * 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 org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapFixedInterval;
+import org.apache.lucene.index.BasePostingsFormatTestCase;
+import org.apache.lucene.util._TestUtil;
+
+/**
+ * Basic tests of a PF using VariableGap terms dictionary (fixed interval)
+ */
+// nocommit: one for the other selector
+public class TestVarGapFixedIntervalPostingsFormat extends BasePostingsFormatTestCase {
+  private final Codec codec = _TestUtil.alwaysPostingsFormat(new Lucene41VarGapFixedInterval(_TestUtil.nextInt(random(), 1, 1000)));
+
+  @Override
+  protected Codec getCodec() {
+    return codec;
+  }
+}

Added: lucene/dev/branches/lucene5127/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41vargap/Lucene41VarGapFixedInterval.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5127/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41vargap/Lucene41VarGapFixedInterval.java?rev=1507075&view=auto
==============================================================================
--- lucene/dev/branches/lucene5127/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41vargap/Lucene41VarGapFixedInterval.java (added)
+++ lucene/dev/branches/lucene5127/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41vargap/Lucene41VarGapFixedInterval.java Thu Jul 25 17:46:31 2013
@@ -0,0 +1,141 @@
+package org.apache.lucene.codecs.lucene41vargap;
+
+/*
+ * 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 org.apache.lucene.codecs.FieldsConsumer;
+import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.PostingsReaderBase;
+import org.apache.lucene.codecs.PostingsWriterBase;
+import org.apache.lucene.codecs.blockterms.BlockTermsReader;
+import org.apache.lucene.codecs.blockterms.BlockTermsWriter;
+import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexWriter;
+import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase;
+import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase;
+import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader;
+import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+
+// TODO: we could make separate base class that can wrapp
+// any PostingsBaseFormat and make it ord-able...
+
+/**
+ * Customized version of {@link Lucene41PostingsFormat} that uses
+ * {@link VariableGapTermsIndexWriter} with a fixed interval.
+ */
+public final class Lucene41VarGapFixedInterval extends PostingsFormat {
+  final int termIndexInterval;
+  
+  public Lucene41VarGapFixedInterval() {
+    this(FixedGapTermsIndexWriter.DEFAULT_TERM_INDEX_INTERVAL);
+  }
+  
+  public Lucene41VarGapFixedInterval(int termIndexInterval) {
+    super("Lucene41VarGapFixedInterval");
+    this.termIndexInterval = termIndexInterval;
+  }
+
+  @Override
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    PostingsWriterBase docs = new Lucene41PostingsWriter(state);
+
+    // TODO: should we make the terms index more easily
+    // pluggable?  Ie so that this codec would record which
+    // index impl was used, and switch on loading?
+    // Or... you must make a new Codec for this?
+    TermsIndexWriterBase indexWriter;
+    boolean success = false;
+    try {
+      indexWriter = new VariableGapTermsIndexWriter(state, new VariableGapTermsIndexWriter.EveryNTermSelector(termIndexInterval));
+      success = true;
+    } finally {
+      if (!success) {
+        docs.close();
+      }
+    }
+
+    success = false;
+    try {
+      // Must use BlockTermsWriter (not BlockTree) because
+      // BlockTree doens't support ords (yet)...
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, docs);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        try {
+          docs.close();
+        } finally {
+          indexWriter.close();
+        }
+      }
+    }
+  }
+
+  @Override
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+    PostingsReaderBase postings = new Lucene41PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
+    TermsIndexReaderBase indexReader;
+
+    boolean success = false;
+    try {
+      indexReader = new VariableGapTermsIndexReader(state.directory,
+                                                 state.fieldInfos,
+                                                 state.segmentInfo.name,
+                                                 state.segmentSuffix, state.context);
+      success = true;
+    } finally {
+      if (!success) {
+        postings.close();
+      }
+    }
+
+    success = false;
+    try {
+      FieldsProducer ret = new BlockTermsReader(indexReader,
+                                                state.directory,
+                                                state.fieldInfos,
+                                                state.segmentInfo,
+                                                postings,
+                                                state.context,
+                                                state.segmentSuffix);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        try {
+          postings.close();
+        } finally {
+          indexReader.close();
+        }
+      }
+    }
+  }
+
+  /** Extension of freq postings file */
+  static final String FREQ_EXTENSION = "frq";
+
+  /** Extension of prox postings file */
+  static final String PROX_EXTENSION = "prx";
+}

Added: lucene/dev/branches/lucene5127/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41vargap/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5127/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41vargap/package.html?rev=1507075&view=auto
==============================================================================
--- lucene/dev/branches/lucene5127/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41vargap/package.html (added)
+++ lucene/dev/branches/lucene5127/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene41vargap/package.html Thu Jul 25 17:46:31 2013
@@ -0,0 +1,25 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<html>
+<head>
+   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+</head>
+<body>
+Codecs for testing that support {@link org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader}
+</body>
+</html>

Modified: lucene/dev/branches/lucene5127/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5127/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java?rev=1507075&r1=1507074&r2=1507075&view=diff
==============================================================================
--- lucene/dev/branches/lucene5127/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java (original)
+++ lucene/dev/branches/lucene5127/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java Thu Jul 25 17:46:31 2013
@@ -33,6 +33,7 @@ import org.apache.lucene.codecs.assertin
 import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
 import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
+import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapFixedInterval;
 import org.apache.lucene.codecs.lucene42.Lucene42Codec;
 import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat;
 import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
@@ -137,7 +138,8 @@ public class RandomCodec extends Lucene4
         new MockVariableIntBlockPostingsFormat( _TestUtil.nextInt(random, 1, 127)),
         new MockRandomPostingsFormat(random),
         new NestedPulsingPostingsFormat(),
-        new Lucene41WithOrds(),
+        new Lucene41WithOrds(_TestUtil.nextInt(random, 1, 1000)),
+        new Lucene41VarGapFixedInterval(_TestUtil.nextInt(random, 1, 1000)),
         new SimpleTextPostingsFormat(),
         new AssertingPostingsFormat(),
         new MemoryPostingsFormat(true, random.nextFloat()),

Modified: lucene/dev/branches/lucene5127/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5127/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat?rev=1507075&r1=1507074&r2=1507075&view=diff
==============================================================================
--- lucene/dev/branches/lucene5127/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat (original)
+++ lucene/dev/branches/lucene5127/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat Thu Jul 25 17:46:31 2013
@@ -20,6 +20,7 @@ org.apache.lucene.codecs.mocksep.MockSep
 org.apache.lucene.codecs.nestedpulsing.NestedPulsingPostingsFormat
 org.apache.lucene.codecs.ramonly.RAMOnlyPostingsFormat
 org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds
+org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapFixedInterval
 org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings
 org.apache.lucene.codecs.asserting.AssertingPostingsFormat
 org.apache.lucene.codecs.lucene40.Lucene40RWPostingsFormat