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/02/14 18:15:47 UTC

svn commit: r1070580 [2/2] - in /lucene/dev/branches/bulkpostings/lucene/src: java/org/apache/lucene/index/codecs/bulkvint/ java/org/apache/lucene/index/codecs/fixed/ java/org/apache/lucene/index/codecs/intblock/ java/org/apache/lucene/index/codecs/pfo...

Added: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/fixed/InterleavedIntBlockIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/fixed/InterleavedIntBlockIndexOutput.java?rev=1070580&view=auto
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/fixed/InterleavedIntBlockIndexOutput.java (added)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/fixed/InterleavedIntBlockIndexOutput.java Mon Feb 14 17:15:47 2011
@@ -0,0 +1,99 @@
+package org.apache.lucene.index.codecs.fixed;
+
+/**
+ * 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.codecs.intblock.FixedIntBlockIndexOutput;
+import org.apache.lucene.index.codecs.sep.IntIndexOutput;
+
+/**
+ * Interleaves docs and freqs into a single file, by buffer freqs
+ * until the docs block is flushed, and then flushing the freqs block.
+ */
+final class InterleavedIntBlockIndexOutput extends IntIndexOutput {
+  private final FixedIntBlockIndexOutput a;
+  private final FixedIntBlockIndexOutput b;
+  private final int buffer[];
+  private int bufferIdx;
+  private boolean isB;
+  
+  /**
+   * @param a Fixed int block codec for docs
+   * @param b Fixed int block codec for freqs
+   * <p>
+   * NOTE: the two codecs must use the same blocksize.
+   */
+  public InterleavedIntBlockIndexOutput(FixedIntBlockIndexOutput a, FixedIntBlockIndexOutput b) {
+    if (a.blockSize != b.blockSize) {
+      throw new IllegalArgumentException("interleaved blocks must have the same block size");
+    }
+    this.a = a;
+    this.b = b;
+    this.buffer = new int[a.blockSize];
+  }
+  
+  @Override
+  public void write(int v) throws IOException {
+    if (isB) {
+      buffer[bufferIdx++] = v;
+      if (bufferIdx == buffer.length) {
+        // we have written a full block of documents,
+        // so flush any pending freqs.
+        flushFreqs();
+        // we don't need to force a flush on the block output,
+        // as we know we are at blocksize here and it just flushed.
+      }
+    } else {
+      a.write(v);
+    }
+    isB = !isB;
+  }
+
+  private void flushFreqs() throws IOException {
+    for (int i = 0; i < bufferIdx; i++) {
+      b.write(buffer[i]);
+    }
+    bufferIdx = 0;
+  }
+
+  @Override
+  public Index index() throws IOException {
+    return a.index();
+  }
+
+  /**
+   * Force a flush of any pending docs/freqs.
+   * This is necessary when we cross field boundaries that have different
+   * omitTF settings.
+   */
+  public void flush() throws IOException {
+    a.flush();
+    flushFreqs();
+    b.flush();
+  }
+  
+  @Override
+  public void close() throws IOException {
+    try {
+      flush();
+    } finally {
+      b.close();
+    }
+  }
+}

