You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by bu...@apache.org on 2007/05/31 09:48:54 UTC

svn commit: r543076 - in /lucene/java/trunk: ./ src/java/org/apache/lucene/index/ src/site/src/documentation/content/xdocs/ src/test/org/apache/lucene/index/

Author: buschmi
Date: Thu May 31 00:48:53 2007
New Revision: 543076

URL: http://svn.apache.org/viewvc?view=rev&rev=543076
Log:
LUCENE-866: Adds multi-level skip lists to the posting lists.

Added:
    lucene/java/trunk/src/java/org/apache/lucene/index/DefaultSkipListReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/DefaultSkipListWriter.java
    lucene/java/trunk/src/java/org/apache/lucene/index/MultiLevelSkipListReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/MultiLevelSkipListWriter.java
    lucene/java/trunk/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java
Modified:
    lucene/java/trunk/CHANGES.txt
    lucene/java/trunk/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/java/trunk/src/java/org/apache/lucene/index/SegmentTermDocs.java
    lucene/java/trunk/src/java/org/apache/lucene/index/SegmentTermEnum.java
    lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosReader.java
    lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosWriter.java
    lucene/java/trunk/src/site/src/documentation/content/xdocs/fileformats.xml

Modified: lucene/java/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/java/trunk/CHANGES.txt?view=diff&rev=543076&r1=543075&r2=543076
==============================================================================
--- lucene/java/trunk/CHANGES.txt (original)
+++ lucene/java/trunk/CHANGES.txt Thu May 31 00:48:53 2007
@@ -209,6 +209,12 @@
     BufferedIndexInput, but only when used during merging.  Together,
     these increases yield 10-18% overall performance gain vs the
     previous 1K defaults.  (Mike McCandless)
+
+ 7. LUCENE-866: Adds multi-level skip lists to the posting lists. This speeds 
+    up most queries that use skipTo(), especially on big indexes with large posting 
+    lists. For average AND queries the speedup is about 20%, for queries that 
+    contain very frequence and very unique terms the speedup can be over 80%.
+    (Michael Busch)
  
 Documentation
 

Added: lucene/java/trunk/src/java/org/apache/lucene/index/DefaultSkipListReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DefaultSkipListReader.java?view=auto&rev=543076
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DefaultSkipListReader.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DefaultSkipListReader.java Thu May 31 00:48:53 2007
@@ -0,0 +1,114 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.Arrays;
+
+import org.apache.lucene.store.IndexInput;
+
+/**
+ * Implements the skip list reader for the default posting list format
+ * that stores positions and payloads.
+ *
+ */
+public class DefaultSkipListReader extends MultiLevelSkipListReader {
+  private boolean currentFieldStoresPayloads;
+  private long freqPointer[];
+  private long proxPointer[];
+  private int payloadLength[];
+  
+  private long lastFreqPointer;
+  private long lastProxPointer;
+  private int lastPayloadLength;
+                           
+
+  DefaultSkipListReader(IndexInput skipStream, int maxSkipLevels, int skipInterval) {
+    super(skipStream, maxSkipLevels, skipInterval);
+    freqPointer = new long[maxSkipLevels];
+    proxPointer = new long[maxSkipLevels];
+    payloadLength = new int[maxSkipLevels];
+  }
+  
+  void init(long skipPointer, long freqBasePointer, long proxBasePointer, int df, boolean storesPayloads) {
+    super.init(skipPointer, df);
+    this.currentFieldStoresPayloads = storesPayloads;
+    lastFreqPointer = freqBasePointer;
+    lastProxPointer = proxBasePointer;
+
+    Arrays.fill(freqPointer, freqBasePointer);
+    Arrays.fill(proxPointer, proxBasePointer);
+    Arrays.fill(payloadLength, 0);
+  }
+
+  /** Returns the freq pointer of the doc to which the last call of 
+   * {@link MultiLevelSkipListReader#skipTo(int)} has skipped.  */
+  long getFreqPointer() {
+    return lastFreqPointer;
+  }
+
+  /** Returns the prox pointer of the doc to which the last call of 
+   * {@link MultiLevelSkipListReader#skipTo(int)} has skipped.  */
+  long getProxPointer() {
+    return lastProxPointer;
+  }
+  
+  /** Returns the payload length of the payload stored just before 
+   * the doc to which the last call of {@link MultiLevelSkipListReader#skipTo(int)} 
+   * has skipped.  */
+  int getPayloadLength() {
+    return lastPayloadLength;
+  }
+  
+  protected void seekChild(int level) throws IOException {
+    super.seekChild(level);
+    freqPointer[level] = lastFreqPointer;
+    proxPointer[level] = lastProxPointer;
+    payloadLength[level] = lastPayloadLength;
+  }
+  
+  protected void setLastSkipData(int level) {
+    super.setLastSkipData(level);
+    lastFreqPointer = freqPointer[level];
+    lastProxPointer = proxPointer[level];
+    lastPayloadLength = payloadLength[level];
+  }
+
+
+  protected int readSkipData(int level, IndexInput skipStream) throws IOException {
+    int delta;
+    if (currentFieldStoresPayloads) {
+      // the current field stores payloads.
+      // if the doc delta is odd then we have
+      // to read the current payload length
+      // because it differs from the length of the
+      // previous payload
+      delta = skipStream.readVInt();
+      if ((delta & 1) != 0) {
+        payloadLength[level] = skipStream.readVInt();
+      }
+      delta >>>= 1;
+    } else {
+      delta = skipStream.readVInt();
+    }
+    freqPointer[level] += skipStream.readVInt();
+    proxPointer[level] += skipStream.readVInt();
+    
+    return delta;
+  }
+}

