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 2012/08/10 00:29:37 UTC

svn commit: r1371519 - in /lucene/dev/branches/pforcodec_3892/lucene/core/src: java/org/apache/lucene/codecs/block/ java/org/apache/lucene/codecs/blockpacked/ resources/META-INF/services/ test/org/apache/lucene/codecs/block/ test/org/apache/lucene/code...

Author: mikemccand
Date: Thu Aug  9 22:29:36 2012
New Revision: 1371519

URL: http://svn.apache.org/viewvc?rev=1371519&view=rev
Log:
LUCENE-3892: replace Block with BlockPacked

Added:
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/
      - copied from r1371500, lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPostingsFormat.java
      - copied, changed from r1371500, lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsFormat.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPostingsReader.java
      - copied, changed from r1371500, lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsReader.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPostingsWriter.java
      - copied, changed from r1371500, lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsWriter.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockSkipReader.java
      - copied, changed from r1371500, lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipReader.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockSkipWriter.java
      - copied, changed from r1371500, lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipWriter.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/block/
      - copied from r1371500, lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/blockpacked/
Removed:
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPackedPostingsFormat.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPackedPostingsReader.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPackedPostingsWriter.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPackedSkipReader.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPackedSkipWriter.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/
    lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/blockpacked/
Modified:
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/ForUtil.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
    lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/block/TestForUtil.java