Added: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/fixed/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/fixed/package.html?rev=1070580&view=auto
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/fixed/package.html (added)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/fixed/package.html Mon Feb 14 17:15:47 2011
@@ -0,0 +1,27 @@
+<!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>
+Fixed: specialized support for fixed intblock codecs.
+Interleaves doc and freq blocks into a single .doc file.
+For now, positions, payloads, and skip data are separate files (pos,skp,pyl)
+</body>
+</html>

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java?rev=1070580&r1=1070579&r2=1070580&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java Mon Feb 14 17:15:47 2011
@@ -39,7 +39,8 @@ import org.apache.lucene.store.IndexInpu
 public abstract class FixedIntBlockIndexInput extends IntIndexInput {
 
   private final IndexInput in;
-  protected final int blockSize;
+  /** @lucene.internal */
+  public final int blockSize;
   
   public FixedIntBlockIndexInput(final IndexInput in) throws IOException {
     this.in = in;
@@ -55,6 +56,14 @@ public abstract class FixedIntBlockIndex
     // TODO: can this be simplified?
     return new Reader(clone, buffer, this.getBlockReader(clone, buffer));
   }
+  
+  /** Return a reader piggybacking on a previous reader.
+   * They share the same underlying indexinput (e.g. interleaved docs/freqs)
+   */
+  public Reader reader(Reader parent) throws IOException {
+    final int[] buffer = new int[blockSize];
+    return new Reader(parent.in, buffer, this.getBlockReader(parent.in, buffer));
+  }
 
   @Override
   public void close() throws IOException {
@@ -69,11 +78,20 @@ public abstract class FixedIntBlockIndex
   protected abstract BlockReader getBlockReader(IndexInput in, int[] buffer) throws IOException;
 
   public interface BlockReader {
+    /** Tead a block of integers */
     public void readBlock() throws IOException;
+    /** 
+     * Skip over a block of integers. 
+     * Decoding of the integers is not actually needed.
+     * A trivial implementation can always be to call readBlock(),
+     * but its preferred to avoid decoding and minimize i/o.
+     */
+    public void skipBlock() throws IOException;
     // nocommit -- need seek here so mmapdir "knows"
   }
 
-  private static class Reader extends BulkPostingsEnum.BlockReader {
+  /** @lucene.internal */
+  public static class Reader extends BulkPostingsEnum.BlockReader {
     private final IndexInput in;
 
     protected final int[] pending;
@@ -93,6 +111,7 @@ public abstract class FixedIntBlockIndex
 
     void seek(final long fp, final int upto) throws IOException {
       offset = upto;
+      //System.out.println("parent fill: " + in.getFilePointer() + " last=" + lastBlockFP + " fp=" + fp);
       if (fp != lastBlockFP) {
         // Seek to new block; this may in fact be the next
         // block ie when caller is doing sequential scan (eg
@@ -104,6 +123,34 @@ public abstract class FixedIntBlockIndex
         // Seek within current block
         //System.out.println("  seek in-block fp=" + fp + " upto=" + offset);
       }
+      //System.out.println("fill complete: " + in.getFilePointer());
+    }
+    
+    /**
+     * Position both this reader and its child reader to an index.
+     * If the child's data is not actually needed (e.g. reading only docs but skipping over freqs),
+     * then the parameter <code>fill</code> is true.
+     */
+    public void seek(IntIndexInput.Index idx, Reader child, boolean fill) throws IOException {
+      //nocommit: could this be more ugly?
+      Index index = (Index) idx;
+      final long fp = index.fp;
+
+      // synchronize both the parent and child to the index offset, as they are in parallel.
+      child.offset = offset = index.upto;
+      
+      // nocommit: if the child previously skipBlock'ed, we fill both.. can we do better? 
+      if ((index.fp != lastBlockFP) || (fill && child.lastBlockFP == -1)) {
+        in.seek(fp);
+        fill();
+        if (fill) {
+          child.fill();
+        } else {
+          child.skipBlock(); // the child blocks are not actually needed.
+        }
+      } else {
+        // seek within block
+      }
     }
 
     @Override
@@ -132,6 +179,11 @@ public abstract class FixedIntBlockIndex
       blockReader.readBlock();
       return blockSize;
     }
+    
+    public void skipBlock() throws IOException {
+      lastBlockFP = -1; /* nocommit: clear lastblockFP */
+      blockReader.skipBlock();
+    }
   }
 
   private class Index extends IntIndexInput.Index {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java?rev=1070580&r1=1070579&r2=1070580&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java Mon Feb 14 17:15:47 2011
@@ -37,7 +37,8 @@ import org.apache.lucene.store.IndexOutp
 public abstract class FixedIntBlockIndexOutput extends IntIndexOutput {
 
   protected final IndexOutput out;
-  private final int blockSize;
+  /** @lucene.internal */
+  public final int blockSize;
   protected final int[] buffer;
   private int upto;
 
@@ -116,22 +117,27 @@ public abstract class FixedIntBlockIndex
     }
   }
 
+  /** Force a flush of any possible pending ints to the output. */
+  public void flush() throws IOException {
+    // NOTE: entries in the block after current upto are
+    // invalid
+    if (!abort) {
+      while(upto != 0) {
+        // nocommit -- risky since in theory a "smart" int
+        // encoder could do run-length-encoding and thus
+        // never flush on an infinite stream of 0s; maybe
+        // flush upto instead?  or random ints heh
+        // stuff 0s until final block is flushed
+        //System.out.println("upto=" + upto + " stuff 0; blockSize=" + blockSize);
+        write(0);
+      }
+    }
+  }
+
   @Override
   public void close() throws IOException {
     try {
-      // NOTE: entries in the block after current upto are
-      // invalid
-      if (!abort) {
-        while(upto != 0) {
-          // nocommit -- risky since in theory a "smart" int
-          // encoder could do run-length-encoding and thus
-          // never flush on an infinite stream of 0s; maybe
-          // flush upto instead?  or random ints heh
-          // stuff 0s until final block is flushed
-          //System.out.println("upto=" + upto + " stuff 0; blockSize=" + blockSize);
-          write(0);
-        }
-      }
+      flush();
       /*
       if (upto > 0) {
         while(upto < blockSize) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FORFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FORFactory.java?rev=1070580&r1=1070579&r2=1070580&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FORFactory.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FORFactory.java Mon Feb 14 17:15:47 2011
@@ -17,14 +17,15 @@ package org.apache.lucene.index.codecs.p
  * limitations under the License.
  */
 
-import org.apache.lucene.store.Directory;
-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.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.index.codecs.fixed.FixedIntStreamFactory;
+import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexInput;
+import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexOutput;
 
 import java.io.IOException;
 
-public class FORFactory extends IntStreamFactory {
+public class FORFactory extends FixedIntStreamFactory {
   private final int blockSize;
 
   /** blockSize is only used when creating the
@@ -33,11 +34,13 @@ public class FORFactory extends IntStrea
     this.blockSize = blockSize;
   }
 
-  public IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException {
-    return new FORIndexInput(dir, fileName, readBufferSize);
+  @Override
+  public FixedIntBlockIndexInput openInput(IndexInput in, String fileName, boolean isChild) throws IOException {
+    return new FORIndexInput(in);
   }
 
-  public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
-    return new FORIndexOutput(dir, fileName, blockSize);
+  @Override
+  public FixedIntBlockIndexOutput createOutput(IndexOutput out, String fileName, boolean isChild) throws IOException {
+    return new FORIndexOutput(out, blockSize);
   }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FORIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FORIndexInput.java?rev=1070580&r1=1070579&r2=1070580&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FORIndexInput.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FORIndexInput.java Mon Feb 14 17:15:47 2011
@@ -17,7 +17,6 @@ package org.apache.lucene.index.codecs.p
  * limitations under the License.
  */
 
-import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexInput;
 import org.apache.lucene.util.pfor.ForDecompress;
@@ -26,15 +25,17 @@ import java.io.IOException;
 
 public class FORIndexInput extends FixedIntBlockIndexInput {
 
-  public FORIndexInput(Directory dir, String fileName, int readBufferSize) throws IOException {
-    super(dir.openInput(fileName, readBufferSize));
+  public FORIndexInput(IndexInput in) throws IOException {
+    super(in);
   }
 
   private static class BlockReader implements FixedIntBlockIndexInput.BlockReader {
     private final ForDecompress decompressor;
-
+    private final IndexInput in; // nocommit: add skipBlock to the decompressor?
+    
     public BlockReader(IndexInput in, int[] buffer) {
       decompressor = new ForDecompress(in, buffer, 0, buffer.length);
+      this.in = in;
     }
 
     public void seek(long pos) throws IOException {
@@ -45,6 +46,14 @@ public class FORIndexInput extends Fixed
       decompressor.decompress();
       //System.out.println("  FOR.readBlock");
     }
+    
+    // nocommit: abstraction, and can this be simplified?!
+    public void skipBlock() throws IOException {
+      int header = in.readInt(); // should FOR/PFOR use vint header?
+      final int numFrameBits = ((header >>> 8) & 31) + 1;
+      final int numBytes = numFrameBits << 4;
+      in.seek(in.getFilePointer() + numBytes); // seek past block
+    }
   }
 
   protected FixedIntBlockIndexInput.BlockReader getBlockReader(IndexInput in, int[] buffer) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FORIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FORIndexOutput.java?rev=1070580&r1=1070579&r2=1070580&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FORIndexOutput.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FORIndexOutput.java Mon Feb 14 17:15:47 2011
@@ -21,7 +21,7 @@ package org.apache.lucene.index.codecs.p
  *  expected to give poor performance; it's really only for
  *  testing the pluggability.  One should typically use pfor instead. */
 
-import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.pfor.ForCompress;
 import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexOutput;
 
@@ -38,14 +38,9 @@ public class FORIndexOutput extends Fixe
   private final ForCompress compressor;
   private final byte[] output;
 
-  // nocommit
-  private final String fileName;
+  public FORIndexOutput(IndexOutput out, int blockSize) throws IOException {
+    super(out, blockSize);
 
-  public FORIndexOutput(Directory dir, String fileName, int blockSize) throws IOException {
-    super(dir.createOutput(fileName), blockSize);
-
-    // nocommit
-    this.fileName = fileName;
     compressor = new ForCompress();
     // nocommit -- can't hardwire 1024; it's a function of blockSize
     ByteBuffer byteBuffer = ByteBuffer.allocate(1024);

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FrameOfRefCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FrameOfRefCodec.java?rev=1070580&r1=1070579&r2=1070580&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FrameOfRefCodec.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FrameOfRefCodec.java Mon Feb 14 17:15:47 2011
@@ -28,8 +28,8 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.FieldsProducer;
-import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
-import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
+import org.apache.lucene.index.codecs.fixed.FixedPostingsReaderImpl;
+import org.apache.lucene.index.codecs.fixed.FixedPostingsWriterImpl;
 import org.apache.lucene.index.codecs.standard.StandardCodec;
 import org.apache.lucene.index.codecs.BlockTermsWriter;
 import org.apache.lucene.index.codecs.BlockTermsReader;
@@ -48,7 +48,7 @@ public class FrameOfRefCodec extends Cod
 
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new FORFactory(128));
+    PostingsWriterBase postingsWriter = new FixedPostingsWriterImpl(state, new FORFactory(128));
 
     boolean success = false;
     TermsIndexWriterBase indexWriter;
@@ -79,7 +79,7 @@ public class FrameOfRefCodec extends Cod
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir,
+    PostingsReaderBase postingsReader = new FixedPostingsReaderImpl(state.dir,
                                                                   state.segmentInfo,
                                                                   state.readBufferSize,
                                                                   new FORFactory(128),
@@ -126,14 +126,14 @@ public class FrameOfRefCodec extends Cod
 
   @Override
   public void files(Directory dir, SegmentInfo segmentInfo, String id, Set<String> files) {
-    SepPostingsReaderImpl.files(segmentInfo, id, files);
+    FixedPostingsReaderImpl.files(segmentInfo, id, files);
     BlockTermsReader.files(dir, segmentInfo, id, files);
     VariableGapTermsIndexReader.files(dir, segmentInfo, id, files);
   }
 
   @Override
   public void getExtensions(Set<String> extensions) {
-    SepPostingsWriterImpl.getExtensions(extensions);
+    FixedPostingsWriterImpl.getExtensions(extensions);
     BlockTermsReader.getExtensions(extensions);
     VariableGapTermsIndexReader.getIndexExtensions(extensions);
   }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PForDeltaFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PForDeltaFactory.java?rev=1070580&r1=1070579&r2=1070580&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PForDeltaFactory.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PForDeltaFactory.java Mon Feb 14 17:15:47 2011
@@ -17,14 +17,15 @@ package org.apache.lucene.index.codecs.p
  * limitations under the License.
  */
 
-import org.apache.lucene.store.Directory;
-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.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.index.codecs.fixed.FixedIntStreamFactory;
+import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexInput;
+import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexOutput;
 
 import java.io.IOException;
 
-public class PForDeltaFactory extends IntStreamFactory {
+public class PForDeltaFactory extends FixedIntStreamFactory {
   private final int blockSize;
 
   /** blockSize is only used when creating the
@@ -33,11 +34,13 @@ public class PForDeltaFactory extends In
     this.blockSize = blockSize;
   }
 
-  public IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException {
-    return new PForDeltaIndexInput(dir, fileName, readBufferSize);
+  @Override
+  public FixedIntBlockIndexInput openInput(IndexInput in, String fileName, boolean isChild) throws IOException {
+    return new PForDeltaIndexInput(in);
   }
 
-  public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
-    return new PForDeltaIndexOutput(dir, fileName, blockSize);
+  @Override
+  public FixedIntBlockIndexOutput createOutput(IndexOutput out, String fileName, boolean isChild) throws IOException {
+    return new PForDeltaIndexOutput(out, blockSize);
   }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PForDeltaIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PForDeltaIndexInput.java?rev=1070580&r1=1070579&r2=1070580&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PForDeltaIndexInput.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PForDeltaIndexInput.java Mon Feb 14 17:15:47 2011
@@ -17,7 +17,6 @@ package org.apache.lucene.index.codecs.p
  * limitations under the License.
  */
 
-import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexInput;
 import org.apache.lucene.util.pfor.PForDecompress;
@@ -26,15 +25,17 @@ import java.io.IOException;
 
 public class PForDeltaIndexInput extends FixedIntBlockIndexInput {
 
-  public PForDeltaIndexInput(Directory dir, String fileName, int readBufferSize) throws IOException {
-    super(dir.openInput(fileName, readBufferSize));
+  public PForDeltaIndexInput(IndexInput in) throws IOException {
+    super(in);
   }
 
   private static class BlockReader implements FixedIntBlockIndexInput.BlockReader {
     private final PForDecompress decompressor;
-
+    private final IndexInput in;
+    
     public BlockReader(IndexInput in, int[] buffer) {
       decompressor = new PForDecompress(in, buffer, 0, buffer.length);
+      this.in = in;
     }
 
     public void seek(long pos) throws IOException {
@@ -44,6 +45,11 @@ public class PForDeltaIndexInput extends
     public void readBlock() throws IOException {
       decompressor.decompress();
     }
+    
+    public void skipBlock() throws IOException {
+      int numBytes = in.readInt(); // nocommit: should PFOR use vint header?
+      in.seek(in.getFilePointer() + numBytes); // seek past block
+    }
   }
 
   protected BlockReader getBlockReader(IndexInput in, int[] buffer) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PForDeltaIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PForDeltaIndexOutput.java?rev=1070580&r1=1070579&r2=1070580&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PForDeltaIndexOutput.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PForDeltaIndexOutput.java Mon Feb 14 17:15:47 2011
@@ -17,7 +17,7 @@ package org.apache.lucene.index.codecs.p
  * limitations under the License.
  */
 
-import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexOutput;
 import org.apache.lucene.util.pfor.PForCompress;
 
@@ -32,8 +32,8 @@ public class PForDeltaIndexOutput extend
   private final PForCompress compressor;
   private final byte[] output;
 
-  public PForDeltaIndexOutput(Directory dir, String fileName, int blockSize) throws IOException {
-    super(dir.createOutput(fileName), blockSize);
+  public PForDeltaIndexOutput(IndexOutput out, int blockSize) throws IOException {
+    super(out, blockSize);
 
     compressor = new PForCompress();
     // nocommit -- can't hardwire 1024; it's a function of blockSize

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PatchedFrameOfRefCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PatchedFrameOfRefCodec.java?rev=1070580&r1=1070579&r2=1070580&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PatchedFrameOfRefCodec.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PatchedFrameOfRefCodec.java Mon Feb 14 17:15:47 2011
@@ -28,8 +28,8 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.FieldsProducer;
-import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
-import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
+import org.apache.lucene.index.codecs.fixed.FixedPostingsReaderImpl;
+import org.apache.lucene.index.codecs.fixed.FixedPostingsWriterImpl;
 import org.apache.lucene.index.codecs.standard.StandardCodec;
 import org.apache.lucene.index.codecs.BlockTermsWriter;
 import org.apache.lucene.index.codecs.BlockTermsReader;
@@ -48,7 +48,7 @@ public class PatchedFrameOfRefCodec exte
 
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new PForDeltaFactory(128));
+    PostingsWriterBase postingsWriter = new FixedPostingsWriterImpl(state, new PForDeltaFactory(128));
 
     boolean success = false;
     TermsIndexWriterBase indexWriter;
@@ -79,7 +79,7 @@ public class PatchedFrameOfRefCodec exte
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir,
+    PostingsReaderBase postingsReader = new FixedPostingsReaderImpl(state.dir,
                                                                   state.segmentInfo,
                                                                   state.readBufferSize,
                                                                   new PForDeltaFactory(128),
@@ -126,14 +126,14 @@ public class PatchedFrameOfRefCodec exte
 
   @Override
   public void files(Directory dir, SegmentInfo segmentInfo, String id, Set<String> files) {
-    SepPostingsReaderImpl.files(segmentInfo, id, files);
+    FixedPostingsReaderImpl.files(segmentInfo, id, files);
     BlockTermsReader.files(dir, segmentInfo, id, files);
     VariableGapTermsIndexReader.files(dir, segmentInfo, id, files);
   }
 
   @Override
   public void getExtensions(Set<String> extensions) {
-    SepPostingsWriterImpl.getExtensions(extensions);
+    FixedPostingsWriterImpl.getExtensions(extensions);
     BlockTermsReader.getExtensions(extensions);
     VariableGapTermsIndexReader.getIndexExtensions(extensions);
   }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta2/PForDeltaFixedIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta2/PForDeltaFixedIntBlockCodec.java?rev=1070580&r1=1070579&r2=1070580&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta2/PForDeltaFixedIntBlockCodec.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta2/PForDeltaFixedIntBlockCodec.java Mon Feb 14 17:15:47 2011
@@ -27,11 +27,9 @@ import org.apache.lucene.index.SegmentRe
 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.fixed.FixedIntStreamFactory;
+import org.apache.lucene.index.codecs.fixed.FixedPostingsReaderImpl;
+import org.apache.lucene.index.codecs.fixed.FixedPostingsWriterImpl;
 import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexInput;
 import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexOutput;
 import org.apache.lucene.index.codecs.PostingsWriterBase;
@@ -42,6 +40,7 @@ import org.apache.lucene.index.codecs.Te
 import org.apache.lucene.index.codecs.TermsIndexWriterBase;
 import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
 import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
+import org.apache.lucene.index.codecs.sep.IntStreamFactory;
 import org.apache.lucene.index.codecs.standard.StandardCodec;
 import org.apache.lucene.store.*;
 import org.apache.lucene.util.BytesRef;
@@ -109,11 +108,11 @@ public class PForDeltaFixedIntBlockCodec
       return new PForDeltaIntFactory();
     }
 
-    private class PForDeltaIntFactory extends IntStreamFactory {
+    private class PForDeltaIntFactory extends FixedIntStreamFactory {
 
       @Override
-      public IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException {
-        return new FixedIntBlockIndexInput(dir.openInput(fileName, readBufferSize)) {
+      public FixedIntBlockIndexInput openInput(IndexInput in, String filename, boolean isChild) throws IOException {
+        return new FixedIntBlockIndexInput(in) {
 
           @Override
           protected BlockReader getBlockReader(final IndexInput in, final int[] buffer) throws IOException {
@@ -134,14 +133,19 @@ public class PForDeltaFixedIntBlockCodec
                   decodeOneBlockWithPForDelta(compressedData, blockSize, buffer);
                 }
               }
+              
+              public void skipBlock() throws IOException {
+                final int compressedSizeInInt = in.readInt(); // nocommit: should vint header be used?
+                in.seek(in.getFilePointer() + (compressedSizeInInt * 4)); // seek past block
+              }
             };
           }
         };
       }
 
       @Override
-      public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException {
-        return new FixedIntBlockIndexOutput(dir.createOutput(fileName), blockSize) {
+      public FixedIntBlockIndexOutput createOutput(IndexOutput out, String fileName, boolean isChild) throws IOException {
+        return new FixedIntBlockIndexOutput(out, blockSize) {
           @Override
           protected void flushBlock() throws IOException {
             int compressedSizeInInts = 0; 
@@ -164,7 +168,7 @@ public class PForDeltaFixedIntBlockCodec
 
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new PForDeltaIntFactory());
+    PostingsWriterBase postingsWriter = new FixedPostingsWriterImpl(state, new PForDeltaIntFactory());
 
     boolean success = false;
     TermsIndexWriterBase indexWriter;
@@ -195,7 +199,7 @@ public class PForDeltaFixedIntBlockCodec
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir,
+    PostingsReaderBase postingsReader = new FixedPostingsReaderImpl(state.dir,
                                                                       state.segmentInfo,
                                                                       state.readBufferSize,
                                                                       new PForDeltaIntFactory(), state.codecId);
@@ -241,14 +245,14 @@ public class PForDeltaFixedIntBlockCodec
 
   @Override
   public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) {
-    SepPostingsReaderImpl.files(segmentInfo, codecId, files);
+    FixedPostingsReaderImpl.files(segmentInfo, codecId, files);
     BlockTermsReader.files(dir, segmentInfo, codecId, files);
     VariableGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
   }
 
   @Override
   public void getExtensions(Set<String> extensions) {
-    SepPostingsWriterImpl.getExtensions(extensions);
+    FixedPostingsWriterImpl.getExtensions(extensions);
     BlockTermsReader.getExtensions(extensions);
     VariableGapTermsIndexReader.getIndexExtensions(extensions);
   }

Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java?rev=1070580&r1=1070579&r2=1070580&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java Mon Feb 14 17:15:47 2011
@@ -90,6 +90,10 @@ public class MockFixedIntBlockCodec exte
                 buffer[i] = in.readVInt();
               }
             }
+            
+            public void skipBlock() throws IOException {
+              readBlock();
+            }
           };
         }
       };

Modified: lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1070580&r1=1070579&r2=1070580&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java Mon Feb 14 17:15:47 2011
@@ -2884,7 +2884,10 @@ public class TestIndexWriter extends Luc
     Document doc = new Document();
     Field f = newField(random, "field", "two", Field.Store.NO, Field.Index.ANALYZED);
     doc.add(f);
-    final int NUM_GAPS = 100;
+    // THIS TEST DEPENDS ON THE FACT YOU WILL JUMP! (e.g. if you set a massive skip interval,
+    // don't impl skipping at all (like the SimpleText hack), or do something else interesting,
+    // it fails!)
+    final int NUM_GAPS = 200;
     for(int i=0;i<NUM_GAPS;i++) {
       f.setValue("one");
       w.addDocument(doc);