Added: lucene/java/trunk/src/java/org/apache/lucene/index/DefaultSkipListWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DefaultSkipListWriter.java?view=auto&rev=543076
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DefaultSkipListWriter.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DefaultSkipListWriter.java Thu May 31 00:48:53 2007
@@ -0,0 +1,124 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.Arrays;
+
+import org.apache.lucene.store.IndexOutput;
+
+
+/**
+ * Implements the skip list writer for the default posting list format
+ * that stores positions and payloads.
+ *
+ */
+class DefaultSkipListWriter extends MultiLevelSkipListWriter {
+  private int[] lastSkipDoc;
+  private int[] lastSkipPayloadLength;
+  private long[] lastSkipFreqPointer;
+  private long[] lastSkipProxPointer;
+  
+  private IndexOutput freqOutput;
+  private IndexOutput proxOutput;
+
+  private int curDoc;
+  private boolean curStorePayloads;
+  private int curPayloadLength;
+  private long curFreqPointer;
+  private long curProxPointer;
+  
+  DefaultSkipListWriter(int skipInterval, int numberOfSkipLevels, int docCount, IndexOutput freqOutput, IndexOutput proxOutput) {
+    super(skipInterval, numberOfSkipLevels, docCount);
+    this.freqOutput = freqOutput;
+    this.proxOutput = proxOutput;
+    
+    lastSkipDoc = new int[numberOfSkipLevels];
+    lastSkipPayloadLength = new int[numberOfSkipLevels];
+    lastSkipFreqPointer = new long[numberOfSkipLevels];
+    lastSkipProxPointer = new long[numberOfSkipLevels];
+  }
+
+  /**
+   * Sets the values for the current skip data. 
+   */
+  void setSkipData(int doc, boolean storePayloads, int payloadLength) {
+    this.curDoc = doc;
+    this.curStorePayloads = storePayloads;
+    this.curPayloadLength = payloadLength;
+    this.curFreqPointer = freqOutput.getFilePointer();
+    this.curProxPointer = proxOutput.getFilePointer();
+  }
+  
+  protected void resetSkip() {
+    super.resetSkip();
+    Arrays.fill(lastSkipDoc, 0);
+    Arrays.fill(lastSkipPayloadLength, -1);  // we don't have to write the first length in the skip list
+    Arrays.fill(lastSkipFreqPointer, freqOutput.getFilePointer());
+    Arrays.fill(lastSkipProxPointer, proxOutput.getFilePointer());
+  }
+  
+  protected void writeSkipData(int level, IndexOutput skipBuffer) throws IOException {
+    // To efficiently store payloads in the posting lists we do not store the length of
+    // every payload. Instead we omit the length for a payload if the previous payload had
+    // the same length.
+    // However, in order to support skipping the payload length at every skip point must be known.
+    // So we use the same length encoding that we use for the posting lists for the skip data as well:
+    // Case 1: current field does not store payloads
+    //           SkipDatum                 --> DocSkip, FreqSkip, ProxSkip
+    //           DocSkip,FreqSkip,ProxSkip --> VInt
+    //           DocSkip records the document number before every SkipInterval th  document in TermFreqs. 
+    //           Document numbers are represented as differences from the previous value in the sequence.
+    // Case 2: current field stores payloads
+    //           SkipDatum                 --> DocSkip, PayloadLength?, FreqSkip,ProxSkip
+    //           DocSkip,FreqSkip,ProxSkip --> VInt
+    //           PayloadLength             --> VInt    
+    //         In this case DocSkip/2 is the difference between
+    //         the current and the previous value. If DocSkip
+    //         is odd, then a PayloadLength encoded as VInt follows,
+    //         if DocSkip is even, then it is assumed that the
+    //         current payload length equals the length at the previous
+    //         skip point
+    if (curStorePayloads) {
+      int delta = curDoc - lastSkipDoc[level];
+      if (curPayloadLength == lastSkipPayloadLength[level]) {
+        // the current payload length equals the length at the previous skip point,
+        // so we don't store the length again
+        skipBuffer.writeVInt(delta * 2);
+      } else {
+        // the payload length is different from the previous one. We shift the DocSkip, 
+        // set the lowest bit and store the current payload length as VInt.
+        skipBuffer.writeVInt(delta * 2 + 1);
+        skipBuffer.writeVInt(curPayloadLength);
+        lastSkipPayloadLength[level] = curPayloadLength;
+      }
+    } else {
+      // current field does not store payloads
+      skipBuffer.writeVInt(curDoc - lastSkipDoc[level]);
+    }
+    skipBuffer.writeVInt((int) (curFreqPointer - lastSkipFreqPointer[level]));
+    skipBuffer.writeVInt((int) (curProxPointer - lastSkipProxPointer[level]));
+
+    lastSkipDoc[level] = curDoc;
+    //System.out.println("write doc at level " + level + ": " + curDoc);
+    
+    lastSkipFreqPointer[level] = curFreqPointer;
+    lastSkipProxPointer[level] = curProxPointer;
+  }
+
+}