Copied: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPostingsFormat.java (from r1371500, lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsFormat.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPostingsFormat.java?p2=lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPostingsFormat.java&p1=lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsFormat.java&r1=1371500&r2=1371519&rev=1371519&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsFormat.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPostingsFormat.java Thu Aug  9 22:29:36 2012
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.blockpacked;
+package org.apache.lucene.codecs.block;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -34,7 +34,7 @@ import org.apache.lucene.util.IOUtils;
  * Encodes/decode postings in packed int blocks for faster
  * decode.
  */
-public final class BlockPackedPostingsFormat extends PostingsFormat {
+public final class BlockPostingsFormat extends PostingsFormat {
   public static final String DOC_EXTENSION = "doc";
   public static final String POS_EXTENSION = "pos";
   public static final String PAY_EXTENSION = "pay";
@@ -47,12 +47,12 @@ public final class BlockPackedPostingsFo
   // NOTE: must be factor of ... 64?
   public final static int BLOCK_SIZE = 128;
 
-  public BlockPackedPostingsFormat() {
+  public BlockPostingsFormat() {
     this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
   }
 
-  public BlockPackedPostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
-    super("BlockPacked");
+  public BlockPostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
+    super("Block");
     this.minTermBlockSize = minTermBlockSize;
     assert minTermBlockSize > 1;
     this.maxTermBlockSize = maxTermBlockSize;
@@ -66,7 +66,7 @@ public final class BlockPackedPostingsFo
 
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    PostingsWriterBase postingsWriter = new BlockPackedPostingsWriter(state);
+    PostingsWriterBase postingsWriter = new BlockPostingsWriter(state);
 
     boolean success = false;
     try {
@@ -85,11 +85,11 @@ public final class BlockPackedPostingsFo
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    PostingsReaderBase postingsReader = new BlockPackedPostingsReader(state.dir,
-                                                                      state.fieldInfos,
-                                                                      state.segmentInfo,
-                                                                      state.context,
-                                                                      state.segmentSuffix);
+    PostingsReaderBase postingsReader = new BlockPostingsReader(state.dir,
+                                                                state.fieldInfos,
+                                                                state.segmentInfo,
+                                                                state.context,
+                                                                state.segmentSuffix);
     boolean success = false;
     try {
       FieldsProducer ret = new BlockTreeTermsReader(state.dir,

Copied: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPostingsReader.java (from r1371500, lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsReader.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPostingsReader.java?p2=lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPostingsReader.java&p1=lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsReader.java&r1=1371500&r2=1371519&rev=1371519&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsReader.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPostingsReader.java Thu Aug  9 22:29:36 2012
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.blockpacked;
+package org.apache.lucene.codecs.block;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,9 +17,9 @@ package org.apache.lucene.codecs.blockpa
  * limitations under the License.
  */
 
-import static org.apache.lucene.codecs.blockpacked.BlockPackedPostingsFormat.BLOCK_SIZE;
-import static org.apache.lucene.codecs.blockpacked.ForUtil.MAX_DATA_SIZE;
-import static org.apache.lucene.codecs.blockpacked.ForUtil.MAX_ENCODED_SIZE;
+import static org.apache.lucene.codecs.block.BlockPostingsFormat.BLOCK_SIZE;
+import static org.apache.lucene.codecs.block.ForUtil.MAX_DATA_SIZE;
+import static org.apache.lucene.codecs.block.ForUtil.MAX_ENCODED_SIZE;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -49,10 +49,10 @@ import org.apache.lucene.util.IOUtils;
  * Concrete class that reads docId(maybe frq,pos,offset,payloads) list
  * with postings format.
  *
- * @see BlockPackedSkipReader for details
+ * @see BlockSkipReader for details
  *
  */
-public final class BlockPackedPostingsReader extends PostingsReaderBase {
+public final class BlockPostingsReader extends PostingsReaderBase {
 
   private final IndexInput docIn;
   private final IndexInput posIn;
@@ -65,36 +65,36 @@ public final class BlockPackedPostingsRe
   // nocommit
   final String segment;
 
-  public BlockPackedPostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException {
+  public BlockPostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException {
     boolean success = false;
     segment = segmentInfo.name;
     IndexInput docIn = null;
     IndexInput posIn = null;
     IndexInput payIn = null;
     try {
-      docIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockPackedPostingsFormat.DOC_EXTENSION),
+      docIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockPostingsFormat.DOC_EXTENSION),
                             ioContext);
       CodecUtil.checkHeader(docIn,
-                            BlockPackedPostingsWriter.DOC_CODEC,
-                            BlockPackedPostingsWriter.VERSION_START,
-                            BlockPackedPostingsWriter.VERSION_START);
+                            BlockPostingsWriter.DOC_CODEC,
+                            BlockPostingsWriter.VERSION_START,
+                            BlockPostingsWriter.VERSION_START);
       forUtil = new ForUtil(docIn);
 
       if (fieldInfos.hasProx()) {
-        posIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockPackedPostingsFormat.POS_EXTENSION),
+        posIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockPostingsFormat.POS_EXTENSION),
                               ioContext);
         CodecUtil.checkHeader(posIn,
-                              BlockPackedPostingsWriter.POS_CODEC,
-                              BlockPackedPostingsWriter.VERSION_START,
-                              BlockPackedPostingsWriter.VERSION_START);
+                              BlockPostingsWriter.POS_CODEC,
+                              BlockPostingsWriter.VERSION_START,
+                              BlockPostingsWriter.VERSION_START);
 
         if (fieldInfos.hasPayloads() || fieldInfos.hasOffsets()) {
-          payIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockPackedPostingsFormat.PAY_EXTENSION),
+          payIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockPostingsFormat.PAY_EXTENSION),
                                 ioContext);
           CodecUtil.checkHeader(payIn,
-                                BlockPackedPostingsWriter.PAY_CODEC,
-                                BlockPackedPostingsWriter.VERSION_START,
-                                BlockPackedPostingsWriter.VERSION_START);
+                                BlockPostingsWriter.PAY_CODEC,
+                                BlockPostingsWriter.VERSION_START,
+                                BlockPostingsWriter.VERSION_START);
         }
       }
 
@@ -113,12 +113,12 @@ public final class BlockPackedPostingsRe
   public void init(IndexInput termsIn) throws IOException {
     // Make sure we are talking to the matching past writer
     CodecUtil.checkHeader(termsIn,
-                          BlockPackedPostingsWriter.TERMS_CODEC,
-                          BlockPackedPostingsWriter.VERSION_START,
-                          BlockPackedPostingsWriter.VERSION_START);
+                          BlockPostingsWriter.TERMS_CODEC,
+                          BlockPostingsWriter.VERSION_START,
+                          BlockPostingsWriter.VERSION_START);
     final int indexBlockSize = termsIn.readVInt();
     if (indexBlockSize != BLOCK_SIZE) {
-      throw new IllegalStateException("index-time blockSize (" + indexBlockSize + ") != read-time blockSize (" + BLOCK_SIZE + ")");
+      throw new IllegalStateException("index-time BLOCK_SIZE (" + indexBlockSize + ") != read-time BLOCK_SIZE (" + BLOCK_SIZE + ")");
     }
   }
 
