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/12/08 17:32:55 UTC

svn commit: r1211962 - in /lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs: ./ appending/

Author: rmuir
Date: Thu Dec  8 16:32:54 2011
New Revision: 1211962

URL: http://svn.apache.org/viewvc?rev=1211962&view=rev
Log:
LUCENE-3621: switch appendingcodec to use appending blocktree

Added:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingTermsReader.java   (with props)
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingTermsWriter.java   (with props)
Removed:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingTermsDictReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingTermsDictWriter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingTermsIndexReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingTermsIndexWriter.java
Modified:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsWriter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingPostingsFormat.java

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java?rev=1211962&r1=1211961&r2=1211962&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java Thu Dec  8 16:32:54 2011
@@ -166,16 +166,16 @@ public class BlockTreeTermsReader extend
   }
 
   protected void readHeader(IndexInput input) throws IOException {
-    CodecUtil.checkHeader(input, BlockTreeTermsWriter.CODEC_NAME,
-                          BlockTreeTermsWriter.VERSION_START,
-                          BlockTreeTermsWriter.VERSION_CURRENT);
+    CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_CODEC_NAME,
+                          BlockTreeTermsWriter.TERMS_VERSION_START,
+                          BlockTreeTermsWriter.TERMS_VERSION_CURRENT);
     dirOffset = input.readLong();    
   }
 
   protected void readIndexHeader(IndexInput input) throws IOException {
-    CodecUtil.checkHeader(input, BlockTreeTermsWriter.CODEC_NAME,
-                          BlockTreeTermsWriter.VERSION_START,
-                          BlockTreeTermsWriter.VERSION_CURRENT);
+    CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_INDEX_CODEC_NAME,
+                          BlockTreeTermsWriter.TERMS_INDEX_VERSION_START,
+                          BlockTreeTermsWriter.TERMS_INDEX_VERSION_CURRENT);
     indexDirOffset = input.readLong();    
   }
   

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsWriter.java?rev=1211962&r1=1211961&r2=1211962&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsWriter.java Thu Dec  8 16:32:54 2011
@@ -99,18 +99,21 @@ public class BlockTreeTermsWriter extend
   static final int OUTPUT_FLAG_IS_FLOOR = 0x1;
   static final int OUTPUT_FLAG_HAS_TERMS = 0x2;
 
-  final static String CODEC_NAME = "BLOCK_TREE_TERMS_DICT";
-
-  // Initial format
-  public static final int VERSION_START = 0;
-
-  public static final int VERSION_CURRENT = VERSION_START;
-
   /** Extension of terms file */
   static final String TERMS_EXTENSION = "tim";
+  final static String TERMS_CODEC_NAME = "BLOCK_TREE_TERMS_DICT";
+  // Initial format
+  public static final int TERMS_VERSION_START = 0;
+  public static final int TERMS_VERSION_CURRENT = TERMS_VERSION_START;
+
+  /** Extension of terms index file */
   static final String TERMS_INDEX_EXTENSION = "tip";
+  final static String TERMS_INDEX_CODEC_NAME = "BLOCK_TREE_TERMS_INDEX";
+  // Initial format
+  public static final int TERMS_INDEX_VERSION_START = 0;
+  public static final int TERMS_INDEX_VERSION_CURRENT = TERMS_INDEX_VERSION_START;
 
-  protected final IndexOutput out;
+  private final IndexOutput out;
   private final IndexOutput indexOut;
   final int minItemsInBlock;
   final int maxItemsInBlock;
@@ -178,22 +181,22 @@ public class BlockTreeTermsWriter extend
   }
   
   protected void writeHeader(IndexOutput out) throws IOException {
-    CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT); 
+    CodecUtil.writeHeader(out, TERMS_CODEC_NAME, TERMS_VERSION_CURRENT); 
     out.writeLong(0);                             // leave space for end index pointer    
   }
 
   protected void writeIndexHeader(IndexOutput out) throws IOException {
-    CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT); 
+    CodecUtil.writeHeader(out, TERMS_INDEX_CODEC_NAME, TERMS_INDEX_VERSION_CURRENT); 
     out.writeLong(0);                             // leave space for end index pointer    
   }
 