Added: lucene/java/trunk/src/java/org/apache/lucene/index/MultiLevelSkipListReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/MultiLevelSkipListReader.java?view=auto&rev=543076
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/MultiLevelSkipListReader.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/MultiLevelSkipListReader.java Thu May 31 00:48:53 2007
@@ -0,0 +1,271 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.Arrays;
+
+import org.apache.lucene.store.BufferedIndexInput;
+import org.apache.lucene.store.IndexInput;
+
+/**
+ * This abstract class reads skip lists with multiple levels.
+ * 
+ * See {@link MultiLevelSkipListWriter} for the information about the encoding 
+ * of the multi level skip lists. 
+ * 
+ * Subclasses must implement the abstract method {@link #readSkipData(int, IndexInput)}
+ * which defines the actual format of the skip data.
+ */
+abstract class MultiLevelSkipListReader {
+  // the maximum number of skip levels possible for this index
+  private int maxNumberOfSkipLevels; 
+  
+  // number of levels in this skip list
+  private int numberOfSkipLevels;
+  
+  // Expert: defines the number of top skip levels to buffer in memory.
+  // Reducing this number results in less memory usage, but possibly
+  // slower performance due to more random I/Os.
+  // Please notice that the space each level occupies is limited by
+  // the skipInterval. The top level can not contain more than
+  // skipLevel entries, the second top level can not contain more
+  // than skipLevel^2 entries and so forth.
+  private int numberOfLevelsToBuffer = 1;
+  
+  private int docCount;
+  private boolean haveSkipped;
+  
+  private IndexInput[] skipStream;    // skipStream for each level
+  private long skipPointer[];         // the start pointer of each skip level
+  private int skipInterval[];         // skipInterval of each level
+  private int[] numSkipped;           // number of docs skipped per level
+    
+  private int[] skipDoc;              // doc id of current skip entry per level 
+  private int lastDoc;                // doc id of last read skip entry with docId <= target
+  private long[] childPointer;        // child pointer of current skip entry per level
+  private long lastChildPointer;      // childPointer of last read skip entry with docId <= target
+  
+  private boolean inputIsBuffered;
+  
+  public MultiLevelSkipListReader(IndexInput skipStream, int maxSkipLevels, int skipInterval) {
+    this.skipStream = new IndexInput[maxSkipLevels];
+    this.skipPointer = new long[maxSkipLevels];
+    this.childPointer = new long[maxSkipLevels];
+    this.numSkipped = new int[maxSkipLevels];
+    this.maxNumberOfSkipLevels = maxSkipLevels;
+    this.skipInterval = new int[maxSkipLevels];
+    this.skipStream [0]= skipStream;
+    this.inputIsBuffered = (skipStream instanceof BufferedIndexInput);
+    this.skipInterval[0] = skipInterval;
+    for (int i = 1; i < maxSkipLevels; i++) {
+      // cache skip intervals
+      this.skipInterval[i] = this.skipInterval[i - 1] * skipInterval;
+    }
+    skipDoc = new int[maxSkipLevels];
+  }
+
+  
+  /** Returns the id of the doc to which the last call of {@link #skipTo(int)}
+   *  has skipped.  */
+  int getDoc() {
+    return lastDoc;
+  }
+  
+  
+  /** Skips entries to the first beyond the current whose document number is
+   *  greater than or equal to <i>target</i>. Returns the current doc count. 
+   */
+  int skipTo(int target) throws IOException {
+    if (!haveSkipped) {
+      // first time, load skip levels
+      loadSkipLevels();
+      haveSkipped = true;
+    }
+  
+    // walk up the levels until highest level is found that has a skip
+    // for this target
+    int level = 0;
+    while (level < numberOfSkipLevels - 1 && target > skipDoc[level + 1]) {
+      level++;
+    }    
+
+    while (level >= 0) {
+      if (target > skipDoc[level]) {
+        if (!loadNextSkip(level)) {
+          continue;
+        }
+      } else {
+        // no more skips on this level, go down one level
+        if (level > 0 && lastChildPointer > skipStream[level - 1].getFilePointer()) {
+          seekChild(level - 1);
+        } 
+        level--;
+      }
+    }
+    
+    return numSkipped[0] - skipInterval[0] - 1;
+  }
+  
+  private boolean loadNextSkip(int level) throws IOException {
+    // we have to skip, the target document is greater than the current
+    // skip list entry        
+    setLastSkipData(level);
+      
+    numSkipped[level] += skipInterval[level];
+      
+    if (numSkipped[level] > docCount) {
+      // this skip list is exhausted
+      skipDoc[level] = Integer.MAX_VALUE;
+      if (numberOfSkipLevels > level) numberOfSkipLevels = level; 
+      return false;
+    }
+
+    // read next skip entry
+    skipDoc[level] += readSkipData(level, skipStream[level]);
+    
+    if (level != 0) {
+      // read the child pointer if we are not on the leaf level
+      childPointer[level] = skipStream[level].readVLong() + skipPointer[level - 1];
+    }
+    
+    return true;
+
+  }
+  
+  /** Seeks the skip entry on the given level */
+  protected void seekChild(int level) throws IOException {
+    skipStream[level].seek(lastChildPointer);
+    numSkipped[level] = numSkipped[level + 1] - skipInterval[level + 1];
+    skipDoc[level] = lastDoc;
+    if (level > 0) {
+        childPointer[level] = skipStream[level].readVLong() + skipPointer[level - 1];
+    }
+  }
+  
+  void close() throws IOException {
+    for (int i = 1; i < skipStream.length; i++) {
+      if (skipStream[i] != null) {
+        skipStream[i].close();
+      }
+    }
+  }
+
+  /** initializes the reader */
+  void init(long skipPointer, int df) {
+    this.skipPointer[0] = skipPointer;
+    this.docCount = df;
+    Arrays.fill(skipDoc, 0);
+    Arrays.fill(numSkipped, 0);
+    haveSkipped = false;
+    for (int i = 1; i < numberOfSkipLevels; i++) {
+      skipStream[0] = null;
+    }
+  }
+  
+  /** Loads the skip levels  */
+  private void loadSkipLevels() throws IOException {
+    numberOfSkipLevels = docCount == 0 ? 0 : (int) Math.floor(Math.log(docCount) / Math.log(skipInterval[0]));
+    if (numberOfSkipLevels > maxNumberOfSkipLevels) {
+      numberOfSkipLevels = maxNumberOfSkipLevels;
+    }
+
+    skipStream[0].seek(skipPointer[0]);
+    
+    int toBuffer = numberOfLevelsToBuffer;
+    
+    for (int i = numberOfSkipLevels - 1; i > 0; i--) {
+      // the length of the current level
+      long length = skipStream[0].readVLong();
+      
+      // the start pointer of the current level
+      skipPointer[i] = skipStream[0].getFilePointer();
+      if (toBuffer > 0) {
+        // buffer this level
+        skipStream[i] = new SkipBuffer(skipStream[0], (int) length);
+        toBuffer--;
+      } else {
+        // clone this stream, it is already at the start of the current level
+        skipStream[i] = (IndexInput) skipStream[0].clone();
+        if (inputIsBuffered && length < BufferedIndexInput.BUFFER_SIZE) {
+          ((BufferedIndexInput) skipStream[i]).setBufferSize((int) length);
+        }
+        
+        // move base stream beyond the current level
+        skipStream[0].seek(skipStream[0].getFilePointer() + length);
+      }
+    }
+   
+    // use base stream for the lowest level
+    skipPointer[0] = skipStream[0].getFilePointer();
+  }
+  
+  /**
+   * Subclasses must implement the actual skip data encoding in this method.
+   *  
+   * @param level the level skip data shall be read from
+   * @param skipStream the skip stream to read from
+   */  
+  protected abstract int readSkipData(int level, IndexInput skipStream) throws IOException;
+  
+  /** Copies the values of the last read skip entry on this level */
+  protected void setLastSkipData(int level) {
+    lastDoc = skipDoc[level];
+    lastChildPointer = childPointer[level];
+  }
+
+  
+  /** used to buffer the top skip levels */
+  private final static class SkipBuffer extends IndexInput {
+    private byte[] data;
+    private long pointer;
+    private int pos;
+    
+    SkipBuffer(IndexInput input, int length) throws IOException {
+      data = new byte[length];
+      pointer = input.getFilePointer();
+      input.readBytes(data, 0, length);
+    }
+    
+    public void close() throws IOException {
+      data = null;
+    }
+
+    public long getFilePointer() {
+      return pointer + pos;
+    }
+
+    public long length() {
+      return data.length;
+    }
+
+    public byte readByte() throws IOException {
+      return data[pos++];
+    }
+
+    public void readBytes(byte[] b, int offset, int len) throws IOException {
+      System.arraycopy(data, pos, b, offset, len);
+      pos += len;
+    }
+
+    public void seek(long pos) throws IOException {
+      this.pos =  (int) (pos - pointer);
+    }
+    
+  }
+}