@@ -325,7 +325,7 @@ public final class BlockPackedPostingsRe
 
     private int docBufferUpto;
 
-    private BlockPackedSkipReader skipper;
+    private BlockSkipReader skipper;
     private boolean skipped;
 
     final IndexInput startDocIn;
@@ -357,7 +357,7 @@ public final class BlockPackedPostingsRe
     private Bits liveDocs;
 
     public BlockDocsEnum(FieldInfo fieldInfo) throws IOException {
-      this.startDocIn = BlockPackedPostingsReader.this.docIn;
+      this.startDocIn = BlockPostingsReader.this.docIn;
       this.docIn = (IndexInput) startDocIn.clone();
       indexHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
       indexHasPos = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
@@ -490,8 +490,8 @@ public final class BlockPackedPostingsRe
 
         if (skipper == null) {
           // Lazy init: first time this enum has ever been used for skipping
-          skipper = new BlockPackedSkipReader((IndexInput) docIn.clone(),
-                                        BlockPackedPostingsWriter.maxSkipLevels,
+          skipper = new BlockSkipReader((IndexInput) docIn.clone(),
+                                        BlockPostingsWriter.maxSkipLevels,
                                         BLOCK_SIZE,
                                         indexHasPos,
                                         indexHasOffsets,
@@ -577,7 +577,7 @@ public final class BlockPackedPostingsRe
     private int docBufferUpto;
     private int posBufferUpto;
 
-    private BlockPackedSkipReader skipper;
+    private BlockSkipReader skipper;
     private boolean skipped;
 
     final IndexInput startDocIn;
@@ -628,9 +628,9 @@ public final class BlockPackedPostingsRe
     private Bits liveDocs;
     
     public BlockDocsAndPositionsEnum(FieldInfo fieldInfo) throws IOException {
-      this.startDocIn = BlockPackedPostingsReader.this.docIn;
+      this.startDocIn = BlockPostingsReader.this.docIn;
       this.docIn = (IndexInput) startDocIn.clone();
-      this.posIn = (IndexInput) BlockPackedPostingsReader.this.posIn.clone();
+      this.posIn = (IndexInput) BlockPostingsReader.this.posIn.clone();
       encoded = new byte[MAX_ENCODED_SIZE];
       indexHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
       indexHasPayloads = fieldInfo.hasPayloads();
@@ -795,8 +795,8 @@ public final class BlockPackedPostingsRe
           if (DEBUG) {
             System.out.println("    create skipper");
           }
-          skipper = new BlockPackedSkipReader((IndexInput) docIn.clone(),
-                                        BlockPackedPostingsWriter.maxSkipLevels,
+          skipper = new BlockSkipReader((IndexInput) docIn.clone(),
+                                        BlockPostingsWriter.maxSkipLevels,
                                         BLOCK_SIZE,
                                         true,
                                         indexHasOffsets,
@@ -993,7 +993,7 @@ public final class BlockPackedPostingsRe
     private int docBufferUpto;
     private int posBufferUpto;
 
-    private BlockPackedSkipReader skipper;
+    private BlockSkipReader skipper;
     private boolean skipped;
 
     final IndexInput startDocIn;
@@ -1050,10 +1050,10 @@ public final class BlockPackedPostingsRe
     private Bits liveDocs;
     
     public EverythingEnum(FieldInfo fieldInfo) throws IOException {
-      this.startDocIn = BlockPackedPostingsReader.this.docIn;
+      this.startDocIn = BlockPostingsReader.this.docIn;
       this.docIn = (IndexInput) startDocIn.clone();
-      this.posIn = (IndexInput) BlockPackedPostingsReader.this.posIn.clone();
-      this.payIn = (IndexInput) BlockPackedPostingsReader.this.payIn.clone();
+      this.posIn = (IndexInput) BlockPostingsReader.this.posIn.clone();
+      this.payIn = (IndexInput) BlockPostingsReader.this.payIn.clone();
       encoded = new byte[MAX_ENCODED_SIZE];
       indexHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
       if (indexHasOffsets) {
@@ -1288,8 +1288,8 @@ public final class BlockPackedPostingsRe
           if (DEBUG) {
             System.out.println("    create skipper");
           }
-          skipper = new BlockPackedSkipReader((IndexInput) docIn.clone(),
-                                        BlockPackedPostingsWriter.maxSkipLevels,
+          skipper = new BlockSkipReader((IndexInput) docIn.clone(),
+                                        BlockPostingsWriter.maxSkipLevels,
                                         BLOCK_SIZE,
                                         true,
                                         indexHasOffsets,
@@ -1337,7 +1337,6 @@ public final class BlockPackedPostingsRe
       }
 
       // Now scan:
-      // Now scan:
       while (true) {
         if (DEBUG) {
           System.out.println("  scan doc=" + accum + " docBufferUpto=" + docBufferUpto);

Copied: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPostingsWriter.java (from r1371500, lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsWriter.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPostingsWriter.java?p2=lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPostingsWriter.java&p1=lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsWriter.java&r1=1371500&r2=1371519&rev=1371519&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsWriter.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockPostingsWriter.java Thu Aug  9 22:29:36 2012
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.blockpacked;
+package org.apache.lucene.codecs.block;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,10 +17,10 @@ package org.apache.lucene.codecs.blockpa
  * limitations under the License.
  */
 
-import static org.apache.lucene.codecs.blockpacked.BlockPackedPostingsFormat.BLOCK_SIZE;
-import static org.apache.lucene.codecs.blockpacked.BlockPackedPostingsReader.DEBUG;
-import static org.apache.lucene.codecs.blockpacked.ForUtil.MAX_DATA_SIZE;
-import static org.apache.lucene.codecs.blockpacked.ForUtil.MAX_ENCODED_SIZE;
+import static org.apache.lucene.codecs.block.BlockPostingsFormat.BLOCK_SIZE;
+import static org.apache.lucene.codecs.block.BlockPostingsReader.DEBUG;
+import static org.apache.lucene.codecs.block.ForUtil.MAX_DATA_SIZE;
+import static org.apache.lucene.codecs.block.ForUtil.MAX_ENCODED_SIZE;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -48,19 +48,19 @@ import org.apache.lucene.util.packed.Pac
  *
  * Postings list for each term will be stored separately. 
  *
- * @see BlockPackedSkipWriter for details about skipping setting and postings layout.
+ * @see BlockSkipWriter for details about skipping setting and postings layout.
  *
  */
-public final class BlockPackedPostingsWriter extends PostingsWriterBase {
+public final class BlockPostingsWriter extends PostingsWriterBase {
 
   // nocommit move these constants to the PF:
 
   static final int maxSkipLevels = 10;
 
-  final static String TERMS_CODEC = "BlockPackedPostingsWriterTerms";
-  final static String DOC_CODEC = "BlockPackedPostingsWriterDoc";
-  final static String POS_CODEC = "BlockPackedPostingsWriterPos";
-  final static String PAY_CODEC = "BlockPackedPostingsWriterPay";
+  final static String TERMS_CODEC = "BlockPostingsWriterTerms";
+  final static String DOC_CODEC = "BlockPostingsWriterDoc";
+  final static String POS_CODEC = "BlockPostingsWriterPos";
+  final static String PAY_CODEC = "BlockPostingsWriterPay";
 
   // Increment version to change it:
   final static int VERSION_START = 0;
@@ -111,12 +111,12 @@ public final class BlockPackedPostingsWr
   final byte[] encoded;
 
   private final ForUtil forUtil;
-  private final BlockPackedSkipWriter skipWriter;
+  private final BlockSkipWriter skipWriter;
   
-  public BlockPackedPostingsWriter(SegmentWriteState state, float acceptableOverheadRatio) throws IOException {
+  public BlockPostingsWriter(SegmentWriteState state, float acceptableOverheadRatio) throws IOException {
     super();
 
-    docOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockPackedPostingsFormat.DOC_EXTENSION),
+    docOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockPostingsFormat.DOC_EXTENSION),
                                           state.context);
     IndexOutput posOut = null;
     IndexOutput payOut = null;
@@ -126,7 +126,7 @@ public final class BlockPackedPostingsWr
       forUtil = new ForUtil(acceptableOverheadRatio, docOut);
       if (state.fieldInfos.hasProx()) {
         posDeltaBuffer = new int[MAX_DATA_SIZE];
-        posOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockPackedPostingsFormat.POS_EXTENSION),
+        posOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockPostingsFormat.POS_EXTENSION),
                                               state.context);
         CodecUtil.writeHeader(posOut, POS_CODEC, VERSION_CURRENT);
 
@@ -147,7 +147,7 @@ public final class BlockPackedPostingsWr
         }
 
         if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) {
-          payOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockPackedPostingsFormat.PAY_EXTENSION),
+          payOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockPostingsFormat.PAY_EXTENSION),
                                                 state.context);
           CodecUtil.writeHeader(payOut, PAY_CODEC, VERSION_CURRENT);
         }
@@ -171,8 +171,8 @@ public final class BlockPackedPostingsWr
     freqBuffer = new int[MAX_DATA_SIZE];
 
     // nocommit should we try skipping every 2/4 blocks...?
-    skipWriter = new BlockPackedSkipWriter(maxSkipLevels,
-                                     BlockPackedPostingsFormat.BLOCK_SIZE, 
+    skipWriter = new BlockSkipWriter(maxSkipLevels,
+                                     BLOCK_SIZE, 
                                      state.segmentInfo.getDocCount(),
                                      docOut,
                                      posOut,
@@ -181,7 +181,7 @@ public final class BlockPackedPostingsWr
     encoded = new byte[MAX_ENCODED_SIZE];
   }
 
-  public BlockPackedPostingsWriter(SegmentWriteState state) throws IOException {
+  public BlockPostingsWriter(SegmentWriteState state) throws IOException {
     this(state, PackedInts.DEFAULT);
   }
 

Copied: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockSkipReader.java (from r1371500, lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipReader.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockSkipReader.java?p2=lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockSkipReader.java&p1=lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipReader.java&r1=1371500&r2=1371519&rev=1371519&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipReader.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockSkipReader.java Thu Aug  9 22:29:36 2012
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.blockpacked;
+package org.apache.lucene.codecs.block;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -50,8 +50,8 @@ import org.apache.lucene.store.IndexInpu
  * Therefore, we'll trim df before passing it to the interface. see trim(int)
  *
  */
-final class BlockPackedSkipReader extends MultiLevelSkipListReader {
-  private boolean DEBUG = BlockPackedPostingsReader.DEBUG;
+final class BlockSkipReader extends MultiLevelSkipListReader {
+  private boolean DEBUG = BlockPostingsReader.DEBUG;
   private final int blockSize;
 
   private long docPointer[];
@@ -68,7 +68,7 @@ final class BlockPackedSkipReader extend
   private long lastDocPointer;
   private int lastPosBufferUpto;
 
-  public BlockPackedSkipReader(IndexInput skipStream, int maxSkipLevels, int blockSize, boolean hasPos, boolean hasOffsets, boolean hasPayloads) {
+  public BlockSkipReader(IndexInput skipStream, int maxSkipLevels, int blockSize, boolean hasPos, boolean hasOffsets, boolean hasPayloads) {
     super(skipStream, maxSkipLevels, blockSize, 8);
     this.blockSize = blockSize;
     docPointer = new long[maxSkipLevels];

Copied: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockSkipWriter.java (from r1371500, lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipWriter.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockSkipWriter.java?p2=lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockSkipWriter.java&p1=lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipWriter.java&r1=1371500&r2=1371519&rev=1371519&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipWriter.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/BlockSkipWriter.java Thu Aug  9 22:29:36 2012
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.blockpacked;
+package org.apache.lucene.codecs.block;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -43,8 +43,8 @@ import org.apache.lucene.codecs.MultiLev
  * 4. start offset.
  *
  */
-final class BlockPackedSkipWriter extends MultiLevelSkipListWriter {
-  private boolean DEBUG = BlockPackedPostingsReader.DEBUG;
+final class BlockSkipWriter extends MultiLevelSkipListWriter {
+  private boolean DEBUG = BlockPostingsReader.DEBUG;
   
   private int[] lastSkipDoc;
   private long[] lastSkipDocPointer;
@@ -68,7 +68,7 @@ final class BlockPackedSkipWriter extend
   private boolean fieldHasOffsets;
   private boolean fieldHasPayloads;
 
-  public BlockPackedSkipWriter(int maxSkipLevels, int blockSize, int docCount, IndexOutput docOut, IndexOutput posOut, IndexOutput payOut) {
+  public BlockSkipWriter(int maxSkipLevels, int blockSize, int docCount, IndexOutput docOut, IndexOutput posOut, IndexOutput payOut) {
     super(blockSize, 8, maxSkipLevels, docCount);
     this.docOut = docOut;
     this.posOut = posOut;

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/ForUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/ForUtil.java?rev=1371519&r1=1371500&r2=1371519&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/ForUtil.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/ForUtil.java Thu Aug  9 22:29:36 2012
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.blockpacked;
+package org.apache.lucene.codecs.block;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -16,7 +16,7 @@ package org.apache.lucene.codecs.blockpa
  * limitations under the License.
  */
 
-import static org.apache.lucene.codecs.blockpacked.BlockPackedPostingsFormat.BLOCK_SIZE;
+import static org.apache.lucene.codecs.block.BlockPostingsFormat.BLOCK_SIZE;
 
 import java.io.IOException;
 import java.util.Arrays;

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat?rev=1371519&r1=1371518&r2=1371519&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat Thu Aug  9 22:29:36 2012
@@ -21,4 +21,3 @@ org.apache.lucene.codecs.bulkvint.BulkVI
 org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat
 org.apache.lucene.codecs.memory.DirectPostingsFormat
 org.apache.lucene.codecs.block.BlockPostingsFormat
-org.apache.lucene.codecs.blockpacked.BlockPackedPostingsFormat

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/block/TestForUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/block/TestForUtil.java?rev=1371519&r1=1371500&r2=1371519&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/block/TestForUtil.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/block/TestForUtil.java Thu Aug  9 22:29:36 2012
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.blockpacked;
+package org.apache.lucene.codecs.block;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,9 +17,9 @@ package org.apache.lucene.codecs.blockpa
  * limitations under the License.
  */
 
-import static org.apache.lucene.codecs.blockpacked.BlockPackedPostingsFormat.BLOCK_SIZE;
-import static org.apache.lucene.codecs.blockpacked.ForUtil.MAX_DATA_SIZE;
-import static org.apache.lucene.codecs.blockpacked.ForUtil.MAX_ENCODED_SIZE;
+import static org.apache.lucene.codecs.block.BlockPostingsFormat.BLOCK_SIZE;
+import static org.apache.lucene.codecs.block.ForUtil.MAX_DATA_SIZE;
+import static org.apache.lucene.codecs.block.ForUtil.MAX_ENCODED_SIZE;
 
 import java.io.IOException;
 import java.util.Arrays;