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 2010/12/21 23:35:31 UTC

svn commit: r1051687 - in /lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs: CodecProvider.java bulkvint/ bulkvint/BulkVIntCodec.java

Author: rmuir
Date: Tue Dec 21 22:35:31 2010
New Revision: 1051687

URL: http://svn.apache.org/viewvc?rev=1051687&view=rev
Log:
LUCENE-2723: add a vint-based intblock codec for benchmarking purposes

Added:
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/bulkvint/
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/bulkvint/BulkVIntCodec.java   (with props)
Modified:
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java?rev=1051687&r1=1051686&r2=1051687&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java Tue Dec 21 22:35:31 2010
@@ -23,6 +23,7 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.lucene.index.codecs.bulkvint.BulkVIntCodec;
 import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
 import org.apache.lucene.index.codecs.pulsing.PulsingCodec;
 import org.apache.lucene.index.codecs.simpletext.SimpleTextCodec;
@@ -50,7 +51,7 @@ public class CodecProvider {
 
   private final Set<String> knownExtensions = new HashSet<String>();
 
-  public final static String[] CORE_CODECS = new String[] {"Standard", "Pulsing", "PreFlex", "SimpleText", "PatchedFrameOfRef", "FrameOfRef", "PatchedFrameOfRef2"};
+  public final static String[] CORE_CODECS = new String[] {"Standard", "Pulsing", "PreFlex", "SimpleText", "PatchedFrameOfRef", "FrameOfRef", "PatchedFrameOfRef2", "BulkVInt"};
 
   public synchronized void register(Codec codec) {
     if (codec.name == null) {
@@ -176,5 +177,6 @@ class DefaultCodecProvider extends Codec
     register(new PatchedFrameOfRefCodec());
     register(new FrameOfRefCodec());
     register(new PForDeltaFixedIntBlockCodec(128));
+    register(new BulkVIntCodec(128));
   }
 }

Added: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/bulkvint/BulkVIntCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/bulkvint/BulkVIntCodec.java?rev=1051687&view=auto
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/bulkvint/BulkVIntCodec.java (added)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/bulkvint/BulkVIntCodec.java Tue Dec 21 22:35:31 2010
@@ -0,0 +1,226 @@
+package org.apache.lucene.index.codecs.bulkvint;
+
+/**
+ * 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.SegmentInfo;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.FieldsConsumer;
+import org.apache.lucene.index.codecs.FieldsProducer;
+import org.apache.lucene.index.codecs.sep.IntStreamFactory;
+import org.apache.lucene.index.codecs.sep.IntIndexInput;
+import org.apache.lucene.index.codecs.sep.IntIndexOutput;
+import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
+import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
+import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexInput;
+import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexOutput;
+import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
+import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
+import org.apache.lucene.index.codecs.PostingsWriterBase;
+import org.apache.lucene.index.codecs.PostingsReaderBase;
+import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
+import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
+import org.apache.lucene.index.codecs.TermsIndexReaderBase;
+import org.apache.lucene.index.codecs.TermsIndexWriterBase;
+import org.apache.lucene.index.codecs.standard.StandardCodec;
+import org.apache.lucene.store.*;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Silly codec that acts like MockFixedIntBlockCodec mostly (uses vint encoding):
+ * writes a single vint header (uncompressed size of the block in bytes)
+ * writes the block as a list of vints
+ */
+
+public class BulkVIntCodec extends Codec {
+
+  private final int blockSize;
+
+  public BulkVIntCodec(int blockSize) {
+    this.blockSize = blockSize;
+    name = "BulkVInt";
+  }
+
+  @Override
+  public String toString() {
+    return name + "(blockSize=" + blockSize + ")";
+  }
+
+  // only for testing
+  public IntStreamFactory getIntFactory() {
+    return new BulkVIntFactory();
+  }
+
+  private class BulkVIntFactory extends IntStreamFactory {
+
+    @Override
+    public IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException {
+      return new FixedIntBlockIndexInput(dir.openInput(fileName, readBufferSize)) {
+
+        @Override
+        protected BlockReader getBlockReader(final IndexInput in, final int[] buffer) throws IOException {
+          return new BlockReader() {
+            final byte bytes[] = new byte[blockSize*5]; // header * max(Vint)
+            
+            public void readBlock() throws IOException {
+              final int numBytes = in.readVInt(); // read header
+              in.readBytes(bytes, 0, numBytes); // readBytes
+              
+              int upto = 0;
+              
+              // decode bytes
+              for(int i=0;i<buffer.length;i++) {
+                byte b = bytes[upto++];
+                int j = b & 0x7F;
+                for (int shift = 7; (b & 0x80) != 0; shift += 7) {
+                  b = bytes[upto++];
+                  j |= (b & 0x7F) << shift;
+                }
+                buffer[i] = j;
+              }
+              
+            }
+          };
+        }
+      };
+    }
+
+    @Override
+    public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
+      return new FixedIntBlockIndexOutput(dir.createOutput(fileName), blockSize) {
+        final byte bytes[] = new byte[blockSize*5]; // header * max(Vint)
+        
+        @Override
+        protected void flushBlock() throws IOException {
+          int upto = 0;
+          
+          // encode ints
+          for(int i=0;i<buffer.length;i++) {
+            int j = buffer[i];
+            while ((j & ~0x7F) != 0) {
+              bytes[upto++] = (byte)((j & 0x7f) | 0x80);
+              j >>>= 7;
+            }
+            bytes[upto++] = (byte)j;
+          }
+              
+          // write header (length in bytes)
+          out.writeVInt(upto);
+          
+          // write block
+          out.writeBytes(bytes, 0, upto);
+        }
+      };
+    }
+  }
+
+  @Override
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new BulkVIntFactory());
+
+    boolean success = false;
+    TermsIndexWriterBase indexWriter;
+    try {
+      indexWriter = new FixedGapTermsIndexWriter(state);
+      success = true;
+    } finally {
+      if (!success) {
+        postingsWriter.close();
+      }
+    }
+
+    success = false;
+    try {
+      FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        try {
+          postingsWriter.close();
+        } finally {
+          indexWriter.close();
+        }
+      }
+    }
+  }
+
+  @Override
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+    PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir,
+                                                                      state.segmentInfo,
+                                                                      state.readBufferSize,
+                                                                      new BulkVIntFactory(), state.codecId);
+
+    TermsIndexReaderBase indexReader;
+    boolean success = false;
+    try {
+      indexReader = new FixedGapTermsIndexReader(state.dir,
+                                                       state.fieldInfos,
+                                                       state.segmentInfo.name,
+                                                       state.termsIndexDivisor,
+                                                       BytesRef.getUTF8SortedAsUnicodeComparator(), state.codecId);
+      success = true;
+    } finally {
+      if (!success) {
+        postingsReader.close();
+      }
+    }
+
+    success = false;
+    try {
+      FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
+                                                       state.dir,
+                                                       state.fieldInfos,
+                                                       state.segmentInfo.name,
+                                                       postingsReader,
+                                                       state.readBufferSize,
+                                                       BytesRef.getUTF8SortedAsUnicodeComparator(),
+                                                       StandardCodec.TERMS_CACHE_SIZE,
+                                                       state.codecId);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        try {
+          postingsReader.close();
+        } finally {
+          indexReader.close();
+        }
+      }
+    }
+  }
+
+  @Override
+  public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) {
+    SepPostingsReaderImpl.files(segmentInfo, codecId, files);
+    PrefixCodedTermsReader.files(dir, segmentInfo, codecId, files);
+    FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
+  }
+
+  @Override
+  public void getExtensions(Set<String> extensions) {
+    SepPostingsWriterImpl.getExtensions(extensions);
+    PrefixCodedTermsReader.getExtensions(extensions);
+    FixedGapTermsIndexReader.getIndexExtensions(extensions);
+  }
+}