Added: lucene/java/trunk/src/java/org/apache/lucene/index/MultiLevelSkipListWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/MultiLevelSkipListWriter.java?view=auto&rev=543076
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/MultiLevelSkipListWriter.java (added)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/MultiLevelSkipListWriter.java Thu May 31 00:48:53 2007
@@ -0,0 +1,151 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.store.IndexOutput;
+import org.apache.lucene.store.RAMOutputStream;
+
+/**
+ * This abstract class writes skip lists with multiple levels.
+ * 
+ * Example for skipInterval = 3:
+ *                                                     c            (skip level 2)
+ *                 c                 c                 c            (skip level 1) 
+ *     x     x     x     x     x     x     x     x     x     x      (skip level 0)
+ * d d d d d d d d d d d d d d d d d d d d d d d d d d d d d d d d  (posting list)
+ *     3     6     9     12    15    18    21    24    27    30     (df)
+ * 
+ * d - document
+ * x - skip data
+ * c - skip data with child pointer
+ * 
+ * Skip level i contains every skipInterval-th entry from skip level i-1.
+ * Therefore the number of entries on level i is: floor(df / ((skipInterval ^ (i + 1))).
+ * 
+ * Each skip entry on a level i>0 contains a pointer to the corresponding skip entry in list i-1.
+ * This guarantess a logarithmic amount of skips to find the target document.
+ * 
+ * While this class takes care of writing the different skip levels,
+ * subclasses must define the actual format of the skip data.
+ * 
+ */
+abstract class MultiLevelSkipListWriter {
+  // number of levels in this skip list
+  private int numberOfSkipLevels;
+  
+  // the skip interval in the list with level = 0
+  private int skipInterval;
+  
+  // for every skip level a different buffer is used 
+  private RAMOutputStream[] skipBuffer;
+
+  protected MultiLevelSkipListWriter(int skipInterval, int maxSkipLevels, int df) {
+    this.skipInterval = skipInterval;
+    
+    // calculate the maximum number of skip levels for this document frequency
+    numberOfSkipLevels = df == 0 ? 0 : (int) Math.floor(Math.log(df) / Math.log(skipInterval));
+    
+    // make sure it does not exceed maxSkipLevels
+    if (numberOfSkipLevels > maxSkipLevels) {
+      numberOfSkipLevels = maxSkipLevels;
+    }
+  }
+  
+  protected void init() {
+    skipBuffer = new RAMOutputStream[numberOfSkipLevels];
+    for (int i = 0; i < numberOfSkipLevels; i++) {
+      skipBuffer[i] = new RAMOutputStream();
+    }
+  }
+
+  protected void resetSkip() {
+    // creates new buffers or empties the existing ones
+    if (skipBuffer == null) {
+      init();
+    } else {
+      for (int i = 0; i < skipBuffer.length; i++) {
+        skipBuffer[i].reset();
+      }
+    }      
+  }
+
+  /**
+   * Subclasses must implement the actual skip data encoding in this method.
+   *  
+   * @param level the level skip data shall be writting for
+   * @param skipBuffer the skip buffer to write to
+   */
+  protected abstract void writeSkipData(int level, IndexOutput skipBuffer) throws IOException;
+  
+  /**
+   * Writes the current skip data to the buffers. The current document frequency determines
+   * the max level is skip data is to be written to. 
+   * 
+   * @param df the current document frequency 
+   * @throws IOException
+   */
+  void bufferSkip(int df) throws IOException {
+    int numLevels;
+   
+    // determine max level
+    for (numLevels = 0; (df % skipInterval) == 0 && numLevels < numberOfSkipLevels; df /= skipInterval) {
+      numLevels++;
+    }
+    
+    long childPointer = 0;
+    
+    for (int level = 0; level < numLevels; level++) {
+      writeSkipData(level, skipBuffer[level]);
+      
+      long newChildPointer = skipBuffer[level].getFilePointer();
+      
+      if (level != 0) {
+        // store child pointers for all levels except the lowest
+        skipBuffer[level].writeVLong(childPointer);
+      }
+      
+      //remember the childPointer for the next level
+      childPointer = newChildPointer;
+    }
+  }
+
+  /**
+   * Writes the buffered skip lists to the given output.
+   * 
+   * @param output the IndexOutput the skip lists shall be written to 
+   * @return the pointer the skip list starts
+   */
+  long writeSkip(IndexOutput output) throws IOException {
+    long skipPointer = output.getFilePointer();
+    if (skipBuffer == null || skipBuffer.length == 0) return skipPointer;
+    
+    for (int level = numberOfSkipLevels - 1; level > 0; level--) {
+      long length = skipBuffer[level].getFilePointer();
+      if (length > 0) {
+        output.writeVLong(length);
+        skipBuffer[level].writeTo(output);
+      }
+    }
+    skipBuffer[0].writeTo(output);
+    
+    return skipPointer;
+  }
+
+}

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/SegmentMerger.java?view=diff&rev=543076&r1=543075&r2=543076
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/SegmentMerger.java Thu May 31 00:48:53 2007
@@ -26,7 +26,6 @@
 import org.apache.lucene.document.FieldSelectorResult;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
 
 /**
  * The SegmentMerger class combines two or more Segments, represented by an IndexReader ({@link #add},
@@ -50,6 +49,8 @@
 
   private Vector readers = new Vector();
   private FieldInfos fieldInfos;
+  
+  private int mergedDocs;
 
   /** This ctor used only by test code.
    * 
@@ -93,14 +94,14 @@
   final int merge() throws CorruptIndexException, IOException {
     int value;
     
-    value = mergeFields();
+    mergedDocs = mergeFields();
     mergeTerms();
     mergeNorms();
 
     if (fieldInfos.hasVectors())
       mergeVectors();
 
-    return value;
+    return mergedDocs;
   }
   
   /**
@@ -241,7 +242,9 @@
   private IndexOutput proxOutput = null;
   private TermInfosWriter termInfosWriter = null;
   private int skipInterval;
+  private int maxSkipLevels;
   private SegmentMergeQueue queue = null;
+  private DefaultSkipListWriter skipListWriter = null;
 
   private final void mergeTerms() throws CorruptIndexException, IOException {
     try {
@@ -251,6 +254,8 @@
               new TermInfosWriter(directory, segment, fieldInfos,
                                   termIndexInterval);
       skipInterval = termInfosWriter.skipInterval;
+      maxSkipLevels = termInfosWriter.maxSkipLevels;
+      skipListWriter = new DefaultSkipListWriter(skipInterval, maxSkipLevels, mergedDocs, freqOutput, proxOutput);
       queue = new SegmentMergeQueue(readers.size());
 
       mergeTermInfos();
@@ -319,7 +324,7 @@
 
     int df = appendPostings(smis, n);		  // append posting data
 
-    long skipPointer = writeSkip();
+    long skipPointer = skipListWriter.writeSkip(freqOutput);
 
     if (df > 0) {
       // add an entry to the dictionary with pointers to prox and freq files
@@ -344,7 +349,7 @@
           throws CorruptIndexException, IOException {
     int lastDoc = 0;
     int df = 0;					  // number of docs w/ term
-    resetSkip();
+    skipListWriter.resetSkip();
     boolean storePayloads = fieldInfos.fieldInfo(smis[0].term.field).storePayloads;
     int lastPayloadLength = -1;   // ensures that we write the first length
     for (int i = 0; i < n; i++) {
@@ -366,7 +371,8 @@
         df++;
 
         if ((df % skipInterval) == 0) {
-          bufferSkip(lastDoc, storePayloads, lastPayloadLength);
+          skipListWriter.setSkipData(lastDoc, storePayloads, lastPayloadLength);
+          skipListWriter.bufferSkip(df);
         }
 
         int docCode = (doc - lastDoc) << 1;	  // use low bit to flag freq=1
@@ -411,75 +417,6 @@
       }
     }
     return df;
-  }
-
-  private RAMOutputStream skipBuffer = new RAMOutputStream();
-  private int lastSkipDoc;
-  private int lastSkipPayloadLength;
-  private long lastSkipFreqPointer;
-  private long lastSkipProxPointer;
-
-  private void resetSkip() {
-    skipBuffer.reset();
-    lastSkipDoc = 0;
-    lastSkipPayloadLength = -1;  // we don't have to write the first length in the skip list
-    lastSkipFreqPointer = freqOutput.getFilePointer();
-    lastSkipProxPointer = proxOutput.getFilePointer();
-  }
-
-  private void bufferSkip(int doc, boolean storePayloads, int payloadLength) throws IOException {
-    long freqPointer = freqOutput.getFilePointer();
-    long proxPointer = proxOutput.getFilePointer();
-
-    // To efficiently store payloads in the posting lists we do not store the length of
-    // every payload. Instead we omit the length for a payload if the previous payload had
-    // the same length.
-    // However, in order to support skipping the payload length at every skip point must be known.
-    // So we use the same length encoding that we use for the posting lists for the skip data as well:
-    // Case 1: current field does not store payloads
-    //           SkipDatum                 --> DocSkip, FreqSkip, ProxSkip
-    //           DocSkip,FreqSkip,ProxSkip --> VInt
-    //           DocSkip records the document number before every SkipInterval th  document in TermFreqs. 
-    //           Document numbers are represented as differences from the previous value in the sequence.
-    // Case 2: current field stores payloads
-    //           SkipDatum                 --> DocSkip, PayloadLength?, FreqSkip,ProxSkip
-    //           DocSkip,FreqSkip,ProxSkip --> VInt
-    //           PayloadLength             --> VInt    
-    //         In this case DocSkip/2 is the difference between
-    //         the current and the previous value. If DocSkip
-    //         is odd, then a PayloadLength encoded as VInt follows,
-    //         if DocSkip is even, then it is assumed that the
-    //         current payload length equals the length at the previous
-    //         skip point
-    if (storePayloads) {
-      int delta = doc - lastSkipDoc;
-      if (payloadLength == lastSkipPayloadLength) {
-        // the current payload length equals the length at the previous skip point,
-        // so we don't store the length again
-        skipBuffer.writeVInt(delta * 2);
-      } else {
-        // the payload length is different from the previous one. We shift the DocSkip, 
-        // set the lowest bit and store the current payload length as VInt.
-        skipBuffer.writeVInt(delta * 2 + 1);
-        skipBuffer.writeVInt(payloadLength);
-        lastSkipPayloadLength = payloadLength;
-      }
-    } else {
-      // current field does not store payloads
-      skipBuffer.writeVInt(doc - lastSkipDoc);
-    }
-    skipBuffer.writeVInt((int) (freqPointer - lastSkipFreqPointer));
-    skipBuffer.writeVInt((int) (proxPointer - lastSkipProxPointer));
-
-    lastSkipDoc = doc;
-    lastSkipFreqPointer = freqPointer;
-    lastSkipProxPointer = proxPointer;
-  }
-
-  private long writeSkip() throws IOException {
-    long skipPointer = freqOutput.getFilePointer();
-    skipBuffer.writeTo(freqOutput);
-    return skipPointer;
   }
 
   private void mergeNorms() throws IOException {

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/SegmentTermDocs.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/SegmentTermDocs.java?view=diff&rev=543076&r1=543075&r2=543076
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/SegmentTermDocs.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/SegmentTermDocs.java Thu May 31 00:48:53 2007
@@ -31,16 +31,15 @@
   int freq;
 
   private int skipInterval;
-  private int numSkips;
-  private int skipCount;
-  private IndexInput skipStream;
-  private int skipDoc;
-  private long freqPointer;
-  private long proxPointer;
+  private int maxSkipLevels;
+  private DefaultSkipListReader skipListReader;
+  
+  private long freqBasePointer;
+  private long proxBasePointer;
+
   private long skipPointer;
   private boolean haveSkipped;
   
-  private int payloadLengthAtLastSkip;
   protected boolean currentFieldStoresPayloads;
 
   protected SegmentTermDocs(SegmentReader parent) {
@@ -48,6 +47,7 @@
     this.freqStream = (IndexInput) parent.freqStream.clone();
     this.deletedDocs = parent.deletedDocs;
     this.skipInterval = parent.tis.getSkipInterval();
+    this.maxSkipLevels = parent.tis.getMaxSkipLevels();
   }
 
   public void seek(Term term) throws IOException {
@@ -74,7 +74,6 @@
 
   void seek(TermInfo ti, Term term) throws IOException {
     count = 0;
-    payloadLengthAtLastSkip = 0;
     FieldInfo fi = parent.fieldInfos.fieldInfo(term.field);
     currentFieldStoresPayloads = (fi != null) ? fi.storePayloads : false;
     if (ti == null) {
@@ -82,21 +81,18 @@
     } else {
       df = ti.docFreq;
       doc = 0;
-      skipDoc = 0;
-      skipCount = 0;
-      numSkips = df / skipInterval;
-      freqPointer = ti.freqPointer;
-      proxPointer = ti.proxPointer;
-      skipPointer = freqPointer + ti.skipOffset;
-      freqStream.seek(freqPointer);
+      freqBasePointer = ti.freqPointer;
+      proxBasePointer = ti.proxPointer;
+      skipPointer = freqBasePointer + ti.skipOffset;
+      freqStream.seek(freqBasePointer);
       haveSkipped = false;
     }
   }
 
   public void close() throws IOException {
     freqStream.close();
-    if (skipStream != null)
-      skipStream.close();
+    if (skipListReader != null)
+      skipListReader.close();
   }
 
   public final int doc() { return doc; }
@@ -111,11 +107,11 @@
         return false;
 
       int docCode = freqStream.readVInt();
-      doc += docCode >>> 1;			  // shift off low bit
-      if ((docCode & 1) != 0)			  // if low bit is set
-        freq = 1;				  // freq is one
+      doc += docCode >>> 1;       // shift off low bit
+      if ((docCode & 1) != 0)       // if low bit is set
+        freq = 1;         // freq is one
       else
-        freq = freqStream.readVInt();		  // else read freq
+        freq = freqStream.readVInt();     // else read freq
 
       count++;
 
@@ -135,11 +131,11 @@
 
       // manually inlined call to next() for speed
       final int docCode = freqStream.readVInt();
-      doc += docCode >>> 1;			  // shift off low bit
-      if ((docCode & 1) != 0)			  // if low bit is set
-        freq = 1;				  // freq is one
+      doc += docCode >>> 1;       // shift off low bit
+      if ((docCode & 1) != 0)       // if low bit is set
+        freq = 1;         // freq is one
       else
-        freq = freqStream.readVInt();		  // else read freq
+        freq = freqStream.readVInt();     // else read freq
       count++;
 
       if (deletedDocs == null || !deletedDocs.get(doc)) {
@@ -157,64 +153,22 @@
   /** Optimized implementation. */
   public boolean skipTo(int target) throws IOException {
     if (df >= skipInterval) {                      // optimized case
+      if (skipListReader == null)
+        skipListReader = new DefaultSkipListReader((IndexInput) freqStream.clone(), maxSkipLevels, skipInterval); // lazily clone
 
-      if (skipStream == null)
-        skipStream = (IndexInput) freqStream.clone(); // lazily clone
-
-      if (!haveSkipped) {                          // lazily seek skip stream
-        skipStream.seek(skipPointer);
+      if (!haveSkipped) {                          // lazily initialize skip stream
+        skipListReader.init(skipPointer, freqBasePointer, proxBasePointer, df, currentFieldStoresPayloads);
         haveSkipped = true;
       }
 
-      // scan skip data
-      int lastSkipDoc = skipDoc;
-      int lastPayloadLength = 0;
-      long lastFreqPointer = freqStream.getFilePointer();
-      long lastProxPointer = -1;
-      int numSkipped = -1 - (count % skipInterval);
-
-      while (target > skipDoc) {
-        lastSkipDoc = skipDoc;
-        lastFreqPointer = freqPointer;
-        lastProxPointer = proxPointer;
-        lastPayloadLength = payloadLengthAtLastSkip;
-        
-        if (skipDoc != 0 && skipDoc >= doc)
-          numSkipped += skipInterval;
-        
-        if(skipCount >= numSkips)
-          break;
-
-        if (currentFieldStoresPayloads) {
-          // the current field stores payloads.
-          // if the doc delta is odd then we have
-          // to read the current payload length
-          // because it differs from the length of the
-          // previous payload
-          int delta = skipStream.readVInt();
-          if ((delta & 1) != 0) {
-            payloadLengthAtLastSkip = skipStream.readVInt();
-          }
-          delta >>>= 1;
-          skipDoc += delta;
-        } else {
-          skipDoc += skipStream.readVInt();
-        }
-        freqPointer += skipStream.readVInt();
-        proxPointer += skipStream.readVInt();
-
-        skipCount++;
-      }
-      
-      // if we found something to skip, then skip it
-      if (lastFreqPointer > freqStream.getFilePointer()) {
-        freqStream.seek(lastFreqPointer);
-        skipProx(lastProxPointer, lastPayloadLength);
-
-        doc = lastSkipDoc;
-        count += numSkipped;
-      }
-
+      int newCount = skipListReader.skipTo(target); 
+      if (newCount > count) {
+        freqStream.seek(skipListReader.getFreqPointer());
+        skipProx(skipListReader.getProxPointer(), skipListReader.getPayloadLength());
+
+        doc = skipListReader.getDoc();
+        count = newCount;
+      }      
     }
 
     // done skipping, now just scan
@@ -224,5 +178,4 @@
     } while (target > doc);
     return true;
   }
