You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by bu...@apache.org on 2011/02/22 02:01:11 UTC

svn commit: r1073192 [10/32] - in /lucene/dev/branches/realtime_search: ./ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/lucene/contrib/ant/ dev-tools/idea/lucene/contrib/demo/ dev-tools/idea/lucene/contrib/highlighter/ dev-tools/idea/lucene/...

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java Tue Feb 22 01:00:39 2011
@@ -63,24 +63,23 @@ public class BlockTermsWriter extends Fi
   FieldInfo currentField;
   private final TermsIndexWriterBase termsIndexWriter;
   private final List<TermsWriter> fields = new ArrayList<TermsWriter>();
-  private final Comparator<BytesRef> termComp;
-  private final String segment;
+
+  //private final String segment;
 
   public BlockTermsWriter(
       TermsIndexWriterBase termsIndexWriter,
       SegmentWriteState state,
-      PostingsWriterBase postingsWriter,
-      Comparator<BytesRef> termComp) throws IOException
+      PostingsWriterBase postingsWriter)
+    throws IOException
   {
     final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_EXTENSION);
     this.termsIndexWriter = termsIndexWriter;
-    this.termComp = termComp;
     out = state.directory.createOutput(termsFileName);
     fieldInfos = state.fieldInfos;
     writeHeader(out);
     currentField = null;
     this.postingsWriter = postingsWriter;
-    segment = state.segmentName;
+    //segment = state.segmentName;
 
     //System.out.println("BTW.init seg=" + state.segmentName);
 
@@ -161,7 +160,6 @@ public class BlockTermsWriter extends Fi
     private long numTerms;
     private final TermsIndexWriterBase.FieldWriter fieldIndexWriter;
     long sumTotalTermFreq;
-    private final BytesRef lastTerm = new BytesRef();
 
     private TermEntry[] pendingTerms;
 
@@ -185,12 +183,12 @@ public class BlockTermsWriter extends Fi
     
     @Override
     public Comparator<BytesRef> getComparator() {
-      return termComp;
+      return BytesRef.getUTF8SortedAsUnicodeComparator();
     }
 
     @Override
     public PostingsConsumer startTerm(BytesRef text) throws IOException {
-      //System.out.println("BTW.startTerm seg=" + segment + " term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text);
+      //System.out.println("BTW.startTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment);
       postingsWriter.startTerm();
       return postingsWriter;
     }
@@ -201,7 +199,7 @@ public class BlockTermsWriter extends Fi
     public void finishTerm(BytesRef text, TermStats stats) throws IOException {
 
       assert stats.docFreq > 0;
-      //System.out.println("BTW.finishTerm seg=" + segment + " term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " df=" + stats.docFreq);
+      //System.out.println("BTW.finishTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment + " df=" + stats.docFreq);
 
       final boolean isIndexTerm = fieldIndexWriter.checkIndexTerm(text, stats);
 
@@ -213,6 +211,7 @@ public class BlockTermsWriter extends Fi
           flushBlock();
         }
         fieldIndexWriter.add(text, stats, out.getFilePointer());
