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/01/21 20:38:14 UTC

svn commit: r1061979 [4/7] - in /lucene/dev/branches/realtime_search: ./ lucene/ lucene/contrib/ lucene/contrib/ant/ lucene/contrib/db/ lucene/contrib/db/bdb-je/ lucene/contrib/db/bdb/ lucene/contrib/demo/ lucene/contrib/demo/src/java/org/apache/lucene...

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java Fri Jan 21 19:38:06 2011
@@ -78,7 +78,7 @@ final class SloppyPhraseScorer extends P
 
             int matchLength = end - start;
             if (matchLength <= slop)
-                freq += getSimilarity().sloppyFreq(matchLength); // score match
+                freq += similarity.sloppyFreq(matchLength); // score match
 
             if (pp.position > end)
                 end = pp.position;

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermQuery.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermQuery.java Fri Jan 21 19:38:06 2011
@@ -54,7 +54,7 @@ public class TermQuery extends Query {
       throws IOException {
       assert termStates != null : "PerReaderTermState must not be null";
       this.termStates = termStates;
-      this.similarity = getSimilarity(searcher);
+      this.similarity = searcher.getSimilarity();
       if (docFreq != -1) {
         idfExp = similarity.idfExplain(term, searcher, docFreq);
       } else {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermScorer.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermScorer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermScorer.java Fri Jan 21 19:38:06 2011
@@ -38,7 +38,8 @@ final class TermScorer extends Scorer {
   private int[] docs;
   private int[] freqs;
   private final DocsEnum.BulkReadResult bulkResult;
-
+  private final Similarity similarity;
+  
   /**
    * Construct a <code>TermScorer</code>.
    * 
@@ -53,15 +54,15 @@ final class TermScorer extends Scorer {
    *          The field norms of the document fields for the <code>Term</code>.
    */
   TermScorer(Weight weight, DocsEnum td, Similarity similarity, byte[] norms) {
-    super(similarity, weight);
-    
+    super(weight);
+    this.similarity = similarity;
     this.docsEnum = td;
     this.norms = norms;
     this.weightValue = weight.getValue();
     bulkResult = td.getBulkResult();
 
     for (int i = 0; i < SCORE_CACHE_SIZE; i++)
-      scoreCache[i] = getSimilarity().tf(i) * weightValue;
+      scoreCache[i] = similarity.tf(i) * weightValue;
   }
 
   @Override
@@ -136,9 +137,9 @@ final class TermScorer extends Scorer {
     float raw =                                   // compute tf(f)*weight
       freq < SCORE_CACHE_SIZE                        // check cache
       ? scoreCache[freq]                             // cache hit
-      : getSimilarity().tf(freq)*weightValue;        // cache miss
+      : similarity.tf(freq)*weightValue;        // cache miss
 
-    return norms == null ? raw : raw * getSimilarity().decodeNormValue(norms[doc]); // normalize for field
+    return norms == null ? raw : raw * similarity.decodeNormValue(norms[doc]); // normalize for field
   }
 
   /**

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Weight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Weight.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Weight.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Weight.java Fri Jan 21 19:38:06 2011
@@ -34,7 +34,7 @@ import org.apache.lucene.index.IndexRead
  * {@link IndexReader} dependent state should reside in the {@link Scorer}.
  * <p>
  * Since {@link Weight} creates {@link Scorer} instances for a given
- * {@link AtomicReaderContext} ({@link #scorer(AtomicReaderContext, boolean, boolean)})
+ * {@link AtomicReaderContext} ({@link #scorer(IndexReader.AtomicReaderContext, ScorerContext)})
  * callers must maintain the relationship between the searcher's top-level
  * {@link ReaderContext} and the context used to create a {@link Scorer}. 
  * <p>
@@ -49,7 +49,7 @@ import org.apache.lucene.index.IndexRead
  * <li>The query normalization factor is passed to {@link #normalize(float)}. At
  * this point the weighting is complete.
  * <li>A <code>Scorer</code> is constructed by
- * {@link #scorer(AtomicReaderContext, ScorerContext)}.
+ * {@link #scorer(IndexReader.AtomicReaderContext, ScorerContext)}.
  * </ol>
  * 
  * @since 2.9
@@ -89,7 +89,7 @@ public abstract class Weight implements 
    * 
    * @param context
    *          the {@link AtomicReaderContext} for which to return the {@link Scorer}.
-   * @param scorercontext the {@link ScorerContext} holding the scores context variables
+   * @param scorerContext the {@link ScorerContext} holding the scores context variables
    * @return a {@link Scorer} which scores documents in/out-of order.
    * @throws IOException
    */
@@ -102,7 +102,7 @@ public abstract class Weight implements 
    * Returns true iff this implementation scores docs only out of order. This
    * method is used in conjunction with {@link Collector}'s
    * {@link Collector#acceptsDocsOutOfOrder() acceptsDocsOutOfOrder} and
-   * {@link #scorer(AtomicReaderContext, boolean, boolean)} to
+   * {@link #scorer(IndexReader.AtomicReaderContext, ScorerContext)} to
    * create a matching {@link Scorer} instance for a given {@link Collector}, or
    * vice versa.
    * <p>

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/cache/DocTermsIndexCreator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/cache/DocTermsIndexCreator.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/cache/DocTermsIndexCreator.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/cache/DocTermsIndexCreator.java Fri Jan 21 19:38:06 2011
@@ -341,9 +341,9 @@ public class DocTermsIndexCreator extend
       }
 
       @Override
-      public SeekStatus seek(BytesRef term, TermState state) throws IOException {
+      public void seek(BytesRef term, TermState state) throws IOException {
         assert state != null && state instanceof OrdTermState;
-        return this.seek(((OrdTermState)state).ord);
+        this.seek(((OrdTermState)state).ord);
       }
 
       @Override

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/CustomScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/CustomScoreQuery.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/CustomScoreQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/CustomScoreQuery.java Fri Jan 21 19:38:06 2011
@@ -30,7 +30,6 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Similarity;
 import org.apache.lucene.util.ToStringUtils;
 
 /**
@@ -183,13 +182,11 @@ public class CustomScoreQuery extends Qu
   //=========================== W E I G H T ============================
   
   private class CustomWeight extends Weight {
-    Similarity similarity;
     Weight subQueryWeight;
     Weight[] valSrcWeights;
     boolean qStrict;
 
     public CustomWeight(IndexSearcher searcher) throws IOException {
-      this.similarity = getSimilarity(searcher);
       this.subQueryWeight = subQuery.weight(searcher);
       this.valSrcWeights = new Weight[valSrcQueries.length];
       for(int i = 0; i < valSrcQueries.length; i++) {
@@ -254,7 +251,7 @@ public class CustomScoreQuery extends Qu
       for(int i = 0; i < valSrcScorers.length; i++) {
          valSrcScorers[i] = valSrcWeights[i].scorer(context, scorerContext.scoreDocsInOrder(true));
       }
-      return new CustomScorer(similarity, context.reader, this, subQueryScorer, valSrcScorers);
+      return new CustomScorer(context.reader, this, subQueryScorer, valSrcScorers);
     }
 
     @Override
@@ -303,9 +300,9 @@ public class CustomScoreQuery extends Qu
     private float vScores[]; // reused in score() to avoid allocating this array for each doc 
 
     // constructor
-    private CustomScorer(Similarity similarity, IndexReader reader, CustomWeight w,
+    private CustomScorer(IndexReader reader, CustomWeight w,
         Scorer subQueryScorer, Scorer[] valSrcScorers) throws IOException {
-      super(similarity,w);
+      super(w);
       this.qWeight = w.getValue();
       this.subQueryScorer = subQueryScorer;
       this.valSrcScorers = valSrcScorers;

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSource.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSource.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSource.java Fri Jan 21 19:38:06 2011
@@ -19,7 +19,6 @@ package org.apache.lucene.search.functio
 
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader.CompositeReaderContext;
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.search.function.DocValues;
 
@@ -52,7 +51,7 @@ public abstract class ValueSource implem
    * Return the DocValues used by the function query.
    * @deprecated (4.0) This method is temporary, to ease the migration to segment-based
    * searching. Please change your code to not pass {@link CompositeReaderContext} to these
-   * APIs. Use {@link #getValues(AtomicReaderContext)} instead
+   * APIs. Use {@link #getValues(IndexReader.AtomicReaderContext)} instead
    */
   @Deprecated
   public DocValues getValues(ReaderContext context) throws IOException {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSourceQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSourceQuery.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSourceQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSourceQuery.java Fri Jan 21 19:38:06 2011
@@ -64,12 +64,10 @@ public class ValueSourceQuery extends Qu
   }
 
   class ValueSourceWeight extends Weight {
-    Similarity similarity;
     float queryNorm;
     float queryWeight;
 
     public ValueSourceWeight(IndexSearcher searcher) {
-      this.similarity = getSimilarity(searcher);
     }
 
     /*(non-Javadoc) @see org.apache.lucene.search.Weight#getQuery() */
@@ -100,7 +98,7 @@ public class ValueSourceQuery extends Qu
 
     @Override
     public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
-      return new ValueSourceScorer(similarity, context, this);
+      return new ValueSourceScorer(context, this);
     }
 
     /*(non-Javadoc) @see org.apache.lucene.search.Weight#explain(org.apache.lucene.index.IndexReader, int) */
@@ -133,8 +131,8 @@ public class ValueSourceQuery extends Qu
     private int doc = -1;
 
     // constructor
-    private ValueSourceScorer(Similarity similarity, AtomicReaderContext context, ValueSourceWeight w) throws IOException {
-      super(similarity,w);
+    private ValueSourceScorer(AtomicReaderContext context, ValueSourceWeight w) throws IOException {
+      super(w);
       final IndexReader reader = context.reader;
       qWeight = w.getValue();
       // this is when/where the values are first created.

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java Fri Jan 21 19:38:06 2011
@@ -153,7 +153,6 @@ public class PayloadNearQuery extends Sp
     Spans spans;
     protected float payloadScore;
     private int payloadsSeen;
-    Similarity similarity = getSimilarity();
 
     protected PayloadNearSpanScorer(Spans spans, Weight weight,
         Similarity similarity, byte[] norms) throws IOException {
@@ -211,7 +210,7 @@ public class PayloadNearQuery extends Sp
           payloadsSeen = 0;
           do {
             int matchLength = spans.end() - spans.start();
-            freq += getSimilarity().sloppyFreq(matchLength);
+            freq += similarity.sloppyFreq(matchLength);
             Spans[] spansArr = new Spans[1];
             spansArr[0] = spans;
             getPayloads(spansArr);            

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java Fri Jan 21 19:38:06 2011
@@ -100,12 +100,11 @@ public class PayloadTermQuery extends Sp
         freq = 0.0f;
         payloadScore = 0;
         payloadsSeen = 0;
-        Similarity similarity1 = getSimilarity();
         while (more && doc == spans.doc()) {
           int matchLength = spans.end() - spans.start();
 
-          freq += similarity1.sloppyFreq(matchLength);
-          processPayload(similarity1);
+          freq += similarity.sloppyFreq(matchLength);
+          processPayload(similarity);
 
           more = spans.next();// this moves positions to the next match in this
                               // document

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java Fri Jan 21 19:38:06 2011
@@ -25,7 +25,6 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Similarity;
 import org.apache.lucene.util.ToStringUtils;
 
 /**
@@ -107,11 +106,6 @@ public class FieldMaskingSpanQuery exten
   }
 
   @Override
-  public Similarity getSimilarity(IndexSearcher searcher) {
-    return maskedQuery.getSimilarity(searcher);
-  }
-
-  @Override
   public Query rewrite(IndexReader reader) throws IOException {
     FieldMaskingSpanQuery clone = null;
 

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanScorer.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanScorer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanScorer.java Fri Jan 21 19:38:06 2011
@@ -36,10 +36,12 @@ public class SpanScorer extends Scorer {
 
   protected int doc;
   protected float freq;
-
+  protected final Similarity similarity;
+  
   protected SpanScorer(Spans spans, Weight weight, Similarity similarity, byte[] norms)
   throws IOException {
-    super(similarity, weight);
+    super(weight);
+    this.similarity = similarity;
     this.spans = spans;
     this.norms = norms;
     this.value = weight.getValue();
@@ -81,7 +83,7 @@ public class SpanScorer extends Scorer {
     freq = 0.0f;
     do {
       int matchLength = spans.end() - spans.start();
-      freq += getSimilarity().sloppyFreq(matchLength);
+      freq += similarity.sloppyFreq(matchLength);
       more = spans.next();
     } while (more && (doc == spans.doc()));
     return true;
@@ -92,8 +94,8 @@ public class SpanScorer extends Scorer {
 
   @Override
   public float score() throws IOException {
-    float raw = getSimilarity().tf(freq) * value; // raw score
-    return norms == null? raw : raw * getSimilarity().decodeNormValue(norms[doc]); // normalize
+    float raw = similarity.tf(freq) * value; // raw score
+    return norms == null? raw : raw * similarity.decodeNormValue(norms[doc]); // normalize
   }
   
   @Override
@@ -109,7 +111,7 @@ public class SpanScorer extends Scorer {
     int expDoc = advance(doc);
 
     float phraseFreq = (expDoc == doc) ? freq : 0.0f;
-    tfExplanation.setValue(getSimilarity().tf(phraseFreq));
+    tfExplanation.setValue(similarity.tf(phraseFreq));
     tfExplanation.setDescription("tf(phraseFreq=" + phraseFreq + ")");
 
     return tfExplanation;

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java Fri Jan 21 19:38:06 2011
@@ -42,7 +42,7 @@ public class SpanWeight extends Weight {
 
   public SpanWeight(SpanQuery query, IndexSearcher searcher)
     throws IOException {
-    this.similarity = query.getSimilarity(searcher);
+    this.similarity = searcher.getSimilarity();
     this.query = query;
     
     terms=new HashSet<Term>();

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/BufferedIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/BufferedIndexInput.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/BufferedIndexInput.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/BufferedIndexInput.java Fri Jan 21 19:38:06 2011
@@ -144,6 +144,68 @@ public abstract class BufferedIndexInput
     }
   }
 
+  @Override
+  public short readShort() throws IOException {
+    if (2 <= (bufferLength-bufferPosition)) {
+      return (short) (((buffer[bufferPosition++] & 0xFF) <<  8) |  (buffer[bufferPosition++] & 0xFF));
+    } else {
+      return super.readShort();
+    }
+  }
+  
+  @Override
+  public int readInt() throws IOException {
+    if (4 <= (bufferLength-bufferPosition)) {
+      return ((buffer[bufferPosition++] & 0xFF) << 24) | ((buffer[bufferPosition++] & 0xFF) << 16)
+        | ((buffer[bufferPosition++] & 0xFF) <<  8) |  (buffer[bufferPosition++] & 0xFF);
+    } else {
+      return super.readInt();
+    }
+  }
+  
+  @Override
+  public long readLong() throws IOException {
+    if (8 <= (bufferLength-bufferPosition)) {
+      final int i1 = ((buffer[bufferPosition++] & 0xff) << 24) | ((buffer[bufferPosition++] & 0xff) << 16) |
+        ((buffer[bufferPosition++] & 0xff) << 8) | (buffer[bufferPosition++] & 0xff);
+      final int i2 = ((buffer[bufferPosition++] & 0xff) << 24) | ((buffer[bufferPosition++] & 0xff) << 16) |
+        ((buffer[bufferPosition++] & 0xff) << 8) | (buffer[bufferPosition++] & 0xff);
+      return (((long)i1) << 32) | (i2 & 0xFFFFFFFFL);
+    } else {
+      return super.readLong();
+    }
+  }
+
+  @Override
+  public int readVInt() throws IOException {
+    if (5 <= (bufferLength-bufferPosition)) {
+      byte b = buffer[bufferPosition++];
+      int i = b & 0x7F;
+      for (int shift = 7; (b & 0x80) != 0; shift += 7) {
+        b = buffer[bufferPosition++];
+        i |= (b & 0x7F) << shift;
+      }
+      return i;
+    } else {
+      return super.readVInt();
+    }
+  }
+  
+  @Override
+  public long readVLong() throws IOException {
+    if (9 <= bufferLength-bufferPosition) {
+      byte b = buffer[bufferPosition++];
+      long i = b & 0x7F;
+      for (int shift = 7; (b & 0x80) != 0; shift += 7) {
+        b = buffer[bufferPosition++];
+        i |= (b & 0x7FL) << shift;
+      }
+      return i;
+    } else {
+      return super.readVLong();
+    }
+  }
+  
   private void refill() throws IOException {
     long start = bufferStart + bufferPosition;
     long end = start + bufferSize;

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/ByteArrayDataInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/ByteArrayDataInput.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/ByteArrayDataInput.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/ByteArrayDataInput.java Fri Jan 21 19:38:06 2011
@@ -21,7 +21,9 @@ package org.apache.lucene.store;
 public final class ByteArrayDataInput extends DataInput {
 
   private byte[] bytes;
+
   private int pos;
+  private int limit;
 
   // TODO: allow BytesRef (slice) too
   public ByteArrayDataInput(byte[] bytes) {
@@ -29,27 +31,80 @@ public final class ByteArrayDataInput ex
   }
 
   public void reset(byte[] bytes) {
+    reset(bytes, 0, bytes.length);
+  }
+
+  public int getPosition() {
+    return pos;
+  }
+
+  public void reset(byte[] bytes, int offset, int len) {
     this.bytes = bytes;
-    pos = 0;
+    pos = offset;
+    limit = len;
   }
 
   public boolean eof() {
-    return pos == bytes.length;
+    return pos == limit;
   }
 
   public void skipBytes(int count) {
     pos += count;
   }
 
+  @Override
+  public short readShort() {
+    return (short) (((bytes[pos++] & 0xFF) <<  8) |  (bytes[pos++] & 0xFF));
+  }
+ 
+  @Override
+  public int readInt() {
+    return ((bytes[pos++] & 0xFF) << 24) | ((bytes[pos++] & 0xFF) << 16)
+      | ((bytes[pos++] & 0xFF) <<  8) |  (bytes[pos++] & 0xFF);
+  }
+ 
+  @Override
+  public long readLong() {
+    final int i1 = ((bytes[pos++] & 0xff) << 24) | ((bytes[pos++] & 0xff) << 16) |
+      ((bytes[pos++] & 0xff) << 8) | (bytes[pos++] & 0xff);
+    final int i2 = ((bytes[pos++] & 0xff) << 24) | ((bytes[pos++] & 0xff) << 16) |
+      ((bytes[pos++] & 0xff) << 8) | (bytes[pos++] & 0xff);
+    return (((long)i1) << 32) | (i2 & 0xFFFFFFFFL);
+  }
+
+  @Override
+  public int readVInt() {
+    byte b = bytes[pos++];
+    int i = b & 0x7F;
+    for (int shift = 7; (b & 0x80) != 0; shift += 7) {
+      b = bytes[pos++];
+      i |= (b & 0x7F) << shift;
+    }
+    return i;
+  }
+ 
+  @Override
+  public long readVLong() {
+    byte b = bytes[pos++];
+    long i = b & 0x7F;
+    for (int shift = 7; (b & 0x80) != 0; shift += 7) {
+      b = bytes[pos++];
+      i |= (b & 0x7FL) << shift;
+    }
+    return i;
+  }
+
   // NOTE: AIOOBE not EOF if you read too much
   @Override
   public byte readByte() {
+    assert pos < limit;
     return bytes[pos++];
   }
 
   // NOTE: AIOOBE not EOF if you read too much
   @Override
   public void readBytes(byte[] b, int offset, int len) {
+    assert pos + len <= limit;
     System.arraycopy(bytes, pos, b, offset, len);
     pos += len;
   }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/DataInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/DataInput.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/DataInput.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/DataInput.java Fri Jan 21 19:38:06 2011
@@ -79,7 +79,7 @@ public abstract class DataInput implemen
    * supported.
    * @see DataOutput#writeVInt(int)
    */
-  public final int readVInt() throws IOException {
+  public int readVInt() throws IOException {
     byte b = readByte();
     int i = b & 0x7F;
     for (int shift = 7; (b & 0x80) != 0; shift += 7) {
@@ -99,7 +99,7 @@ public abstract class DataInput implemen
   /** Reads a long stored in variable-length format.  Reads between one and
    * nine bytes.  Smaller values take fewer bytes.  Negative numbers are not
    * supported. */
-  public final long readVLong() throws IOException {
+  public long readVLong() throws IOException {
     byte b = readByte();
     long i = b & 0x7F;
     for (int shift = 7; (b & 0x80) != 0; shift += 7) {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/AttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/AttributeImpl.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/AttributeImpl.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/AttributeImpl.java Fri Jan 21 19:38:06 2011
@@ -20,6 +20,8 @@ package org.apache.lucene.util;
 import java.io.Serializable;
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
+import java.lang.ref.WeakReference;
+import java.util.LinkedList;
 
 /**
  * Base class for Attributes that can be added to a 
@@ -37,72 +39,80 @@ public abstract class AttributeImpl impl
   public abstract void clear();
   
   /**
-   * The default implementation of this method accesses all declared
-   * fields of this object and prints the values in the following syntax:
+   * This method returns the current attribute values as a string in the following format
+   * by calling the {@link #reflectWith(AttributeReflector)} method:
    * 
+   * <ul>
+   * <li><em>iff {@code prependAttClass=true}:</em> {@code "AttributeClass#key=value,AttributeClass#key=value"}
+   * <li><em>iff {@code prependAttClass=false}:</em> {@code "key=value,key=value"}
+   * </ul>
+   *
+   * @see #reflectWith(AttributeReflector)
+   */
+  public final String reflectAsString(final boolean prependAttClass) {
+    final StringBuilder buffer = new StringBuilder();
+    reflectWith(new AttributeReflector() {
+      public void reflect(Class<? extends Attribute> attClass, String key, Object value) {
+        if (buffer.length() > 0) {
+          buffer.append(',');
+        }
+        if (prependAttClass) {
+          buffer.append(attClass.getName()).append('#');
+        }
+        buffer.append(key).append('=').append((value == null) ? "null" : value);
+      }
+    });
+    return buffer.toString();
+  }
+  
+  /**
+   * This method is for introspection of attributes, it should simply
+   * add the key/values this attribute holds to the given {@link AttributeReflector}.
+   *
+   * <p>The default implementation calls {@link AttributeReflector#reflect} for all
+   * non-static fields from the implementing class, using the field name as key
+   * and the field value as value. The Attribute class is also determined by reflection.
+   * Please note that the default implementation can only handle single-Attribute
+   * implementations.
+   *
+   * <p>Custom implementations look like this (e.g. for a combined attribute implementation):
    * <pre>
-   *   public String toString() {
-   *     return "start=" + startOffset + ",end=" + endOffset;
+   *   public void reflectWith(AttributeReflector reflector) {
+   *     reflector.reflect(CharTermAttribute.class, "term", term());
+   *     reflector.reflect(PositionIncrementAttribute.class, "positionIncrement", getPositionIncrement());
    *   }
    * </pre>
-   * 
-   * This method may be overridden by subclasses.
-   */
-  @Override
-  public String toString() {
-    StringBuilder buffer = new StringBuilder();
-    Class<?> clazz = this.getClass();
-    Field[] fields = clazz.getDeclaredFields();
+   *
+   * <p>If you implement this method, make sure that for each invocation, the same set of {@link Attribute}
+   * interfaces and keys are passed to {@link AttributeReflector#reflect} in the same order, but possibly
+   * different values. So don't automatically exclude e.g. {@code null} properties!
+   *
+   * @see #reflectAsString(boolean)
+   */
+  public void reflectWith(AttributeReflector reflector) {
+    final Class<? extends AttributeImpl> clazz = this.getClass();
+    final LinkedList<WeakReference<Class<? extends Attribute>>> interfaces = AttributeSource.getAttributeInterfaces(clazz);
+    if (interfaces.size() != 1) {
+      throw new UnsupportedOperationException(clazz.getName() +
+        " implements more than one Attribute interface, the default reflectWith() implementation cannot handle this.");
+    }
+    final Class<? extends Attribute> interf = interfaces.getFirst().get();
+    final Field[] fields = clazz.getDeclaredFields();
     try {
       for (int i = 0; i < fields.length; i++) {
-        Field f = fields[i];
+        final Field f = fields[i];
         if (Modifier.isStatic(f.getModifiers())) continue;
         f.setAccessible(true);
-        Object value = f.get(this);
-        if (buffer.length()>0) {
-          buffer.append(',');
-        }
-        if (value == null) {
-          buffer.append(f.getName() + "=null");
-        } else {
-          buffer.append(f.getName() + "=" + value);
-        }
+        reflector.reflect(interf, f.getName(), f.get(this));
       }
     } catch (IllegalAccessException e) {
       // this should never happen, because we're just accessing fields
       // from 'this'
       throw new RuntimeException(e);
     }
-    
-    return buffer.toString();
   }
   
   /**
-   * Subclasses must implement this method and should compute
-   * a hashCode similar to this:
-   * <pre>
-   *   public int hashCode() {
-   *     int code = startOffset;
-   *     code = code * 31 + endOffset;
-   *     return code;
-   *   }
-   * </pre> 
-   * 
-   * see also {@link #equals(Object)}
-   */
-  @Override
-  public abstract int hashCode();
-  
-  /**
-   * All values used for computation of {@link #hashCode()} 
-   * should be checked here for equality.
-   * 
-   * see also {@link Object#equals(Object)}
-   */
-  @Override
-  public abstract boolean equals(Object other);
-  
-  /**
    * Copies the values from this Attribute into the passed-in
    * target attribute. The target implementation must support all the
    * Attributes this implementation supports.

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/AttributeSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/AttributeSource.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/AttributeSource.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/AttributeSource.java Fri Jan 21 19:38:06 2011
@@ -180,20 +180,9 @@ public class AttributeSource {
   private static final WeakHashMap<Class<? extends AttributeImpl>,LinkedList<WeakReference<Class<? extends Attribute>>>> knownImplClasses =
     new WeakHashMap<Class<? extends AttributeImpl>,LinkedList<WeakReference<Class<? extends Attribute>>>>();
   
-  /** <b>Expert:</b> Adds a custom AttributeImpl instance with one or more Attribute interfaces.
-   * <p><font color="red"><b>Please note:</b> It is not guaranteed, that <code>att</code> is added to
-   * the <code>AttributeSource</code>, because the provided attributes may already exist.
-   * You should always retrieve the wanted attributes using {@link #getAttribute} after adding
-   * with this method and cast to your class.
-   * The recommended way to use custom implementations is using an {@link AttributeFactory}.
-   * </font></p>
-   */
-  public void addAttributeImpl(final AttributeImpl att) {
-    final Class<? extends AttributeImpl> clazz = att.getClass();
-    if (attributeImpls.containsKey(clazz)) return;
-    LinkedList<WeakReference<Class<? extends Attribute>>> foundInterfaces;
+  static LinkedList<WeakReference<Class<? extends Attribute>>> getAttributeInterfaces(final Class<? extends AttributeImpl> clazz) {
     synchronized(knownImplClasses) {
-      foundInterfaces = knownImplClasses.get(clazz);
+      LinkedList<WeakReference<Class<? extends Attribute>>> foundInterfaces = knownImplClasses.get(clazz);
       if (foundInterfaces == null) {
         // we have a strong reference to the class instance holding all interfaces in the list (parameter "att"),
         // so all WeakReferences are never evicted by GC
@@ -210,7 +199,23 @@ public class AttributeSource {
           actClazz = actClazz.getSuperclass();
         } while (actClazz != null);
       }
+      return foundInterfaces;
     }
+  }
+  
+  /** <b>Expert:</b> Adds a custom AttributeImpl instance with one or more Attribute interfaces.
+   * <p><font color="red"><b>Please note:</b> It is not guaranteed, that <code>att</code> is added to
+   * the <code>AttributeSource</code>, because the provided attributes may already exist.
+   * You should always retrieve the wanted attributes using {@link #getAttribute} after adding
+   * with this method and cast to your class.
+   * The recommended way to use custom implementations is using an {@link AttributeFactory}.
+   * </font></p>
+   */
+  public final void addAttributeImpl(final AttributeImpl att) {
+    final Class<? extends AttributeImpl> clazz = att.getClass();
+    if (attributeImpls.containsKey(clazz)) return;
+    final LinkedList<WeakReference<Class<? extends Attribute>>> foundInterfaces =
+      getAttributeInterfaces(clazz);
     
     // add all interfaces of this AttributeImpl to the maps
     for (WeakReference<Class<? extends Attribute>> curInterfaceRef : foundInterfaces) {
@@ -233,7 +238,7 @@ public class AttributeSource {
    * already in this AttributeSource and returns it. Otherwise a
    * new instance is created, added to this AttributeSource and returned. 
    */
-  public <A extends Attribute> A addAttribute(Class<A> attClass) {
+  public final <A extends Attribute> A addAttribute(Class<A> attClass) {
     AttributeImpl attImpl = attributes.get(attClass);
     if (attImpl == null) {
       if (!(attClass.isInterface() && Attribute.class.isAssignableFrom(attClass))) {
@@ -248,7 +253,7 @@ public class AttributeSource {
   }
   
   /** Returns true, iff this AttributeSource has any attributes */
-  public boolean hasAttributes() {
+  public final boolean hasAttributes() {
     return !this.attributes.isEmpty();
   }
 
@@ -256,7 +261,7 @@ public class AttributeSource {
    * The caller must pass in a Class&lt;? extends Attribute&gt; value. 
    * Returns true, iff this AttributeSource contains the passed-in Attribute.
    */
-  public boolean hasAttribute(Class<? extends Attribute> attClass) {
+  public final boolean hasAttribute(Class<? extends Attribute> attClass) {
     return this.attributes.containsKey(attClass);
   }
 
@@ -271,7 +276,7 @@ public class AttributeSource {
    *         available. If you want to only use the attribute, if it is available (to optimize
    *         consuming), use {@link #hasAttribute}.
    */
-  public <A extends Attribute> A getAttribute(Class<A> attClass) {
+  public final <A extends Attribute> A getAttribute(Class<A> attClass) {
     AttributeImpl attImpl = attributes.get(attClass);
     if (attImpl == null) {
       throw new IllegalArgumentException("This AttributeSource does not have the attribute '" + attClass.getName() + "'.");
@@ -319,7 +324,7 @@ public class AttributeSource {
    * Resets all Attributes in this AttributeSource by calling
    * {@link AttributeImpl#clear()} on each Attribute implementation.
    */
-  public void clearAttributes() {
+  public final void clearAttributes() {
     if (hasAttributes()) {
       if (currentState == null) {
         computeCurrentState();
@@ -334,7 +339,7 @@ public class AttributeSource {
    * Captures the state of all Attributes. The return value can be passed to
    * {@link #restoreState} to restore the state of this or another AttributeSource.
    */
-  public State captureState() {
+  public final State captureState() {
     if (!hasAttributes()) {
       return null;
     }
@@ -360,7 +365,7 @@ public class AttributeSource {
    * reset its value to the default, in which case the caller should first
    * call {@link TokenStream#clearAttributes()} on the targetStream.   
    */
-  public void restoreState(State state) {
+  public final void restoreState(State state) {
     if (state == null)  return;
     
     do {
@@ -431,21 +436,53 @@ public class AttributeSource {
       return false;
   }
   
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder().append('(');
+  /**
+   * This method returns the current attribute values as a string in the following format
+   * by calling the {@link #reflectWith(AttributeReflector)} method:
+   * 
+   * <ul>
+   * <li><em>iff {@code prependAttClass=true}:</em> {@code "AttributeClass#key=value,AttributeClass#key=value"}
+   * <li><em>iff {@code prependAttClass=false}:</em> {@code "key=value,key=value"}
+   * </ul>
+   *
+   * @see #reflectWith(AttributeReflector)
+   */
+  public final String reflectAsString(final boolean prependAttClass) {
+    final StringBuilder buffer = new StringBuilder();
+    reflectWith(new AttributeReflector() {
+      public void reflect(Class<? extends Attribute> attClass, String key, Object value) {
+        if (buffer.length() > 0) {
+          buffer.append(',');
+        }
+        if (prependAttClass) {
+          buffer.append(attClass.getName()).append('#');
+        }
+        buffer.append(key).append('=').append((value == null) ? "null" : value);
+      }
+    });
+    return buffer.toString();
+  }
+  
+  /**
+   * This method is for introspection of attributes, it should simply
+   * add the key/values this AttributeSource holds to the given {@link AttributeReflector}.
+   *
+   * <p>This method iterates over all Attribute implementations and calls the
+   * corresponding {@link AttributeImpl#reflectWith} method.</p>
+   *
+   * @see AttributeImpl#reflectWith
+   */
+  public final void reflectWith(AttributeReflector reflector) {
     if (hasAttributes()) {
       if (currentState == null) {
         computeCurrentState();
       }
       for (State state = currentState; state != null; state = state.next) {
-        if (state != currentState) sb.append(',');
-        sb.append(state.attribute.toString());
+        state.attribute.reflectWith(reflector);
       }
     }
-    return sb.append(')').toString();
   }
-  
+
   /**
    * Performs a clone of all {@link AttributeImpl} instances returned in a new
    * {@code AttributeSource} instance. This method can be used to e.g. create another TokenStream
@@ -453,7 +490,7 @@ public class AttributeSource {
    * You can also use it as a (non-performant) replacement for {@link #captureState}, if you need to look
    * into / modify the captured state.
    */
-  public AttributeSource cloneAttributes() {
+  public final AttributeSource cloneAttributes() {
     final AttributeSource clone = new AttributeSource(this.factory);
     
     if (hasAttributes()) {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/PerReaderTermState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/PerReaderTermState.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/PerReaderTermState.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/PerReaderTermState.java Fri Jan 21 19:38:06 2011
@@ -138,9 +138,9 @@ public final class PerReaderTermState {
 
   /**
    *  Returns the accumulated document frequency of all {@link TermState}
-   *         instances passed to {@link #register(TermState, int)}.
+   *         instances passed to {@link #register(TermState, int, int)}.
    * @return the accumulated document frequency of all {@link TermState}
-   *         instances passed to {@link #register(TermState, int)}.
+   *         instances passed to {@link #register(TermState, int, int)}.
    */
   public int docFreq() {
     return docFreq;

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/SorterTemplate.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/SorterTemplate.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/SorterTemplate.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/SorterTemplate.java Fri Jan 21 19:38:06 2011
@@ -36,14 +36,14 @@ public abstract class SorterTemplate {
   protected abstract void swap(int i, int j);
   
   /** Compares slots {@code i} and {@code j} of you data.
-   * Should be implemented like <code><em>valueOf(j)</em>.compareTo(<em>valueOf(i)</em>)</code> */
+   * Should be implemented like <code><em>valueOf(i)</em>.compareTo(<em>valueOf(j)</em>)</code> */
   protected abstract int compare(int i, int j);
 
   /** Implement this method, that stores the value of slot {@code i} as pivot value */
   protected abstract void setPivot(int i);
   
   /** Implements the compare function for the previously stored pivot value.
-   * Should be implemented like <code>pivot.compareTo(<em>valueOf(i)</em>)</code> */
+   * Should be implemented like <code>pivot.compareTo(<em>valueOf(j)</em>)</code> */
   protected abstract int comparePivot(int j);
   
   /** Sorts via stable in-place InsertionSort algorithm

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestExternalCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestExternalCodecs.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestExternalCodecs.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestExternalCodecs.java Fri Jan 21 19:38:06 2011
@@ -245,7 +245,6 @@ public class TestExternalCodecs extends 
       }
     }
 
-
     // Classes for reading from the postings state
     static class RAMFieldsEnum extends FieldsEnum {
       private final RAMPostings postings;
@@ -543,7 +542,7 @@ public class TestExternalCodecs extends 
       // Terms dict
       success = false;
       try {
-        FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, pulsingWriter, reverseUnicodeComparator);
+        FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, pulsingWriter, reverseUnicodeComparator);
         success = true;
         return ret;
       } finally {
@@ -584,15 +583,15 @@ public class TestExternalCodecs extends 
       // Terms dict reader
       success = false;
       try {
-        FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
-                                                         state.dir,
-                                                         state.fieldInfos,
-                                                         state.segmentInfo.name,
-                                                         pulsingReader,
-                                                         state.readBufferSize,
-                                                         reverseUnicodeComparator,
-                                                         StandardCodec.TERMS_CACHE_SIZE,
-                                                         state.codecId);
+        FieldsProducer ret = new BlockTermsReader(indexReader,
+                                                  state.dir,
+                                                  state.fieldInfos,
+                                                  state.segmentInfo.name,
+                                                  pulsingReader,
+                                                  state.readBufferSize,
+                                                  reverseUnicodeComparator,
+                                                  StandardCodec.TERMS_CACHE_SIZE,
+                                                  state.codecId);
         success = true;
         return ret;
       } finally {
@@ -609,7 +608,7 @@ public class TestExternalCodecs extends 
     @Override
     public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) throws IOException {
       StandardPostingsReader.files(dir, segmentInfo, codecId, files);
-      PrefixCodedTermsReader.files(dir, segmentInfo, codecId, files);
+      BlockTermsReader.files(dir, segmentInfo, codecId, files);
       FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
     }
 
@@ -637,6 +636,7 @@ public class TestExternalCodecs extends 
             setCodecProvider(provider).
             setMergePolicy(newLogMergePolicy(3))
     );
+    w.setInfoStream(VERBOSE ? System.out : null);
     Document doc = new Document();
     // uses default codec:
     doc.add(newField("field1", "this field uses the standard codec as the test", Field.Store.NO, Field.Index.ANALYZED));

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestSearch.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestSearch.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestSearch.java Fri Jan 21 19:38:06 2011
@@ -111,6 +111,9 @@ public class TestSearch extends LuceneTe
       for (int j = 0; j < queries.length; j++) {
         Query query = parser.parse(queries[j]);
         out.println("Query: " + query.toString("contents"));
+        if (VERBOSE) {
+          System.out.println("TEST: query=" + query);
+        }
 
         hits = searcher.search(query, null, 1000).scoreDocs;
 

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java Fri Jan 21 19:38:06 2011
@@ -108,6 +108,9 @@ public class TestSearchForDuplicates ext
 
       Query query = parser.parse(HIGH_PRIORITY);
       out.println("Query: " + query.toString(PRIORITY_FIELD));
+      if (VERBOSE) {
+        System.out.println("TEST: search query=" + query);
+      }
 
       ScoreDoc[] hits = searcher.search(query, null, MAX_DOCS).scoreDocs;
       printHits(out, hits, searcher);

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java Fri Jan 21 19:38:06 2011
@@ -38,13 +38,13 @@ public class TestNumericTokenStream exte
     final BytesRef bytes = new BytesRef();
     stream.reset();
     assertEquals(64, numericAtt.getValueSize());
-    assertEquals(lvalue, numericAtt.getRawValue());
     for (int shift=0; shift<64; shift+=NumericUtils.PRECISION_STEP_DEFAULT) {
       assertTrue("New token is available", stream.incrementToken());
       assertEquals("Shift value wrong", shift, numericAtt.getShift());
       final int hash = bytesAtt.toBytesRef(bytes);
       assertEquals("Hash incorrect", bytes.hashCode(), hash);
       assertEquals("Term is incorrectly encoded", lvalue & ~((1L << shift) - 1L), NumericUtils.prefixCodedToLong(bytes));
+      assertEquals("Term raw value is incorrectly encoded", lvalue & ~((1L << shift) - 1L), numericAtt.getRawValue());
       assertEquals("Type incorrect", (shift == 0) ? NumericTokenStream.TOKEN_TYPE_FULL_PREC : NumericTokenStream.TOKEN_TYPE_LOWER_PREC, typeAtt.type());
     }
     assertFalse("More tokens available", stream.incrementToken());
@@ -61,13 +61,13 @@ public class TestNumericTokenStream exte
     final BytesRef bytes = new BytesRef();
     stream.reset();
     assertEquals(32, numericAtt.getValueSize());
-    assertEquals(ivalue, numericAtt.getRawValue());
     for (int shift=0; shift<32; shift+=NumericUtils.PRECISION_STEP_DEFAULT) {
       assertTrue("New token is available", stream.incrementToken());
       assertEquals("Shift value wrong", shift, numericAtt.getShift());
       final int hash = bytesAtt.toBytesRef(bytes);
       assertEquals("Hash incorrect", bytes.hashCode(), hash);
       assertEquals("Term is incorrectly encoded", ivalue & ~((1 << shift) - 1), NumericUtils.prefixCodedToInt(bytes));
+      assertEquals("Term raw value is incorrectly encoded", ((long) ivalue) & ~((1L << shift) - 1L), numericAtt.getRawValue());
       assertEquals("Type incorrect", (shift == 0) ? NumericTokenStream.TOKEN_TYPE_FULL_PREC : NumericTokenStream.TOKEN_TYPE_LOWER_PREC, typeAtt.type());
     }
     assertFalse("More tokens available", stream.incrementToken());

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/TestToken.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/TestToken.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/TestToken.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/TestToken.java Fri Jan 21 19:38:06 2011
@@ -22,8 +22,11 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.Attribute;
 import org.apache.lucene.util.AttributeImpl;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util._TestUtil;
 
 import java.io.StringReader;
+import java.util.HashMap;
 
 public class TestToken extends LuceneTestCase {
 
@@ -241,6 +244,22 @@ public class TestToken extends LuceneTes
       ts.addAttribute(TypeAttribute.class) instanceof Token);
   }
 
+  public void testAttributeReflection() throws Exception {
+    Token t = new Token("foobar", 6, 22, 8);
+    _TestUtil.assertAttributeReflection(t,
+      new HashMap<String,Object>() {{
+        put(CharTermAttribute.class.getName() + "#term", "foobar");
+        put(TermToBytesRefAttribute.class.getName() + "#bytes", new BytesRef("foobar"));
+        put(OffsetAttribute.class.getName() + "#startOffset", 6);
+        put(OffsetAttribute.class.getName() + "#endOffset", 22);
+        put(PositionIncrementAttribute.class.getName() + "#positionIncrement", 1);
+        put(PayloadAttribute.class.getName() + "#payload", null);
+        put(TypeAttribute.class.getName() + "#type", TypeAttribute.DEFAULT_TYPE);
+        put(FlagsAttribute.class.getName() + "#flags", 8);
+      }});
+  }
+
+
   public static <T extends AttributeImpl> T assertCloneIsEqual(T att) {
     @SuppressWarnings("unchecked")
     T clone = (T) att.clone();

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/tokenattributes/TestCharTermAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/tokenattributes/TestCharTermAttributeImpl.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/tokenattributes/TestCharTermAttributeImpl.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/tokenattributes/TestCharTermAttributeImpl.java Fri Jan 21 19:38:06 2011
@@ -19,7 +19,10 @@ package org.apache.lucene.analysis.token
 
 import org.apache.lucene.analysis.TestToken;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util._TestUtil;
 import java.nio.CharBuffer;
+import java.util.HashMap;
 import java.util.Formatter;
 import java.util.Locale;
 import java.util.regex.Pattern;
@@ -126,6 +129,15 @@ public class TestCharTermAttributeImpl e
     assertNotSame(buf, copy.buffer());
   }
   
+  public void testAttributeReflection() throws Exception {
+    CharTermAttributeImpl t = new CharTermAttributeImpl();
+    t.append("foobar");
+    _TestUtil.assertAttributeReflection(t, new HashMap<String,Object>() {{
+      put(CharTermAttribute.class.getName() + "#term", "foobar");
+      put(TermToBytesRefAttribute.class.getName() + "#bytes", new BytesRef("foobar"));
+    }});
+  }
+  
   public void testCharSequenceInterface() {
     final String s = "0123456789"; 
     final CharTermAttributeImpl t = new CharTermAttributeImpl();

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java Fri Jan 21 19:38:06 2011
@@ -55,6 +55,7 @@ public class TestAddIndexes extends Luce
     writer = newWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT,
         new MockAnalyzer())
         .setOpenMode(OpenMode.CREATE));
+    writer.setInfoStream(VERBOSE ? System.out : null);
     // add 100 documents
     addDocs(writer, 100);
     assertEquals(100, writer.maxDoc());

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexReader.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexReader.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexReader.java Fri Jan 21 19:38:06 2011
@@ -882,6 +882,10 @@ public class TestIndexReader extends Luc
       // First build up a starting index:
       MockDirectoryWrapper startDir = newDirectory();
       IndexWriter writer = new IndexWriter(startDir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()));
+      if (VERBOSE) {
+        System.out.println("TEST: create initial index");
+        writer.setInfoStream(System.out);
+      }
       for(int i=0;i<157;i++) {
         Document d = new Document();
         d.add(newField("id", Integer.toString(i), Field.Store.YES, Field.Index.NOT_ANALYZED));
@@ -892,6 +896,19 @@ public class TestIndexReader extends Luc
       }
       writer.close();
 
+      {
+        IndexReader r = IndexReader.open(startDir);
+        IndexSearcher searcher = new IndexSearcher(r);
+        ScoreDoc[] hits = null;
+        try {
+          hits = searcher.search(new TermQuery(searchTerm), null, 1000).scoreDocs;
+        } catch (IOException e) {
+          e.printStackTrace();
+          fail("exception when init searching: " + e);
+        }
+        r.close();
+      }
+
       long diskUsage = startDir.getRecomputedActualSizeInBytes();
       long diskFree = diskUsage+100;
 

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexReaderReopen.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexReaderReopen.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexReaderReopen.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexReaderReopen.java Fri Jan 21 19:38:06 2011
@@ -977,7 +977,11 @@ public class TestIndexReaderReopen exten
   static void modifyIndex(int i, Directory dir) throws IOException {
     switch (i) {
       case 0: {
+        if (VERBOSE) {
+          System.out.println("TEST: modify index");
+        }
         IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()));
+        w.setInfoStream(VERBOSE ? System.out : null);
         w.deleteDocuments(new Term("field2", "a11"));
         w.deleteDocuments(new Term("field2", "b30"));
         w.close();

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java Fri Jan 21 19:38:06 2011
@@ -784,7 +784,7 @@ public class TestIndexWriter extends Luc
     public void testHighFreqTerm() throws IOException {
       MockDirectoryWrapper dir = newDirectory();
       IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(
-          TEST_VERSION_CURRENT, new MockAnalyzer()).setMaxFieldLength(100000000).setRAMBufferSizeMB(0.01));
+          TEST_VERSION_CURRENT, new MockAnalyzer()).setRAMBufferSizeMB(0.01));
       // Massive doc that has 128 K a's
       StringBuilder b = new StringBuilder(1024*1024);
       for(int i=0;i<4096;i++) {
@@ -1236,30 +1236,7 @@ public class TestIndexWriter extends Luc
     writer.close();
     dir.close();
   }
-
-  // LUCENE-1084: test user-specified field length
-  public void testUserSpecifiedMaxFieldLength() throws IOException {
-    Directory dir = newDirectory();
-
-    IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(
-        TEST_VERSION_CURRENT, new MockAnalyzer()).setMaxFieldLength(100000));
-
-    Document doc = new Document();
-    StringBuilder b = new StringBuilder();
-    for(int i=0;i<10000;i++)
-      b.append(" a");
-    b.append(" x");
-    doc.add(newField("field", b.toString(), Field.Store.NO, Field.Index.ANALYZED));
-    writer.addDocument(doc);
-    writer.close();
-
-    IndexReader reader = IndexReader.open(dir, true);
-    Term t = new Term("field", "x");
-    assertEquals(1, reader.docFreq(t));
-    reader.close();
-    dir.close();
-  }
-
+  
   // LUCENE-325: test expungeDeletes, when 2 singular merges
   // are required
   public void testExpungeDeletes() throws IOException {

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriterConfig.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriterConfig.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriterConfig.java Fri Jan 21 19:38:06 2011
@@ -17,7 +17,6 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import java.io.IOException;
 import java.lang.reflect.Field;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
@@ -26,7 +25,6 @@ import java.util.Set;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
-import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.search.DefaultSimilarity;
 import org.apache.lucene.search.Similarity;
@@ -49,22 +47,12 @@ public class TestIndexWriterConfig exten
 
   }
 
-  private static final class MyWarmer extends IndexReaderWarmer {
-    // Does not implement anything - used only for type checking on IndexWriterConfig.
-
-    @Override
-    public void warm(IndexReader reader) throws IOException {
-    }
-
-  }
-
   @Test
   public void testDefaults() throws Exception {
     IndexWriterConfig conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer());
     assertEquals(MockAnalyzer.class, conf.getAnalyzer().getClass());
     assertNull(conf.getIndexCommit());
     assertEquals(KeepOnlyLastCommitDeletionPolicy.class, conf.getIndexDeletionPolicy().getClass());
-    assertEquals(IndexWriterConfig.UNLIMITED_FIELD_LENGTH, conf.getMaxFieldLength());
     assertEquals(ConcurrentMergeScheduler.class, conf.getMergeScheduler().getClass());
     assertEquals(OpenMode.CREATE_OR_APPEND, conf.getOpenMode());
     assertTrue(Similarity.getDefault() == conf.getSimilarity());
@@ -131,7 +119,6 @@ public class TestIndexWriterConfig exten
     // Tests that the values of the constants does not change
     assertEquals(1000, IndexWriterConfig.WRITE_LOCK_TIMEOUT);
     assertEquals(32, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL);
-    assertEquals(Integer.MAX_VALUE, IndexWriterConfig.UNLIMITED_FIELD_LENGTH);
     assertEquals(-1, IndexWriterConfig.DISABLE_AUTO_FLUSH);
     assertEquals(IndexWriterConfig.DISABLE_AUTO_FLUSH, IndexWriterConfig.DEFAULT_MAX_BUFFERED_DELETE_TERMS);
     assertEquals(IndexWriterConfig.DISABLE_AUTO_FLUSH, IndexWriterConfig.DEFAULT_MAX_BUFFERED_DOCS);

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java Fri Jan 21 19:38:06 2011
@@ -50,7 +50,7 @@ public class TestIndexWriterOnJRECrash e
   public void testNRTThreads() throws Exception {
     String vendor = Constants.JAVA_VENDOR;
     assumeTrue(vendor + " JRE not supported.",
-        vendor.startsWith("Sun") || vendor.startsWith("IBM") || vendor.startsWith("Apple"));
+        vendor.startsWith("Sun") || vendor.startsWith("Apple"));
 
     // if we are not the fork
     if (System.getProperty("tests.crashmode") == null) {

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestMultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestMultiFields.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestMultiFields.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestMultiFields.java Fri Jan 21 19:38:06 2011
@@ -97,6 +97,9 @@ public class TestMultiFields extends Luc
 
       for(int i=0;i<100;i++) {
         BytesRef term = terms.get(random.nextInt(terms.size()));
+        if (VERBOSE) {
+          System.out.println("TEST: seek to term= "+ UnicodeUtil.toHexString(term.utf8ToString()));
+        }
         
         DocsEnum docsEnum = terms2.docs(delDocs, term, null);
         assertNotNull(docsEnum);

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java Fri Jan 21 19:38:06 2011
@@ -37,8 +37,8 @@ import org.apache.lucene.index.codecs.Fi
 import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.index.codecs.PostingsWriterBase;
 import org.apache.lucene.index.codecs.PostingsReaderBase;
-import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
-import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
+import org.apache.lucene.index.codecs.BlockTermsReader;
+import org.apache.lucene.index.codecs.BlockTermsWriter;
 import org.apache.lucene.index.codecs.TermsIndexReaderBase;
 import org.apache.lucene.index.codecs.TermsIndexWriterBase;
 import org.apache.lucene.index.codecs.standard.StandardCodec;
@@ -126,7 +126,7 @@ public class MockFixedIntBlockCodec exte
 
     success = false;
     try {
-      FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
       success = true;
       return ret;
     } finally {
@@ -164,15 +164,15 @@ public class MockFixedIntBlockCodec exte
 
     success = false;
     try {
-      FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
-                                                       state.dir,
-                                                       state.fieldInfos,
-                                                       state.segmentInfo.name,
-                                                       postingsReader,
-                                                       state.readBufferSize,
-                                                       BytesRef.getUTF8SortedAsUnicodeComparator(),
-                                                       StandardCodec.TERMS_CACHE_SIZE,
-                                                       state.codecId);
+      FieldsProducer ret = new BlockTermsReader(indexReader,
+                                                state.dir,
+                                                state.fieldInfos,
+                                                state.segmentInfo.name,
+                                                postingsReader,
+                                                state.readBufferSize,
+                                                BytesRef.getUTF8SortedAsUnicodeComparator(),
+                                                StandardCodec.TERMS_CACHE_SIZE,
+                                                state.codecId);
       success = true;
       return ret;
     } finally {
@@ -189,14 +189,14 @@ public class MockFixedIntBlockCodec exte
   @Override
   public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) {
     SepPostingsReaderImpl.files(segmentInfo, codecId, files);
-    PrefixCodedTermsReader.files(dir, segmentInfo, codecId, files);
+    BlockTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
   }
 
   @Override
   public void getExtensions(Set<String> extensions) {
     SepPostingsWriterImpl.getExtensions(extensions);
-    PrefixCodedTermsReader.getExtensions(extensions);
+    BlockTermsReader.getExtensions(extensions);
     FixedGapTermsIndexReader.getIndexExtensions(extensions);
   }
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java Fri Jan 21 19:38:06 2011
@@ -37,8 +37,8 @@ import org.apache.lucene.index.codecs.Fi
 import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.index.codecs.PostingsWriterBase;
 import org.apache.lucene.index.codecs.PostingsReaderBase;
-import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
-import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
+import org.apache.lucene.index.codecs.BlockTermsReader;
+import org.apache.lucene.index.codecs.BlockTermsWriter;
 import org.apache.lucene.index.codecs.TermsIndexReaderBase;
 import org.apache.lucene.index.codecs.TermsIndexWriterBase;
 import org.apache.lucene.index.codecs.standard.StandardCodec;
@@ -150,7 +150,7 @@ public class MockVariableIntBlockCodec e
 
     success = false;
     try {
-      FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
       success = true;
       return ret;
     } finally {
@@ -189,15 +189,15 @@ public class MockVariableIntBlockCodec e
 
     success = false;
     try {
-      FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
-                                                       state.dir,
-                                                       state.fieldInfos,
-                                                       state.segmentInfo.name,
-                                                       postingsReader,
-                                                       state.readBufferSize,
-                                                       BytesRef.getUTF8SortedAsUnicodeComparator(),
-                                                       StandardCodec.TERMS_CACHE_SIZE,
-                                                       state.codecId);
+      FieldsProducer ret = new BlockTermsReader(indexReader,
+                                                state.dir,
+                                                state.fieldInfos,
+                                                state.segmentInfo.name,
+                                                postingsReader,
+                                                state.readBufferSize,
+                                                BytesRef.getUTF8SortedAsUnicodeComparator(),
+                                                StandardCodec.TERMS_CACHE_SIZE,
+                                                state.codecId);
       success = true;
       return ret;
     } finally {
@@ -214,14 +214,14 @@ public class MockVariableIntBlockCodec e
   @Override
   public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) {
     SepPostingsReaderImpl.files(segmentInfo, codecId, files);
-    PrefixCodedTermsReader.files(dir, segmentInfo, codecId, files);
+    BlockTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
   }
 
   @Override
   public void getExtensions(Set<String> extensions) {
     SepPostingsWriterImpl.getExtensions(extensions);
-    PrefixCodedTermsReader.getExtensions(extensions);
+    BlockTermsReader.getExtensions(extensions);
     FixedGapTermsIndexReader.getIndexExtensions(extensions);
   }
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java Fri Jan 21 19:38:06 2011
@@ -26,6 +26,9 @@ import org.apache.lucene.index.IndexFile
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.codecs.BlockTermsReader;
+import org.apache.lucene.index.codecs.BlockTermsWriter;
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
@@ -33,13 +36,11 @@ import org.apache.lucene.index.codecs.Fi
 import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.index.codecs.PostingsReaderBase;
 import org.apache.lucene.index.codecs.PostingsWriterBase;
-import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
-import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
+import org.apache.lucene.index.codecs.TermStats;
 import org.apache.lucene.index.codecs.TermsIndexReaderBase;
 import org.apache.lucene.index.codecs.TermsIndexWriterBase;
 import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
 import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
-import org.apache.lucene.index.codecs.TermStats;
 import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockCodec;
 import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockCodec;
 import org.apache.lucene.index.codecs.mocksep.MockSingleIntFactory;
@@ -152,6 +153,10 @@ public class MockRandomCodec extends Cod
               public boolean isIndexTerm(BytesRef term, TermStats stats) {
                 return random.nextInt(gap) == 17;
               }
+
+              @Override
+              public void newField(FieldInfo fieldInfo) {
+              }
             };
         }
         indexWriter = new VariableGapTermsIndexWriter(state, selector);
@@ -165,7 +170,7 @@ public class MockRandomCodec extends Cod
 
     success = false;
     try {
-      FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
       success = true;
       return ret;
     } finally {
@@ -269,15 +274,15 @@ public class MockRandomCodec extends Cod
 
     success = false;
     try {
-      FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
-                                                      state.dir,
-                                                      state.fieldInfos,
-                                                      state.segmentInfo.name,
-                                                      postingsReader,
-                                                      state.readBufferSize,
-                                                      BytesRef.getUTF8SortedAsUnicodeComparator(),
-                                                      termsCacheSize,
-                                                      state.codecId);
+      FieldsProducer ret = new BlockTermsReader(indexReader,
+                                                state.dir,
+                                                state.fieldInfos,
+                                                state.segmentInfo.name,
+                                                postingsReader,
+                                                state.readBufferSize,
+                                                BytesRef.getUTF8SortedAsUnicodeComparator(),
+                                                termsCacheSize,
+                                                state.codecId);
       success = true;
       return ret;
     } finally {
@@ -297,7 +302,7 @@ public class MockRandomCodec extends Cod
     files.add(seedFileName);
     SepPostingsReaderImpl.files(segmentInfo, codecId, files);
     StandardPostingsReader.files(dir, segmentInfo, codecId, files);
-    PrefixCodedTermsReader.files(dir, segmentInfo, codecId, files);
+    BlockTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
     VariableGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
     
@@ -315,7 +320,7 @@ public class MockRandomCodec extends Cod
   @Override
   public void getExtensions(Set<String> extensions) {
     SepPostingsWriterImpl.getExtensions(extensions);
-    PrefixCodedTermsReader.getExtensions(extensions);
+    BlockTermsReader.getExtensions(extensions);
     FixedGapTermsIndexReader.getIndexExtensions(extensions);
     VariableGapTermsIndexReader.getIndexExtensions(extensions);
     extensions.add(SEED_EXT);

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java Fri Jan 21 19:38:06 2011
@@ -30,8 +30,8 @@ import org.apache.lucene.index.codecs.Fi
 import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.index.codecs.PostingsReaderBase;
 import org.apache.lucene.index.codecs.PostingsWriterBase;
-import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
-import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
+import org.apache.lucene.index.codecs.BlockTermsReader;
+import org.apache.lucene.index.codecs.BlockTermsWriter;
 import org.apache.lucene.index.codecs.TermsIndexReaderBase;
 import org.apache.lucene.index.codecs.TermsIndexWriterBase;
 import org.apache.lucene.index.codecs.standard.StandardCodec;
@@ -70,7 +70,7 @@ public class MockSepCodec extends Codec 
 
     success = false;
     try {
-      FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
       success = true;
       return ret;
     } finally {
@@ -108,15 +108,15 @@ public class MockSepCodec extends Codec 
 
     success = false;
     try {
-      FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
-                                                       state.dir,
-                                                       state.fieldInfos,
-                                                       state.segmentInfo.name,
-                                                       postingsReader,
-                                                       state.readBufferSize,
-                                                       BytesRef.getUTF8SortedAsUnicodeComparator(),
-                                                       StandardCodec.TERMS_CACHE_SIZE,
-                                                       state.codecId);
+      FieldsProducer ret = new BlockTermsReader(indexReader,
+                                                state.dir,
+                                                state.fieldInfos,
+                                                state.segmentInfo.name,
+                                                postingsReader,
+                                                state.readBufferSize,
+                                                BytesRef.getUTF8SortedAsUnicodeComparator(),
+                                                StandardCodec.TERMS_CACHE_SIZE,
+                                                state.codecId);
       success = true;
       return ret;
     } finally {
@@ -133,7 +133,7 @@ public class MockSepCodec extends Codec 
   @Override
   public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) {
     SepPostingsReaderImpl.files(segmentInfo, codecId, files);
-    PrefixCodedTermsReader.files(dir, segmentInfo, codecId, files);
+    BlockTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
   }
 
@@ -144,7 +144,7 @@ public class MockSepCodec extends Codec 
 
   public static void getSepExtensions(Set<String> extensions) {
     SepPostingsWriterImpl.getExtensions(extensions);
-    PrefixCodedTermsReader.getExtensions(extensions);
+    BlockTermsReader.getExtensions(extensions);
     FixedGapTermsIndexReader.getIndexExtensions(extensions);
   }
-}
\ No newline at end of file
+}

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexInput.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexInput.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexInput.java Fri Jan 21 19:38:06 2011
@@ -19,10 +19,11 @@ package org.apache.lucene.index.codecs.m
 
 import java.io.IOException;
 
+import org.apache.lucene.index.codecs.sep.IntIndexInput;
+import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.CodecUtil;
-import org.apache.lucene.index.codecs.sep.IntIndexInput;
 
 /** Reads IndexInputs written with {@link
  *  SingleIntIndexOutput}.  NOTE: this class is just for
@@ -63,6 +64,7 @@ public class MockSingleIntIndexInput ext
     /** Reads next single int */
     @Override
     public int next() throws IOException {
+      //System.out.println("msii.next() fp=" + in.getFilePointer() + " vs " + in.length());
       return in.readVInt();
     }
   }
@@ -71,7 +73,7 @@ public class MockSingleIntIndexInput ext
     private long fp;
 
     @Override
-    public void read(IndexInput indexIn, boolean absolute)
+    public void read(DataInput indexIn, boolean absolute)
       throws IOException {
       if (absolute) {
         fp = indexIn.readVLong();

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/search/JustCompileSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/search/JustCompileSearch.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/search/JustCompileSearch.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/search/JustCompileSearch.java Fri Jan 21 19:38:06 2011
@@ -210,8 +210,8 @@ final class JustCompileSearch {
   
   static final class JustCompileScorer extends Scorer {
 
-    protected JustCompileScorer(Similarity similarity) {
-      super(similarity);
+    protected JustCompileScorer(Weight weight) {
+      super(weight);
     }
 
     @Override