-
 }

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/SegmentTermEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/SegmentTermEnum.java?view=diff&rev=543076&r1=543075&r2=543076
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/SegmentTermEnum.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/SegmentTermEnum.java Thu May 31 00:48:53 2007
@@ -37,6 +37,7 @@
   long indexPointer = 0;
   int indexInterval;
   int skipInterval;
+  int maxSkipLevels;
   private int formatM1SkipInterval;
 
   SegmentTermEnum(IndexInput i, FieldInfos fis, boolean isi)
@@ -44,7 +45,8 @@
     input = i;
     fieldInfos = fis;
     isIndex = isi;
-
+    maxSkipLevels = 1; // use single-level skip lists for formats > -3 
+    
     int firstInt = input.readInt();
     if (firstInt >= 0) {
       // original-format file, without explicit format version number
@@ -54,7 +56,6 @@
       // back-compatible settings
       indexInterval = 128;
       skipInterval = Integer.MAX_VALUE; // switch off skipTo optimization
-
     } else {
       // we have a format version number
       format = firstInt;
@@ -73,10 +74,13 @@
         // switch off skipTo optimization for file format prior to 1.4rc2 in order to avoid a bug in 
         // skipTo implementation of these versions
         skipInterval = Integer.MAX_VALUE;
-      }
-      else{
+      } else {
         indexInterval = input.readInt();
         skipInterval = input.readInt();
+        if (format == -3) {
+          // this new format introduces multi-level skipping
+          maxSkipLevels = input.readInt();
+        }
       }
     }
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosReader.java?view=diff&rev=543076&r1=543075&r2=543076
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosReader.java Thu May 31 00:48:53 2007
@@ -64,7 +64,11 @@
   public int getSkipInterval() {
     return origEnum.skipInterval;
   }