+        //System.out.println("  index term!");
       }
 
       if (pendingTerms.length == pendingCount) {
@@ -265,7 +264,7 @@ public class BlockTermsWriter extends Fi
     private final RAMOutputStream bytesWriter = new RAMOutputStream();
 
     private void flushBlock() throws IOException {
-      //System.out.println("BTW.flushBlock pendingCount=" + pendingCount);
+      //System.out.println("BTW.flushBlock seg=" + segment + " pendingCount=" + pendingCount + " fp=" + out.getFilePointer());
 
       // First pass: compute common prefix for all terms
       // in the block, against term before first term in

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java Tue Feb 22 01:00:39 2011
@@ -23,11 +23,6 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
-import org.apache.lucene.index.codecs.pulsing.PulsingCodec;
-import org.apache.lucene.index.codecs.simpletext.SimpleTextCodec;
-import org.apache.lucene.index.codecs.standard.StandardCodec;
-
 /** Holds a set of codecs, keyed by name.  You subclass
  *  this, instantiate it, and register your codecs, then
  *  pass this instance to IndexReader/IndexWriter (via
@@ -96,7 +91,7 @@ public class CodecProvider {
     return infosReader;
   }
 
-  static private CodecProvider defaultCodecs = new DefaultCodecProvider();
+  static private CodecProvider defaultCodecs = new CoreCodecProvider();
 
   public static CodecProvider getDefault() {
     return defaultCodecs;
@@ -164,12 +159,3 @@ public class CodecProvider {
     defaultFieldCodec = codec;
   }
 }
-
-class DefaultCodecProvider extends CodecProvider {
-  DefaultCodecProvider() {
-    register(new StandardCodec());
-    register(new PreFlexCodec());
-    register(new PulsingCodec(1));
-    register(new SimpleTextCodec());
-  }
-}

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java Tue Feb 22 01:00:39 2011
@@ -44,7 +44,7 @@ public class FixedGapTermsIndexReader ex
   // number of places to multiply out the actual ord, and we
   // will overflow int during those multiplies.  So to avoid
   // having to upgrade each multiple to long in multiple
-  // places (error proned), we use long here:
+  // places (error prone), we use long here:
   private long totalIndexInterval;
 
   private int indexDivisor;

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java Tue Feb 22 01:00:39 2011
@@ -152,36 +152,17 @@ public abstract class FixedIntBlockIndex
     @Override
     public void read(final DataInput indexIn, final boolean absolute) throws IOException {
       if (absolute) {
-        fp = indexIn.readVLong();
         upto = indexIn.readVInt();
-      } else {
-        final long delta = indexIn.readVLong();
-        if (delta == 0) {
-          // same block
-          upto += indexIn.readVInt();
-        } else {
-          // new block
-          fp += delta;
-          upto = indexIn.readVInt();
-        }
-      }
-      assert upto < blockSize;
-    }
-
-    @Override
-    public void read(final IntIndexInput.Reader indexIn, final boolean absolute) throws IOException {
-      if (absolute) {
         fp = indexIn.readVLong();
-        upto = indexIn.next();
       } else {
-        final long delta = indexIn.readVLong();
-        if (delta == 0) {
+        final int uptoDelta = indexIn.readVInt();
+        if ((uptoDelta & 1) == 1) {
           // same block
-          upto += indexIn.next();
+          upto += uptoDelta >>> 1;
         } else {
           // new block
-          fp += delta;
-          upto = indexIn.next();
+          upto = uptoDelta >>> 1;
+          fp += indexIn.readVLong();
         }
       }
       assert upto < blockSize;

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java Tue Feb 22 01:00:39 2011
@@ -77,36 +77,17 @@ public abstract class FixedIntBlockIndex
     @Override
     public void write(IndexOutput indexOut, boolean absolute) throws IOException {
       if (absolute) {
-        indexOut.writeVLong(fp);
-        indexOut.writeVInt(upto);
-      } else if (fp == lastFP) {
-        // same block
-        indexOut.writeVLong(0);
-        assert upto >= lastUpto;
-        indexOut.writeVInt(upto - lastUpto);
-      } else {      
-        // new block
-        indexOut.writeVLong(fp - lastFP);
         indexOut.writeVInt(upto);
-      }
-      lastUpto = upto;
-      lastFP = fp;
-    }
-
-    @Override
-    public void write(IntIndexOutput indexOut, boolean absolute) throws IOException {
-      if (absolute) {
         indexOut.writeVLong(fp);
-        indexOut.write(upto);
       } else if (fp == lastFP) {
         // same block
-        indexOut.writeVLong(0);
         assert upto >= lastUpto;
-        indexOut.write(upto - lastUpto);
+        int uptoDelta = upto - lastUpto;
+        indexOut.writeVInt(uptoDelta << 1 | 1);
       } else {      
         // new block
+        indexOut.writeVInt(upto << 1);
         indexOut.writeVLong(fp - lastFP);
-        indexOut.write(upto);
       }
       lastUpto = upto;
       lastFP = fp;

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexInput.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexInput.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexInput.java Tue Feb 22 01:00:39 2011
@@ -171,17 +171,17 @@ public abstract class VariableIntBlockIn
     @Override
     public void read(final DataInput indexIn, final boolean absolute) throws IOException {
       if (absolute) {
+        upto = indexIn.readVInt();
         fp = indexIn.readVLong();
-        upto = indexIn.readByte()&0xFF;
       } else {
-        final long delta = indexIn.readVLong();
-        if (delta == 0) {
+        final int uptoDelta = indexIn.readVInt();
+        if ((uptoDelta & 1) == 1) {
           // same block
-          upto = indexIn.readByte()&0xFF;
+          upto += uptoDelta >>> 1;
         } else {
           // new block
-          fp += delta;
-          upto = indexIn.readByte()&0xFF;
+          upto = uptoDelta >>> 1;
+          fp += indexIn.readVLong();
         }
       }
       // TODO: we can't do this assert because non-causal
@@ -190,24 +190,6 @@ public abstract class VariableIntBlockIn
     }
 
     @Override
-    public void read(final IntIndexInput.Reader indexIn, final boolean absolute) throws IOException {
-      if (absolute) {
-        fp = indexIn.readVLong();
-        upto = indexIn.next()&0xFF;
-      } else {
-        final long delta = indexIn.readVLong();
-        if (delta == 0) {
-          // same block
-          upto = indexIn.next()&0xFF;
-        } else {
-          // new block
-          fp += delta;
-          upto = indexIn.next()&0xFF;
-        }
-      }
-    }
-
-    @Override
     public String toString() {
       return "VarIntBlock.Index fp=" + fp + " upto=" + upto + " maxBlock=" + maxBlockSize;
     }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexOutput.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexOutput.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexOutput.java Tue Feb 22 01:00:39 2011
@@ -42,16 +42,14 @@ public abstract class VariableIntBlockIn
 
   private int upto;
 
-  private static final int MAX_BLOCK_SIZE = 1 << 8;
+  // TODO what Var-Var codecs exist in practice... and what are there blocksizes like?
+  // if its less than 128 we should set that as max and use byte?
 
-  /** NOTE: maxBlockSize plus the max non-causal lookahead
-   *  of your codec must be less than 256.  EG Simple9
+  /** NOTE: maxBlockSize must be the maximum block size 
+   *  plus the max non-causal lookahead of your codec.  EG Simple9
    *  requires lookahead=1 because on seeing the Nth value
    *  it knows it must now encode the N-1 values before it. */
   protected VariableIntBlockIndexOutput(IndexOutput out, int maxBlockSize) throws IOException {
-    if (maxBlockSize > MAX_BLOCK_SIZE) {
-      throw new IllegalArgumentException("maxBlockSize must be <= " + MAX_BLOCK_SIZE + "; got " + maxBlockSize);
-    }
     this.out = out;
     out.writeInt(maxBlockSize);
   }
@@ -88,37 +86,17 @@ public abstract class VariableIntBlockIn
     public void write(IndexOutput indexOut, boolean absolute) throws IOException {
       assert upto >= 0;
       if (absolute) {
+        indexOut.writeVInt(upto);
         indexOut.writeVLong(fp);
-        indexOut.writeByte((byte) upto);
-      } else if (fp == lastFP) {
-        // same block
-        indexOut.writeVLong(0);
-        assert upto >= lastUpto;
-        indexOut.writeByte((byte) upto);
-      } else {      
-        // new block
-        indexOut.writeVLong(fp - lastFP);
-        indexOut.writeByte((byte) upto);
-      }
-      lastUpto = upto;
-      lastFP = fp;
-    }
-
-    @Override
-    public void write(IntIndexOutput indexOut, boolean absolute) throws IOException {
-      assert upto >= 0;
-      if (absolute) {
-        indexOut.writeVLong(fp);
-        indexOut.write(upto);
       } else if (fp == lastFP) {
         // same block
-        indexOut.writeVLong(0);
         assert upto >= lastUpto;
-        indexOut.write(upto);
+        int uptoDelta = upto - lastUpto;
+        indexOut.writeVInt(uptoDelta << 1 | 1);
       } else {      
         // new block
+        indexOut.writeVInt(upto << 1);
         indexOut.writeVLong(fp - lastFP);
-        indexOut.write(upto);
       }
       lastUpto = upto;
       lastFP = fp;

Added: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/package.html?rev=1073192&view=auto
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/package.html (added)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/intblock/package.html Tue Feb 22 01:00:39 2011
@@ -0,0 +1,25 @@
+<!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>
+Intblock: base support for fixed or variable length block integer encoders
+</body>
+</html>

Added: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/package.html?rev=1073192&view=auto
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/package.html (added)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/package.html Tue Feb 22 01:00:39 2011
@@ -0,0 +1,25 @@
+<!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>
+Codecs API: API for customization of the encoding and structure of the index.
+</body>
+</html>

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java Tue Feb 22 01:00:39 2011
@@ -19,14 +19,15 @@ package org.apache.lucene.index.codecs.p
 
 import java.io.IOException;
 import java.util.Collection;
-import java.util.Iterator;
-import java.util.TreeMap;
+import java.util.Comparator;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.Map;
-import java.util.Comparator;
+import java.util.TreeMap;
 
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.CompoundFileReader;
 import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.FieldsEnum;
@@ -35,7 +36,6 @@ import org.apache.lucene.index.SegmentIn
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.CompoundFileReader;
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
@@ -94,13 +94,11 @@ public class PreFlexFields extends Field
       // so that if an index update removes them we'll still have them
       freqStream = dir.openInput(info.name + ".frq", readBufferSize);
       boolean anyProx = false;
-      final int numFields = fieldInfos.size();
-      for(int i=0;i<numFields;i++) {
-        final FieldInfo fieldInfo = fieldInfos.fieldInfo(i);
-        if (fieldInfo.isIndexed) {
-          fields.put(fieldInfo.name, fieldInfo);
-          preTerms.put(fieldInfo.name, new PreTerms(fieldInfo));
-          if (!fieldInfo.omitTermFreqAndPositions) {
+      for (FieldInfo fi : fieldInfos) {
+        if (fi.isIndexed) {
+          fields.put(fi.name, fi);
+          preTerms.put(fi.name, new PreTerms(fi));
+          if (!fi.omitTermFreqAndPositions) {
             anyProx = true;
           }
         }
@@ -538,7 +536,7 @@ public class PreFlexFields extends Field
       // We can easily detect S in UTF8: if a byte has
       // prefix 11110 (0xf0), then that byte and the
       // following 3 bytes encode a single unicode codepoint
-      // in S.  Similary,we can detect E: if a byte has
+      // in S.  Similarly, we can detect E: if a byte has
       // prefix 1110111 (0xee), then that byte and the
       // following 2 bytes encode a single unicode codepoint
       // in E.
@@ -980,7 +978,7 @@ public class PreFlexFields extends Field
 
   private final class PreDocsEnum extends DocsEnum {
     final private SegmentTermDocs docs;
-
+    private int docID = -1;
     PreDocsEnum() throws IOException {
       docs = new SegmentTermDocs(freqStream, getTermsDict(), fieldInfos);
     }
@@ -998,18 +996,18 @@ public class PreFlexFields extends Field
     @Override
     public int nextDoc() throws IOException {
       if (docs.next()) {
-        return docs.doc();
+        return docID = docs.doc();
       } else {
-        return NO_MORE_DOCS;
+        return docID = NO_MORE_DOCS;
       }
     }
 
     @Override
     public int advance(int target) throws IOException {
       if (docs.skipTo(target)) {
-        return docs.doc();
+        return docID = docs.doc();
       } else {
-        return NO_MORE_DOCS;
+        return docID = NO_MORE_DOCS;
       }
     }
 
@@ -1020,7 +1018,7 @@ public class PreFlexFields extends Field
 
     @Override
     public int docID() {
-      return docs.doc();
+      return docID;
     }
 
     @Override
@@ -1036,7 +1034,7 @@ public class PreFlexFields extends Field
 
   private final class PreDocsAndPositionsEnum extends DocsAndPositionsEnum {
     final private SegmentTermPositions pos;
-
+    private int docID = -1;
     PreDocsAndPositionsEnum() throws IOException {
       pos = new SegmentTermPositions(freqStream, proxStream, getTermsDict(), fieldInfos);
     }
@@ -1054,18 +1052,18 @@ public class PreFlexFields extends Field
     @Override
     public int nextDoc() throws IOException {
       if (pos.next()) {
-        return pos.doc();
+        return docID = pos.doc();
       } else {
-        return NO_MORE_DOCS;
+        return docID = NO_MORE_DOCS;
       }
     }
 
     @Override
     public int advance(int target) throws IOException {
       if (pos.skipTo(target)) {
-        return pos.doc();
+        return docID = pos.doc();
       } else {
-        return NO_MORE_DOCS;
+        return docID = NO_MORE_DOCS;
       }
     }
 
@@ -1076,16 +1074,18 @@ public class PreFlexFields extends Field
 
     @Override
     public int docID() {
-      return pos.doc();
+      return docID;
     }
 
     @Override
     public int nextPosition() throws IOException {
+      assert docID != NO_MORE_DOCS;
       return pos.nextPosition();
     }
 
     @Override
     public boolean hasPayload() {
+      assert docID != NO_MORE_DOCS;
       return pos.isPayloadAvailable();
     }
 

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java Tue Feb 22 01:00:39 2011
@@ -209,7 +209,8 @@ public class SegmentTermDocs {
 
   /** Optimized implementation. */
   public boolean skipTo(int target) throws IOException {
-    if (df >= skipInterval) {                      // optimized case
+    // don't skip if the target is close (within skipInterval docs away)
+    if ((target - skipInterval) >= doc && df >= skipInterval) {                      // optimized case
       if (skipListReader == null)
         skipListReader = new DefaultSkipListReader((IndexInput) freqStream.clone(), maxSkipLevels, skipInterval); // lazily clone
 

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermEnum.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermEnum.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermEnum.java Tue Feb 22 01:00:39 2011
@@ -45,7 +45,7 @@ public final class SegmentTermEnum imple
   // whenever you add a new format, make it 1 smaller (negative version logic)!
   public static final int FORMAT_CURRENT = FORMAT_VERSION_UTF8_LENGTH_IN_BYTES;
   
-  // when removing support for old versions, levae the last supported version here
+  // when removing support for old versions, leave the last supported version here
   public static final int FORMAT_MINIMUM = FORMAT_VERSION_UTF8_LENGTH_IN_BYTES;
 
   private TermBuffer termBuffer = new TermBuffer();

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermPositions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermPositions.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermPositions.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermPositions.java Tue Feb 22 01:00:39 2011
@@ -58,6 +58,7 @@ extends SegmentTermDocs  {
     this.proxStreamOrig = proxStream;  // the proxStream will be cloned lazily when nextPosition() is called for the first time
   }
 
+  @Override
   final void seek(TermInfo ti, Term term) throws IOException {
     super.seek(ti, term);
     if (ti != null)
@@ -69,6 +70,7 @@ extends SegmentTermDocs  {
     needToLoadPayload = false;
   }
 
+  @Override
   public final void close() throws IOException {
     super.close();
     if (proxStream != null) proxStream.close();
@@ -100,11 +102,13 @@ extends SegmentTermDocs  {
     return delta;
   }
   
+  @Override
   protected final void skippingDoc() throws IOException {
     // we remember to skip a document lazily
     lazySkipProxCount += freq;
   }
 
+  @Override
   public final boolean next() throws IOException {
     // we remember to skip the remaining positions of the current
     // document lazily
@@ -118,12 +122,14 @@ extends SegmentTermDocs  {
     return false;
   }
 
+  @Override
   public final int read(final int[] docs, final int[] freqs) {
     throw new UnsupportedOperationException("TermPositions does not support processing multiple documents in one call. Use TermDocs instead.");
   }
 
 
   /** Called by super.skipTo(). */
+  @Override
   protected void skipProx(long proxPointer, int payloadLength) throws IOException {
     // we save the pointer, we might have to skip there lazily
     lazySkipPointer = proxPointer;

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java Tue Feb 22 01:00:39 2011
@@ -67,15 +67,18 @@ public final class TermInfosReader {
       this.term = t;
     }
 
+    @Override
     public boolean equals(Object other) {
       CloneableTerm t = (CloneableTerm) other;
       return this.term.equals(t.term);
     }
 
+    @Override
     public int hashCode() {
       return term.hashCode();
     }
 
+    @Override
     public Object clone() {
       return new CloneableTerm(term);
     }

Added: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/package.html?rev=1073192&view=auto
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/package.html (added)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/package.html Tue Feb 22 01:00:39 2011
@@ -0,0 +1,25 @@
+<!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>
+Preflex codec: supports Lucene 3.x indexes (readonly)
+</body>
+</html>

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java Tue Feb 22 01:00:39 2011
@@ -89,7 +89,7 @@ public class PulsingCodec extends Codec 
     // Terms dict
     success = false;
     try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, pulsingWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, pulsingWriter);
       success = true;
       return ret;
     } finally {
@@ -136,7 +136,6 @@ public class PulsingCodec extends Codec 
                                                 state.dir, state.fieldInfos, state.segmentInfo.name,
                                                 pulsingReader,
                                                 state.readBufferSize,
-                                                BytesRef.getUTF8SortedAsUnicodeComparator(),
                                                 StandardCodec.TERMS_CACHE_SIZE,
                                                 state.codecId);
       success = true;

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java Tue Feb 22 01:00:39 2011
@@ -144,7 +144,7 @@ public class PulsingPostingsReaderImpl e
     //System.out.println("  count=" + count + " threshold=" + maxPositions);
 
     if (count <= maxPositions) {
-      //System.out.println("  inlined");
+      //System.out.println("  inlined pos=" + termState.inlinedBytesReader.getPosition());
 
       // Inlined into terms dict -- just read the byte[] blob in,
       // but don't decode it now (we only decode when a DocsEnum
@@ -261,7 +261,7 @@ public class PulsingPostingsReaderImpl e
       while(true) {
         if (postings.eof()) {
           //System.out.println("PR   END");
-          return NO_MORE_DOCS;
+          return docID = NO_MORE_DOCS;
         }
 
         final int code = postings.readVInt();
@@ -319,7 +319,7 @@ public class PulsingPostingsReaderImpl e
         if (doc >= target)
           return doc;
       }
-      return NO_MORE_DOCS;
+      return docID = NO_MORE_DOCS;
     }
   }
 
@@ -368,7 +368,7 @@ public class PulsingPostingsReaderImpl e
 
         if (postings.eof()) {
           //System.out.println("PR   END");
-          return NO_MORE_DOCS;
+          return docID = NO_MORE_DOCS;
         }
 
         final int code = postings.readVInt();
@@ -406,7 +406,7 @@ public class PulsingPostingsReaderImpl e
           return doc;
         }
       }
-      return NO_MORE_DOCS;
+      return docID = NO_MORE_DOCS;
     }
 
     @Override

Added: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/pulsing/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/pulsing/package.html?rev=1073192&view=auto
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/pulsing/package.html (added)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/pulsing/package.html Tue Feb 22 01:00:39 2011
@@ -0,0 +1,25 @@
+<!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>
+Pulsing Codec: inlines low frequency terms' postings into terms dictionary.
+</body>
+</html>

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexInput.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexInput.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexInput.java Tue Feb 22 01:00:39 2011
@@ -41,13 +41,12 @@ public abstract class IntIndexInput impl
 
     public abstract void read(DataInput indexIn, boolean absolute) throws IOException;
 
-    public abstract void read(IntIndexInput.Reader indexIn, boolean absolute) throws IOException;
-
     /** Seeks primary stream to the last read offset */
     public abstract void seek(IntIndexInput.Reader stream) throws IOException;
 
     public abstract void set(Index other);
     
+    @Override
     public abstract Object clone();
   }
 
@@ -56,18 +55,6 @@ public abstract class IntIndexInput impl
     /** Reads next single int */
     public abstract int next() throws IOException;
 
-    /** Encodes as 1 or 2 ints, and can only use 61 of the 64
-     *  long bits. */
-    public long readVLong() throws IOException {
-      final int v = next();
-      if ((v & 1) == 0) {
-        return v >> 1;
-      } else {
-        final long v2 = next();
-        return (v2 << 30) | (v >> 1);
-      }
-    }
-
     /** Reads next chunk of ints */
     private IntsRef bulkResult;
 

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexOutput.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexOutput.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexOutput.java Tue Feb 22 01:00:39 2011
@@ -38,23 +38,6 @@ public abstract class IntIndexOutput imp
    * >= 0.  */
   public abstract void write(int v) throws IOException;
 
-  public static final long MAX_SINGLE_INT_VLONG = Integer.MAX_VALUE - (1<<30);
-  public static final long MAX_VLONG = Long.MAX_VALUE - (1L<<62) - (1L<<61);
-
-  /** Encodes as 1 or 2 ints, and can only use 61 of the 64
-   *  long bits. */
-  public void writeVLong(long v) throws IOException {
-    assert v >= 0: "v=" + v;
-    assert v < MAX_VLONG: "v=" + v;
-    // we cannot pass a negative int 
-    if (v <= MAX_SINGLE_INT_VLONG) {
-      write(((int) v)<<1);
-    } else {
-      write(((int) ((v & MAX_SINGLE_INT_VLONG))<<1) | 1);
-      write(((int) (v >> 30)));
-    }
-  }
-
   public abstract static class Index {
 
     /** Internally records the current location */
@@ -66,8 +49,6 @@ public abstract class IntIndexOutput imp
     /** Writes "location" of current output pointer of primary
      *  output to different output (out) */
     public abstract void write(IndexOutput indexOut, boolean absolute) throws IOException;
-
-    public abstract void write(IntIndexOutput indexOut, boolean absolute) throws IOException;
   }
 
   /** If you are indexing the primary output file, call

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java Tue Feb 22 01:00:39 2011
@@ -56,6 +56,7 @@ public class SepPostingsReaderImpl exten
 
   int skipInterval;
   int maxSkipLevels;
+  int skipMinimum;
 
   public SepPostingsReaderImpl(Directory dir, SegmentInfo segmentInfo, int readBufferSize, IntStreamFactory intFactory, String codecId) throws IOException {
 
@@ -102,6 +103,7 @@ public class SepPostingsReaderImpl exten
       SepPostingsWriterImpl.VERSION_START, SepPostingsWriterImpl.VERSION_START);
     skipInterval = termsIn.readInt();
     maxSkipLevels = termsIn.readInt();
+    skipMinimum = termsIn.readInt();
   }
 
   @Override
@@ -160,6 +162,7 @@ public class SepPostingsReaderImpl exten
       return other;
     }
 
+    @Override
     public void copyFrom(TermState _other) {
       super.copyFrom(_other);
       SepTermState other = (SepTermState) _other;
@@ -230,7 +233,7 @@ public class SepPostingsReaderImpl exten
         //System.out.println("  payloadFP=" + termState.payloadFP);
       }
     }
-    if (termState.docFreq >= skipInterval) {
+    if (termState.docFreq >= skipMinimum) {
       //System.out.println("   readSkip @ " + termState.bytesReader.pos);
       if (isFirstTerm) {
         termState.skipFP = termState.bytesReader.readVLong();
@@ -239,7 +242,7 @@ public class SepPostingsReaderImpl exten
       }
       //System.out.println("  skipFP=" + termState.skipFP);
     } else if (isFirstTerm) {
-      termState.skipFP = termState.bytesReader.readVLong();
+      termState.skipFP = 0;
     }
   }
 
@@ -343,7 +346,7 @@ public class SepPostingsReaderImpl exten
       }
 
       docFreq = termState.docFreq;
-      // NOTE: unused if docFreq < skipInterval:
+      // NOTE: unused if docFreq < skipMinimum:
       skipFP = termState.skipFP;
       count = 0;
       doc = 0;
@@ -419,13 +422,10 @@ public class SepPostingsReaderImpl exten
     @Override
     public int advance(int target) throws IOException {
 
-      // TODO: jump right to next() if target is < X away
-      // from where we are now?
-
-      if (docFreq >= skipInterval) {
+      if ((target - skipInterval) >= doc && docFreq >= skipMinimum) {
 
         // There are enough docs in the posting to have
-        // skip data
+        // skip data, and its not too close
 
         if (skipper == null) {
           // This DocsEnum has never done any skipping
@@ -598,13 +598,10 @@ public class SepPostingsReaderImpl exten
     public int advance(int target) throws IOException {
       //System.out.println("SepD&P advance target=" + target + " vs current=" + doc + " this=" + this);
 
-      // TODO: jump right to next() if target is < X away
-      // from where we are now?
-
-      if (docFreq >= skipInterval) {
+      if ((target - skipInterval) >= doc && docFreq >= skipMinimum) {
 
         // There are enough docs in the posting to have
-        // skip data
+        // skip data, and its not too close
 
         if (skipper == null) {
           //System.out.println("  create skipper");

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java Tue Feb 22 01:00:39 2011
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.Set;
 
 import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
@@ -63,8 +64,23 @@ public final class SepPostingsWriterImpl
   IndexOutput termsOut;
 
   final SepSkipListWriter skipListWriter;
-  final int skipInterval;
-  final int maxSkipLevels;
+  /** Expert: The fraction of TermDocs entries stored in skip tables,
+   * used to accelerate {@link DocsEnum#advance(int)}.  Larger values result in
+   * smaller indexes, greater acceleration, but fewer accelerable cases, while
+   * smaller values result in bigger indexes, less acceleration and more
+   * accelerable cases. More detailed experiments would be useful here. */
+  final int skipInterval = 16;
+  
+  /**
+   * Expert: minimum docFreq to write any skip data at all
+   */
+  final int skipMinimum = skipInterval;
+
+  /** Expert: The maximum number of skip levels. Smaller values result in 
+   * slightly smaller indexes, but slower skipping in big posting lists.
+   */
+  final int maxSkipLevels = 10;
+
   final int totalNumDocs;
 
   boolean storePayloads;
@@ -118,15 +134,11 @@ public final class SepPostingsWriterImpl
 
     totalNumDocs = state.numDocs;
 
-    // TODO: -- abstraction violation
-    skipListWriter = new SepSkipListWriter(state.skipInterval,
-                                           state.maxSkipLevels,
+    skipListWriter = new SepSkipListWriter(skipInterval,
+                                           maxSkipLevels,
                                            state.numDocs,
                                            freqOut, docOut,
                                            posOut, payloadOut);
-
-    skipInterval = state.skipInterval;
-    maxSkipLevels = state.maxSkipLevels;
   }
 
   @Override
@@ -136,6 +148,7 @@ public final class SepPostingsWriterImpl
     // TODO: -- just ask skipper to "start" here
     termsOut.writeInt(skipInterval);                // write skipInterval
     termsOut.writeInt(maxSkipLevels);               // write maxSkipLevels
+    termsOut.writeInt(skipMinimum);                 // write skipMinimum
   }
 
   @Override
@@ -264,7 +277,7 @@ public final class SepPostingsWriterImpl
       }
     }
 
-    if (df >= skipInterval) {
+    if (df >= skipMinimum) {
       //System.out.println("  skipFP=" + skipStart);
       final long skipFP = skipOut.getFilePointer();
       skipListWriter.writeSkip(skipOut);
@@ -276,12 +289,8 @@ public final class SepPostingsWriterImpl
       }
       lastSkipFP = skipFP;
     } else if (isFirstTerm) {
-      // TODO: this is somewhat wasteful; eg if no terms in
-      // this block will use skip data, we don't need to
-      // write this:
-      final long skipFP = skipOut.getFilePointer();
-      indexBytesWriter.writeVLong(skipFP);
-      lastSkipFP = skipFP;
+      // lazily write an absolute delta if a term in this block requires skip data.
+      lastSkipFP = 0;
     }
 
     lastDocID = 0;

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListReader.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListReader.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListReader.java Tue Feb 22 01:00:39 2011
@@ -197,7 +197,9 @@ class SepSkipListReader extends MultiLev
     docIndex[level].read(skipStream, false);
     if (!omitTF) {
       posIndex[level].read(skipStream, false);
-      payloadPointer[level] += skipStream.readVInt();
+      if (currentFieldStoresPayloads) {
+        payloadPointer[level] += skipStream.readVInt();
+      }
     }
     
     return delta;

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListWriter.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListWriter.java Tue Feb 22 01:00:39 2011
@@ -188,7 +188,9 @@ class SepSkipListWriter extends MultiLev
     if (!omitTF) {
       posIndex[level].mark();
       posIndex[level].write(skipBuffer, false);
-      skipBuffer.writeVInt((int) (curPayloadPointer - lastSkipPayloadPointer[level]));
+      if (curStorePayloads) {
+        skipBuffer.writeVInt((int) (curPayloadPointer - lastSkipPayloadPointer[level]));
+      }
     }
 
     lastSkipDoc[level] = curDoc;

Added: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/package.html?rev=1073192&view=auto
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/package.html (added)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/package.html Tue Feb 22 01:00:39 2011
@@ -0,0 +1,25 @@
+<!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>
+Sep: base support for separate files (doc,frq,pos,skp,pyl)
+</body>
+</html>

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java Tue Feb 22 01:00:39 2011
@@ -129,6 +129,7 @@ class SimpleTextFieldsReader extends Fie
       fstEnum = new BytesRefFSTEnum<PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>>(fst);
     }
 
+    @Override
     public SeekStatus seek(BytesRef text, boolean useCache /* ignored */) throws IOException {
 
       //System.out.println("seek to text=" + text.utf8ToString());

Added: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/simpletext/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/simpletext/package.html?rev=1073192&view=auto
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/simpletext/package.html (added)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/simpletext/package.html Tue Feb 22 01:00:39 2011
@@ -0,0 +1,25 @@
+<!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>
+Simpletext Codec: writes human readable postings.
+</body>
+</html>

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java Tue Feb 22 01:00:39 2011
@@ -23,7 +23,6 @@ import java.util.Set;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
@@ -66,7 +65,7 @@ public class StandardCodec extends Codec
 
     success = false;
     try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, docs, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, docs);
       success = true;
       return ret;
     } finally {
@@ -109,7 +108,6 @@ public class StandardCodec extends Codec
                                                 state.segmentInfo.name,
                                                 postings,
                                                 state.readBufferSize,
-                                                BytesRef.getUTF8SortedAsUnicodeComparator(),
                                                 TERMS_CACHE_SIZE,
                                                 state.codecId);
       success = true;

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java Tue Feb 22 01:00:39 2011
@@ -47,6 +47,7 @@ public class StandardPostingsReader exte
 
   int skipInterval;
   int maxSkipLevels;
+  int skipMinimum;
 
   //private String segment;
 
@@ -86,6 +87,7 @@ public class StandardPostingsReader exte
 
     skipInterval = termsIn.readInt();
     maxSkipLevels = termsIn.readInt();
+    skipMinimum = termsIn.readInt();
   }
 
   // Must keep final because we do non-standard clone
@@ -99,12 +101,14 @@ public class StandardPostingsReader exte
     ByteArrayDataInput bytesReader;
     byte[] bytes;
 
+    @Override
     public Object clone() {
       StandardTermState other = new StandardTermState();
       other.copyFrom(this);
       return other;
     }
 
+    @Override
     public void copyFrom(TermState _other) {
       super.copyFrom(_other);
       StandardTermState other = (StandardTermState) _other;
@@ -118,6 +122,7 @@ public class StandardPostingsReader exte
       // (rare!), they will be re-read from disk.
     }
 
+    @Override
     public String toString() {
       return super.toString() + " freqFP=" + freqOffset + " proxFP=" + proxOffset + " skipOffset=" + skipOffset;
     }
@@ -176,7 +181,7 @@ public class StandardPostingsReader exte
     //System.out.println("  freqFP=" + termState.freqOffset);
     assert termState.freqOffset < freqIn.length();
 
-    if (termState.docFreq >= skipInterval) {
+    if (termState.docFreq >= skipMinimum) {
       termState.skipOffset = termState.bytesReader.readVInt();
       //System.out.println("  skipOffset=" + termState.skipOffset + " vs freqIn.length=" + freqIn.length());
       assert termState.freqOffset + termState.skipOffset < freqIn.length();
@@ -375,13 +380,10 @@ public class StandardPostingsReader exte
     @Override
     public int advance(int target) throws IOException {
 
-      // TODO: jump right to next() if target is < X away
-      // from where we are now?
-
-      if (limit >= skipInterval) {
+      if ((target - skipInterval) >= doc && limit >= skipMinimum) {
 
         // There are enough docs in the posting to have
-        // skip data
+        // skip data, and it isn't too close.
 
         if (skipper == null) {
           // This is the first time this enum has ever been used for skipping -- do lazy init
@@ -528,13 +530,10 @@ public class StandardPostingsReader exte
 
       //System.out.println("StandardR.D&PE advance target=" + target);
 
-      // TODO: jump right to next() if target is < X away
-      // from where we are now?
-
-      if (limit >= skipInterval) {
+      if ((target - skipInterval) >= doc && limit >= skipMinimum) {
 
         // There are enough docs in the posting to have
-        // skip data
+        // skip data, and it isn't too close
 
         if (skipper == null) {
           // This is the first time this enum has ever been used for skipping -- do lazy init
@@ -575,6 +574,7 @@ public class StandardPostingsReader exte
       return doc;
     }
 
+    @Override
     public int nextPosition() throws IOException {
 
       if (lazyProxPointer != -1) {
@@ -603,10 +603,12 @@ public class StandardPostingsReader exte
 
     /** Returns the payload at this position, or null if no
      *  payload was indexed. */
+    @Override
     public BytesRef getPayload() throws IOException {
       throw new IOException("No payloads exist for this field!");
     }
 
+    @Override
     public boolean hasPayload() {
       return false;
     }
@@ -724,13 +726,11 @@ public class StandardPostingsReader exte
     public int advance(int target) throws IOException {
 
       //System.out.println("StandardR.D&PE advance seg=" + segment + " target=" + target + " this=" + this);
-      // TODO: jump right to next() if target is < X away
-      // from where we are now?
 
-      if (limit >= skipInterval) {
+      if ((target - skipInterval) >= doc && limit >= skipMinimum) {
 
         // There are enough docs in the posting to have
-        // skip data
+        // skip data, and it isn't too close
 
         if (skipper == null) {
           // This is the first time this enum has ever been used for skipping -- do lazy init
@@ -773,6 +773,7 @@ public class StandardPostingsReader exte
       return doc;
     }
 
+    @Override
     public int nextPosition() throws IOException {
 
       if (lazyProxPointer != -1) {
@@ -833,6 +834,7 @@ public class StandardPostingsReader exte
 
     /** Returns the payload at this position, or null if no
      *  payload was indexed. */
+    @Override
     public BytesRef getPayload() throws IOException {
       assert lazyProxPointer == -1;
       assert posPendingCount < freq;
@@ -850,6 +852,7 @@ public class StandardPostingsReader exte
       return payload;
     }
 
+    @Override
     public boolean hasPayload() {
       return payloadPending && payloadLength > 0;
     }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java Tue Feb 22 01:00:39 2011
@@ -23,6 +23,7 @@ package org.apache.lucene.index.codecs.s
 import java.io.IOException;
 
 import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
@@ -44,8 +45,22 @@ public final class StandardPostingsWrite
   final IndexOutput freqOut;
   final IndexOutput proxOut;
   final DefaultSkipListWriter skipListWriter;
-  final int skipInterval;
-  final int maxSkipLevels;
+  /** Expert: The fraction of TermDocs entries stored in skip tables,
+   * used to accelerate {@link DocsEnum#advance(int)}.  Larger values result in
+   * smaller indexes, greater acceleration, but fewer accelerable cases, while
+   * smaller values result in bigger indexes, less acceleration and more
+   * accelerable cases. More detailed experiments would be useful here. */
+  final int skipInterval = 16;
+  
+  /**
+   * Expert: minimum docFreq to write any skip data at all
+   */
+  final int skipMinimum = skipInterval;
+
+  /** Expert: The maximum number of skip levels. Smaller values result in 
+   * slightly smaller indexes, but slower skipping in big posting lists.
+   */
+  final int maxSkipLevels = 10;
   final int totalNumDocs;
   IndexOutput termsOut;
 
@@ -84,14 +99,11 @@ public final class StandardPostingsWrite
 
     totalNumDocs = state.numDocs;
 
-    skipListWriter = new DefaultSkipListWriter(state.skipInterval,
-                                               state.maxSkipLevels,
+    skipListWriter = new DefaultSkipListWriter(skipInterval,
+                                               maxSkipLevels,
                                                state.numDocs,
                                                freqOut,
                                                proxOut);
-     
-    skipInterval = state.skipInterval;
-    maxSkipLevels = state.maxSkipLevels;
   }
 
   @Override
@@ -100,6 +112,7 @@ public final class StandardPostingsWrite
     CodecUtil.writeHeader(termsOut, CODEC, VERSION_CURRENT);
     termsOut.writeInt(skipInterval);                // write skipInterval
     termsOut.writeInt(maxSkipLevels);               // write maxSkipLevels
+    termsOut.writeInt(skipMinimum);                 // write skipMinimum
   }
 
   @Override
@@ -218,7 +231,7 @@ public final class StandardPostingsWrite
     }
     lastFreqStart = freqStart;
 
-    if (df >= skipInterval) {
+    if (df >= skipMinimum) {
       bytesWriter.writeVInt((int) (skipListWriter.writeSkip(freqOut)-freqStart));
     }
 

Added: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/standard/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/standard/package.html?rev=1073192&view=auto
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/standard/package.html (added)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/standard/package.html Tue Feb 22 01:00:39 2011
@@ -0,0 +1,25 @@
+<!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>
+Standard Codec
+</body>
+</html>

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/messages/Message.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/messages/Message.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/messages/Message.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/messages/Message.java Tue Feb 22 01:00:39 2011
@@ -17,14 +17,13 @@ package org.apache.lucene.messages;
  * limitations under the License.
  */
 
-import java.io.Serializable;
 import java.util.Locale;
 
 /**
  * Message Interface for a lazy loading.
  * For Native Language Support (NLS), system of software internationalization.
  */
-public interface Message extends Serializable {
+public interface Message {
 
   public String getKey();
 

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/messages/MessageImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/messages/MessageImpl.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/messages/MessageImpl.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/messages/MessageImpl.java Tue Feb 22 01:00:39 2011
@@ -25,8 +25,6 @@ import java.util.Locale;
  */
 public class MessageImpl implements Message {
 
-  private static final long serialVersionUID = -3077643314630884523L;
-
   private String key;
 
   private Object[] arguments = new Object[0];