-  protected void writeTrailer(long dirStart) throws IOException {
-    out.seek(CodecUtil.headerLength(CODEC_NAME));
+  protected void writeTrailer(IndexOutput out, long dirStart) throws IOException {
+    out.seek(CodecUtil.headerLength(TERMS_CODEC_NAME));
     out.writeLong(dirStart);    
   }
 
-  protected void writeIndexTrailer(long dirStart) throws IOException {
-    indexOut.seek(CodecUtil.headerLength(CODEC_NAME));
+  protected void writeIndexTrailer(IndexOutput indexOut, long dirStart) throws IOException {
+    indexOut.seek(CodecUtil.headerLength(TERMS_INDEX_CODEC_NAME));
     indexOut.writeLong(dirStart);    
   }
   
@@ -935,8 +938,8 @@ public class BlockTreeTermsWriter extend
           indexOut.writeVLong(field.indexStartFP);
         }
       }
-      writeTrailer(dirStart);
-      writeIndexTrailer(indexDirStart);
+      writeTrailer(out, dirStart);
+      writeIndexTrailer(indexOut, indexDirStart);
     } catch (IOException ioe2) {
       ioe = ioe2;
     } finally {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingPostingsFormat.java?rev=1211962&r1=1211961&r2=1211962&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingPostingsFormat.java Thu Dec  8 16:32:54 2011
@@ -23,19 +23,16 @@ import java.util.Set;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.codecs.BlockTreeTermsReader;
+import org.apache.lucene.index.codecs.BlockTreeTermsWriter;
 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.FixedGapTermsIndexReader;
-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.index.codecs.PostingsReaderBase;
 import org.apache.lucene.index.codecs.PostingsWriterBase;
-import org.apache.lucene.index.codecs.BlockTermsReader;
-import org.apache.lucene.index.codecs.TermsIndexReaderBase;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
 
 /**
  * Appending postings impl
@@ -48,72 +45,39 @@ class AppendingPostingsFormat extends Po
   }
 
   @Override
-  public FieldsConsumer fieldsConsumer(SegmentWriteState state)
-          throws IOException {
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
     PostingsWriterBase docsWriter = new Lucene40PostingsWriter(state);
     boolean success = false;
-    AppendingTermsIndexWriter indexWriter = null;
     try {
-      indexWriter = new AppendingTermsIndexWriter(state);
-      success = true;
-    } finally {
-      if (!success) {
-        docsWriter.close();
-      }
-    }
-    success = false;
-    try {
-      FieldsConsumer ret = new AppendingTermsDictWriter(indexWriter, state, docsWriter);
+      FieldsConsumer ret = new AppendingTermsWriter(state, docsWriter, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
       success = true;
       return ret;
     } finally {
       if (!success) {
-        try {
-          docsWriter.close();
-        } finally {
-          indexWriter.close();
-        }
+        docsWriter.close();
       }
     }
   }
 
   @Override
-  public FieldsProducer fieldsProducer(SegmentReadState state)
-          throws IOException {
-    PostingsReaderBase docsReader = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix);
-    TermsIndexReaderBase indexReader;
-
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+    PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix);
+    
     boolean success = false;
     try {
-      indexReader = new AppendingTermsIndexReader(state.dir,
-              state.fieldInfos,
-              state.segmentInfo.name,
-              state.termsIndexDivisor,
-              BytesRef.getUTF8SortedAsUnicodeComparator(),
-              state.segmentSuffix, state.context);
-      success = true;
-    } finally {
-      if (!success) {
-        docsReader.close();
-      }
-    }
-    success = false;
-    try {
-      FieldsProducer ret = new AppendingTermsDictReader(indexReader,
-              state.dir, state.fieldInfos, state.segmentInfo.name,
-              docsReader,
-              state.context,
-              Lucene40PostingsFormat.TERMS_CACHE_SIZE,
-              state.segmentSuffix);
+      FieldsProducer ret = new AppendingTermsReader(
+                                                    state.dir,
+                                                    state.fieldInfos,
+                                                    state.segmentInfo.name,
+                                                    postings,
+                                                    state.context,
+                                                    state.segmentSuffix,
+                                                    state.termsIndexDivisor);
       success = true;
       return ret;
     } finally {
       if (!success) {
-        try {
-          docsReader.close();
-        } finally {
-          indexReader.close();
-        }
+        postings.close();
       }
     }
   }
@@ -122,7 +86,6 @@ class AppendingPostingsFormat extends Po
   public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files)
           throws IOException {
     Lucene40PostingsReader.files(dir, segmentInfo, segmentSuffix, files);
-    BlockTermsReader.files(dir, segmentInfo, segmentSuffix, files);
-    FixedGapTermsIndexReader.files(dir, segmentInfo, segmentSuffix, files);
+    BlockTreeTermsReader.files(dir, segmentInfo, segmentSuffix, files);
   }
 }

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingTermsReader.java?rev=1211962&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingTermsReader.java (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingTermsReader.java Thu Dec  8 16:32:54 2011
@@ -0,0 +1,61 @@
+package org.apache.lucene.index.codecs.appending;
+
+/**
+ * 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.index.FieldInfos;
+import org.apache.lucene.index.codecs.BlockTreeTermsReader;
+import org.apache.lucene.index.codecs.PostingsReaderBase;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.CodecUtil;
+
+/**
+ * Reads append-only terms from {@link AppendingTermsWriter}
+ * @lucene.experimental
+ */
+public class AppendingTermsReader extends BlockTreeTermsReader {
+
+  public AppendingTermsReader(Directory dir, FieldInfos fieldInfos, String segment, PostingsReaderBase postingsReader, 
+      IOContext ioContext, String segmentSuffix, int indexDivisor) throws IOException {
+    super(dir, fieldInfos, segment, postingsReader, ioContext, segmentSuffix, indexDivisor);
+  }
+
+  @Override
+  protected void readHeader(IndexInput input) throws IOException {
+    CodecUtil.checkHeader(input, AppendingTermsWriter.TERMS_CODEC_NAME,
+        AppendingTermsWriter.TERMS_VERSION_START,
+        AppendingTermsWriter.TERMS_VERSION_CURRENT);  
+  }
+
+  @Override
+  protected void readIndexHeader(IndexInput input) throws IOException {
+    CodecUtil.checkHeader(input, AppendingTermsWriter.TERMS_INDEX_CODEC_NAME,
+        AppendingTermsWriter.TERMS_INDEX_VERSION_START,
+        AppendingTermsWriter.TERMS_INDEX_VERSION_CURRENT);
+  }
+  
+  @Override
+  protected void seekDir(IndexInput input, long dirOffset) throws IOException {
+    input.seek(input.length() - Long.SIZE / 8);
+    long offset = input.readLong();
+    input.seek(offset);
+  }
+}

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingTermsWriter.java?rev=1211962&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingTermsWriter.java (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/appending/AppendingTermsWriter.java Thu Dec  8 16:32:54 2011
@@ -0,0 +1,64 @@
+package org.apache.lucene.index.codecs.appending;
+
+/**
+ * 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.index.SegmentWriteState;
+import org.apache.lucene.index.codecs.BlockTreeTermsWriter;
+import org.apache.lucene.index.codecs.PostingsWriterBase;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.CodecUtil;
+
+/**
+ * Append-only version of {@link BlockTreeTermsWriter}
+ * @lucene.experimental
+ */
+public class AppendingTermsWriter extends BlockTreeTermsWriter {
+  final static String TERMS_CODEC_NAME = "APPENDING_TERMS_DICT";
+  final static int TERMS_VERSION_START = 0;
+  final static int TERMS_VERSION_CURRENT = TERMS_VERSION_START;
+  
+  final static String TERMS_INDEX_CODEC_NAME = "APPENDING_TERMS_INDEX";
+  final static int TERMS_INDEX_VERSION_START = 0;
+  final static int TERMS_INDEX_VERSION_CURRENT = TERMS_INDEX_VERSION_START;
+  
+  public AppendingTermsWriter(SegmentWriteState state, PostingsWriterBase postingsWriter, int minItemsInBlock, int maxItemsInBlock) throws IOException {
+    super(state, postingsWriter, minItemsInBlock, maxItemsInBlock);
+  }
+
+  @Override
+  protected void writeHeader(IndexOutput out) throws IOException {
+    CodecUtil.writeHeader(out, TERMS_CODEC_NAME, TERMS_VERSION_CURRENT);
+  }
+
+  @Override
+  protected void writeIndexHeader(IndexOutput out) throws IOException {
+    CodecUtil.writeHeader(out, TERMS_INDEX_CODEC_NAME, TERMS_INDEX_VERSION_CURRENT);
+  }
+
+  @Override
+  protected void writeTrailer(IndexOutput out, long dirStart) throws IOException {
+    out.writeLong(dirStart);
+  }
+
+  @Override
+  protected void writeIndexTrailer(IndexOutput indexOut, long dirStart) throws IOException {
+    indexOut.writeLong(dirStart);
+  }
+}