-
+  
+  public int getMaxSkipLevels() {
+    return origEnum.maxSkipLevels;
+  }
+  
   final void close() throws IOException {
     if (origEnum != null)
       origEnum.close();

Modified: lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosWriter.java?view=diff&rev=543076&r1=543075&r2=543076
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosWriter.java Thu May 31 00:48:53 2007
@@ -28,7 +28,7 @@
 
 final class TermInfosWriter {
   /** The file format version, a negative number. */
-  public static final int FORMAT = -2;
+  public static final int FORMAT = -3;
 
   private FieldInfos fieldInfos;
   private IndexOutput output;
@@ -56,6 +56,11 @@
    * smaller values result in bigger indexes, less acceleration and more
    * accelerable cases. More detailed experiments would be useful here. */
   int skipInterval = 16;
+  
+  /** Expert: The maximum number of skip levels. Smaller values result in 
+   * slightly smaller indexes, but slower skipping in big posting lists.
+   */
+  int maxSkipLevels = 10;
 
   private long lastIndexPointer = 0;
   private boolean isIndex = false;
@@ -85,6 +90,7 @@
     output.writeLong(0);                          // leave space for size
     output.writeInt(indexInterval);             // write indexInterval
     output.writeInt(skipInterval);              // write skipInterval
+    output.writeInt(maxSkipLevels);              // write maxSkipLevels
   }
 
   /** Adds a new <Term, TermInfo> pair to the set.

Modified: lucene/java/trunk/src/site/src/documentation/content/xdocs/fileformats.xml
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/site/src/documentation/content/xdocs/fileformats.xml?view=diff&rev=543076&r1=543075&r2=543076
==============================================================================
--- lucene/java/trunk/src/site/src/documentation/content/xdocs/fileformats.xml (original)
+++ lucene/java/trunk/src/site/src/documentation/content/xdocs/fileformats.xml Thu May 31 00:48:53 2007
@@ -1139,7 +1139,7 @@
 
                         <p>
                             TermInfoFile (.tis)--&gt;
-                            TIVersion, TermCount, IndexInterval, SkipInterval, TermInfos
+                            TIVersion, TermCount, IndexInterval, SkipInterval, MaxSkipLevels, TermInfos
                         </p>
                         <p>TIVersion --&gt;
                             UInt32
@@ -1153,6 +1153,9 @@
                         <p>SkipInterval --&gt;
                             UInt32
                         </p>
+                        <p>MaxSkipLevels --&gt;
+                            UInt32
+                        </p>
                         <p>TermInfos --&gt;
                             &lt;TermInfo&gt;
                             <sup>TermCount</sup>
@@ -1209,7 +1212,8 @@
                             particular, it is the number of bytes
                             after TermFreqs that the SkipData starts.
                             In other words, it is the length of the
-                            TermFreq data.
+                            TermFreq data. SkipDelta is only stored 
+                            if DocFreq is not smaller than SkipInterval.
                         </p>
                     </li>
                     <li>
@@ -1233,7 +1237,7 @@
 
                         <p>
                             TermInfoIndex (.tii)--&gt;
-                            TIVersion, IndexTermCount, IndexInterval, SkipInterval, TermIndices
+                            TIVersion, IndexTermCount, IndexInterval, SkipInterval, MaxSkipLevels, TermIndices
                         </p>
                         <p>TIVersion --&gt;
                             UInt32
@@ -1261,8 +1265,11 @@
                         </p>
                         <p>SkipInterval is the fraction of TermDocs stored in skip tables. It is used to accelerate TermDocs.skipTo(int).
                             Larger values result in smaller indexes, greater acceleration, but fewer accelerable cases, while
-                            smaller values result in bigger indexes, less acceleration and more
+                            smaller values result in bigger indexes, less acceleration (in case of a small value for MaxSkipLevels) and more
                             accelerable cases.</p>
+                        <p>MaxSkipLevels is the max. number of skip levels stored for each term in the .frq file. A low value results in 
+                           smaller indexes but less acceleration, a larger value results in slighly larger indexes but greater acceleration.
+                           See format of .frq file for more information about skip levels.</p>
                     </li>
                 </ol>
             </section>
@@ -1286,15 +1293,23 @@
                     DocDelta, Freq?
                 </p>
                 <p>SkipData --&gt;
+                    &lt;&lt;SkipLevelLength, SkipLevel&gt;
+                    <sup>NumSkipLevels-1</sup>, SkipLevel&gt;
+                    &lt;SkipDatum&gt;
+                </p>
+                <p>SkipLevel --&gt;
                     &lt;SkipDatum&gt;
-                    <sup>DocFreq/SkipInterval</sup>
+                    <sup>DocFreq/(SkipInterval^(Level + 1))</sup>
                 </p>
                 <p>SkipDatum --&gt;
-                    DocSkip,PayloadLength?,FreqSkip,ProxSkip
+                    DocSkip,PayloadLength?,FreqSkip,ProxSkip,SkipChildLevelPointer?
                 </p>
                 <p>DocDelta,Freq,DocSkip,PayloadLength,FreqSkip,ProxSkip --&gt;
                     VInt
                 </p>
+                <p>SkipChildLevelPointer --&gt;
+                    VLong
+                </p>
                 <p>TermFreqs
                     are ordered by term (the term is implicit, from the .tis file).
                 </p>
@@ -1361,6 +1376,18 @@
                     bytes after that that the 32
                     <sup>nd</sup>
                     starts.
+                </p>
+                <p>Lucene 2.2 introduces the notion of skip levels. Each term can have multiple skip levels.
+                   The amount of skip levels for a term is NumSkipLevels = Min(MaxSkipLevels, floor(log(DocFreq/log(SkipInterval)))).
+                   The number of SkipData entries for a skip level is DocFreq/(SkipInterval^(Level + 1)), whereas the lowest skip
+                   level is Level=0. <br></br>
+                   Example: SkipInterval = 4, MaxSkipLevels = 2, DocFreq = 35. Then skip level 0 has 8 SkipData entries,
+                   containing the 3<sup>rd</sup>, 7<sup>th</sup>, 11<sup>th</sup>, 15<sup>th</sup>, 19<sup>th</sup>, 23<sup>rd</sup>,
+                   27<sup>th</sup>, and 31<sup>st</sup> document numbers in TermFreqs. Skip level 1 has 2 SkipData entries, containing the
+                   15<sup>th</sup> and 31<sup>st</sup> document numbers in TermFreqs. <br></br>
+                   The SkipData entries on all upper levels &gt; 0 contain a SkipChildLevelPointer referencing the corresponding SkipData
+                   entry in level-1. In the example has entry 15 on level 1 a pointer to entry 15 on level 0 and entry 31 on level 1 a pointer
+                   to entry 31 on level 0.                   
                 </p>
 
             </section>

Added: lucene/java/trunk/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java?view=auto&rev=543076
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java (added)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java Thu May 31 00:48:53 2007
@@ -0,0 +1,154 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.io.Reader;
+
+import junit.framework.TestCase;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.LowerCaseTokenizer;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Index;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.RAMDirectory;
+
+/**
+ * This testcase tests whether multi-level skipping is being used
+ * to reduce I/O while skipping through posting lists.
+ * 
+ * Skipping in general is already covered by several other
+ * testcases.
+ * 
+ */
+public class TestMultiLevelSkipList extends TestCase {
+  public void testSimpleSkip() throws IOException {
+    RAMDirectory dir = new RAMDirectory();
+    IndexWriter writer = new IndexWriter(dir, new PayloadAnalyzer(), true);
+    Term term = new Term("test", "a");
+    for (int i = 0; i < 5000; i++) {
+      Document d1 = new Document();
+      d1.add(new Field(term.field(), term.text(), Store.NO, Index.TOKENIZED));
+      writer.addDocument(d1);
+    }
+    writer.flush();
+    writer.optimize();
+    writer.close();
+
+    IndexReader reader = IndexReader.open(dir);
+    SegmentTermPositions tp = (SegmentTermPositions) reader.termPositions(term);
+    tp.freqStream = new CountingStream(tp.freqStream);
+    
+    tp.next();
+
+    checkSkipTo(tp, 14, 185); // no skips
+    checkSkipTo(tp, 17, 190); // one skip on level 0
+    checkSkipTo(tp, 287, 200); // one skip on level 1, two on level 0
+    
+    // this test would fail if we had only one skip level,
+    // because than more bytes would be read from the freqStream
+    checkSkipTo(tp, 4800, 250);// one skip on level 2
+  }
+
+  public void checkSkipTo(TermPositions tp, int target, int maxCounter) throws IOException {
+    tp.skipTo(target);
+    if (maxCounter < counter) {
+      fail("Too many bytes read: " + counter);
+    }
+
+    assertEquals("Wrong document " + tp.doc() + " after skipTo target " + target, target, tp.doc());
+    assertEquals("Frequency is not 1: " + tp.freq(), 1,tp.freq());
+    tp.nextPosition();
+    byte[] b = new byte[1];
+    tp.getPayload(b, 0);
+    assertEquals("Wrong payload for the target " + target + ": " + b[0], (byte) target, b[0]);
+  }
+
+  private static class PayloadAnalyzer extends Analyzer {
+    public TokenStream tokenStream(String fieldName, Reader reader) {
+      return new PayloadFilter(new LowerCaseTokenizer(reader));
+    }
+
+  }
+
+  private static class PayloadFilter extends TokenFilter {
+    static int count = 0;
+    
+    protected PayloadFilter(TokenStream input) {
+      super(input);
+    }
+
+    public Token next() throws IOException {
+      Token t = input.next();
+      if (t != null) {
+        t.setPayload(new Payload(new byte[] { (byte) count++ }));
+      }
+      return t;
+    }
+
+  }
+
+  private int counter = 0;
+
+  // Simply extends IndexInput in a way that we are able to count the number
+  // of bytes read
+  class CountingStream extends IndexInput {
+    private IndexInput input;
+
+    CountingStream(IndexInput input) {
+      this.input = input;
+    }
+
+    public byte readByte() throws IOException {
+      TestMultiLevelSkipList.this.counter++;
+      return this.input.readByte();
+    }
+
+    public void readBytes(byte[] b, int offset, int len) throws IOException {
+      TestMultiLevelSkipList.this.counter += len;
+      this.input.readBytes(b, offset, len);
+    }
+
+    public void close() throws IOException {
+      this.input.close();
+    }
+
+    public long getFilePointer() {
+      return this.input.getFilePointer();
+    }
+
+    public void seek(long pos) throws IOException {
+      this.input.seek(pos);
+    }
+
+    public long length() {
+      return this.input.length();
+    }
+
+    public Object clone() {
+      return new CountingStream((IndexInput) this.input.clone());
+    }
+
+  }
+}



Re: svn commit: r543076 - in /lucene/java/trunk: ./ src/java/org/apache/lucene/index/ src/site/src/documentation/content/xdocs/ src/test/org/apache/lucene/index/

Posted by Erik Hatcher <er...@ehatchersolutions.com>.
On May 31, 2007, at 3:48 AM, buschmi@apache.org wrote:
> + 7. LUCENE-866: Adds multi-level skip lists to the posting lists.  
> This speeds
> +    up most queries that use skipTo(), especially on big indexes  
> with large posting
> +    lists. For average AND queries the speedup is about 20%, for  
> queries that
> +    contain very frequence and very unique terms the speedup can  
> be over 80%.
> +    (Michael Busch)

Minor typo.... frequence => frequent.

	Erik



---------------------------------------------------------------------
To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-dev-help@lucene.apache.org