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 [3/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/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=1061979&r1=1061978&r2=1061979&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 Fri Jan 21 19:38:06 2011
@@ -20,16 +20,18 @@ package org.apache.lucene.index.codecs.s
 import java.io.IOException;
 import java.util.Collection;
 
-import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.codecs.BlockTermState;
 import org.apache.lucene.index.codecs.PostingsReaderBase;
-import org.apache.lucene.index.codecs.PrefixCodedTermState;
+import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CodecUtil;
@@ -130,16 +132,31 @@ public class SepPostingsReaderImpl exten
     }
   }
 
-  private static final class SepTermState extends PrefixCodedTermState {
+  private static final class SepTermState extends BlockTermState {
     // We store only the seek point to the docs file because
     // the rest of the info (freqIndex, posIndex, etc.) is
     // stored in the docs file:
     IntIndexInput.Index docIndex;
-    
+    IntIndexInput.Index posIndex;
+    IntIndexInput.Index freqIndex;
+    long payloadFP;
+    long skipFP;
+
+    // Only used for "primary" term state; these are never
+    // copied on clone:
+    byte[] bytes;
+    ByteArrayDataInput bytesReader;
+
     @Override
     public Object clone() {
       SepTermState other = (SepTermState) super.clone();
       other.docIndex = (IntIndexInput.Index) docIndex.clone();
+      if (freqIndex != null) {
+        other.freqIndex = (IntIndexInput.Index) freqIndex.clone();
+      }
+      if (posIndex != null) {
+        other.posIndex = (IntIndexInput.Index) posIndex.clone();
+      }
       return other;
     }
 
@@ -147,28 +164,87 @@ public class SepPostingsReaderImpl exten
       super.copyFrom(_other);
       SepTermState other = (SepTermState) _other;
       docIndex.set(other.docIndex);
+      if (freqIndex != null && other.freqIndex != null) {
+        freqIndex.set(other.freqIndex);
+      }
+      if (posIndex != null && other.posIndex != null) {
+        posIndex.set(other.posIndex);
+      }
+      payloadFP = other.payloadFP;
+      skipFP = other.skipFP;
     }
 
     @Override
     public String toString() {
-      return "tis.fp=" + filePointer + " docFreq=" + docFreq + " ord=" + ord + " docIndex=" + docIndex;
+      return super.toString() + " docIndex=" + docIndex + " freqIndex=" + freqIndex + " posIndex=" + posIndex + " payloadFP=" + payloadFP + " skipFP=" + skipFP;
     }
   }
 
   @Override
-  public PrefixCodedTermState newTermState() throws IOException {
-    final SepTermState state =  new SepTermState();
+  public BlockTermState newTermState() throws IOException {
+    final SepTermState state = new SepTermState();
     state.docIndex = docIn.index();
+    if (freqIn != null) {
+      state.freqIndex = freqIn.index();
+    }
+    if (posIn != null) {
+      state.posIndex = posIn.index();
+    }
     return state;
   }
 
   @Override
-  public void readTerm(IndexInput termsIn, FieldInfo fieldInfo, PrefixCodedTermState termState, boolean isIndexTerm) throws IOException {
-    ((SepTermState) termState).docIndex.read(termsIn, isIndexTerm);
+  public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
+    final SepTermState termState = (SepTermState) _termState;
+    final int len = termsIn.readVInt();
+    //System.out.println("SepR.readTermsBlock len=" + len);
+    if (termState.bytes == null) {
+      termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
+      termState.bytesReader = new ByteArrayDataInput(termState.bytes);
+    } else if (termState.bytes.length < len) {
+      termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
+    }
+    termState.bytesReader.reset(termState.bytes, 0, len);
+    termsIn.readBytes(termState.bytes, 0, len);
+  }
+
+  @Override
+  public void nextTerm(FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
+    final SepTermState termState = (SepTermState) _termState;
+    //System.out.println("SepR.nextTerm termCount=" + termState.termCount);
+    //System.out.println("  docFreq=" + termState.docFreq);
+    final boolean isFirstTerm = termState.termCount == 0;
+    termState.docIndex.read(termState.bytesReader, isFirstTerm);
+    //System.out.println("  docIndex=" + termState.docIndex);
+    if (!fieldInfo.omitTermFreqAndPositions) {
+      termState.freqIndex.read(termState.bytesReader, isFirstTerm);
+      //System.out.println("  freqIndex=" + termState.freqIndex);
+      termState.posIndex.read(termState.bytesReader, isFirstTerm);
+      //System.out.println("  posIndex=" + termState.posIndex);
+      if (fieldInfo.storePayloads) {
+        if (isFirstTerm) {
+          termState.payloadFP = termState.bytesReader.readVLong();
+        } else {
+          termState.payloadFP += termState.bytesReader.readVLong();
+        }
+        //System.out.println("  payloadFP=" + termState.payloadFP);
+      }
+    }
+    if (termState.docFreq >= skipInterval) {
+      //System.out.println("   readSkip @ " + termState.bytesReader.pos);
+      if (isFirstTerm) {
+        termState.skipFP = termState.bytesReader.readVLong();
+      } else {
+        termState.skipFP += termState.bytesReader.readVLong();
+      }
+      //System.out.println("  skipFP=" + termState.skipFP);
+    } else if (isFirstTerm) {
+      termState.skipFP = termState.bytesReader.readVLong();
+    }
   }
 
   @Override
-  public DocsEnum docs(FieldInfo fieldInfo, PrefixCodedTermState _termState, Bits skipDocs, DocsEnum reuse) throws IOException {
+  public DocsEnum docs(FieldInfo fieldInfo, BlockTermState _termState, Bits skipDocs, DocsEnum reuse) throws IOException {
     final SepTermState termState = (SepTermState) _termState;
     SepDocsEnum docsEnum;
     if (reuse == null || !(reuse instanceof SepDocsEnum)) {
@@ -187,7 +263,7 @@ public class SepPostingsReaderImpl exten
   }
 
   @Override
-  public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, PrefixCodedTermState _termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
+  public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState _termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
     assert !fieldInfo.omitTermFreqAndPositions;
     final SepTermState termState = (SepTermState) _termState;
     SepDocsAndPositionsEnum postingsEnum;
@@ -219,7 +295,7 @@ public class SepPostingsReaderImpl exten
     private Bits skipDocs;
     private final IntIndexInput.Reader docReader;
     private final IntIndexInput.Reader freqReader;
-    private long skipOffset;
+    private long skipFP;
 
     private final IntIndexInput.Index docIndex;
     private final IntIndexInput.Index freqIndex;
@@ -260,18 +336,15 @@ public class SepPostingsReaderImpl exten
       docIndex.seek(docReader);
 
       if (!omitTF) {
-        freqIndex.read(docReader, true);
+        freqIndex.set(termState.freqIndex);
         freqIndex.seek(freqReader);
-        
-        posIndex.read(docReader, true);
-        // skip payload offset
-        docReader.readVLong();
       } else {
         freq = 1;
       }
-      skipOffset = docReader.readVLong();
 
       docFreq = termState.docFreq;
+      // NOTE: unused if docFreq < skipInterval:
+      skipFP = termState.skipFP;
       count = 0;
       doc = 0;
       skipped = false;
@@ -290,9 +363,11 @@ public class SepPostingsReaderImpl exten
         count++;
 
         // Decode next doc
+        //System.out.println("decode docDelta:");
         doc += docReader.next();
           
         if (!omitTF) {
+          //System.out.println("decode freq:");
           freq = freqReader.next();
         }
 
@@ -300,13 +375,13 @@ public class SepPostingsReaderImpl exten
           break;
         }
       }
-
       return doc;
     }
 
     @Override
     public int read() throws IOException {
       // TODO: -- switch to bulk read api in IntIndexInput
+      //System.out.println("sepdocs read");
       final int[] docs = bulkResult.docs.ints;
       final int[] freqs = bulkResult.freqs.ints;
       int i = 0;
@@ -314,14 +389,17 @@ public class SepPostingsReaderImpl exten
       while (i < length && count < docFreq) {
         count++;
         // manually inlined call to next() for speed
+        //System.out.println("decode doc");
         doc += docReader.next();
         if (!omitTF) {
+          //System.out.println("decode freq");
           freq = freqReader.next();
         }
 
         if (skipDocs == null || !skipDocs.get(doc)) {
           docs[i] = doc;
           freqs[i] = freq;
+          //System.out.println("  docs[" + i + "]=" + doc + " count=" + count + " dF=" + docFreq);
           i++;
         }
       }
@@ -361,7 +439,7 @@ public class SepPostingsReaderImpl exten
 
         if (!skipped) {
           // We haven't yet skipped for this posting
-          skipper.init(skipOffset,
+          skipper.init(skipFP,
                        docIndex,
                        freqIndex,
                        posIndex,
@@ -411,14 +489,14 @@ public class SepPostingsReaderImpl exten
     private final IntIndexInput.Reader freqReader;
     private final IntIndexInput.Reader posReader;
     private final IndexInput payloadIn;
-    private long skipOffset;
+    private long skipFP;
 
     private final IntIndexInput.Index docIndex;
     private final IntIndexInput.Index freqIndex;
     private final IntIndexInput.Index posIndex;
     private final IntIndexInput startDocIn;
 
-    private long payloadOffset;
+    private long payloadFP;
 
     private int pendingPosCount;
     private int position;
@@ -444,21 +522,26 @@ public class SepPostingsReaderImpl exten
     SepDocsAndPositionsEnum init(FieldInfo fieldInfo, SepTermState termState, Bits skipDocs) throws IOException {
       this.skipDocs = skipDocs;
       storePayloads = fieldInfo.storePayloads;
+      //System.out.println("Sep D&P init");
 
       // TODO: can't we only do this if consumer
       // skipped consuming the previous docs?
       docIndex.set(termState.docIndex);
       docIndex.seek(docReader);
+      //System.out.println("  docIndex=" + docIndex);
 
-      freqIndex.read(docReader, true);
+      freqIndex.set(termState.freqIndex);
       freqIndex.seek(freqReader);
+      //System.out.println("  freqIndex=" + freqIndex);
 
-      posIndex.read(docReader, true);
+      posIndex.set(termState.posIndex);
+      //System.out.println("  posIndex=" + posIndex);
       posSeekPending = true;
       payloadPending = false;
 
-      payloadOffset = docReader.readVLong();
-      skipOffset = docReader.readVLong();
+      payloadFP = termState.payloadFP;
+      skipFP = termState.skipFP;
+      //System.out.println("  skipFP=" + skipFP);
 
       docFreq = termState.docFreq;
       count = 0;
@@ -484,8 +567,10 @@ public class SepPostingsReaderImpl exten
         // freq=1 case?
 
         // Decode next doc
+        //System.out.println("  sep d&p read doc");
         doc += docReader.next();
-          
+
+        //System.out.println("  sep d&p read freq");
         freq = freqReader.next();
 
         pendingPosCount += freq;
@@ -511,6 +596,7 @@ public class SepPostingsReaderImpl exten
 
     @Override
     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?
@@ -521,6 +607,7 @@ public class SepPostingsReaderImpl exten
         // skip data
 
         if (skipper == null) {
+          //System.out.println("  create skipper");
           // This DocsEnum has never done any skipping
           skipper = new SepSkipListReader((IndexInput) skipIn.clone(),
                                           freqIn,
@@ -530,46 +617,54 @@ public class SepPostingsReaderImpl exten
         }
 
         if (!skipped) {
+          //System.out.println("  init skip data skipFP=" + skipFP);
           // We haven't yet skipped for this posting
-          skipper.init(skipOffset,
+          skipper.init(skipFP,
                        docIndex,
                        freqIndex,
                        posIndex,
-                       payloadOffset,
+                       payloadFP,
                        docFreq,
                        storePayloads);
 
           skipped = true;
         }
-
         final int newCount = skipper.skipTo(target); 
+        //System.out.println("  skip newCount=" + newCount + " vs " + count);
 
         if (newCount > count) {
 
           // Skipper did move
           skipper.getFreqIndex().seek(freqReader);
           skipper.getDocIndex().seek(docReader);
-          //skipper.getPosIndex().seek(posReader);
+          // NOTE: don't seek pos here; do it lazily
+          // instead.  Eg a PhraseQuery may skip to many
+          // docs before finally asking for positions...
           posIndex.set(skipper.getPosIndex());
           posSeekPending = true;
           count = newCount;
           doc = skipper.getDoc();
+          //System.out.println("    moved to doc=" + doc);
           //payloadIn.seek(skipper.getPayloadPointer());
-          payloadOffset = skipper.getPayloadPointer();
+          payloadFP = skipper.getPayloadPointer();
           pendingPosCount = 0;
           pendingPayloadBytes = 0;
           payloadPending = false;
           payloadLength = skipper.getPayloadLength();
+          //System.out.println("    move payloadLen=" + payloadLength);
         }
       }
         
       // Now, linear scan for the rest:
       do {
         if (nextDoc() == NO_MORE_DOCS) {
+          //System.out.println("  advance nextDoc=END");
           return NO_MORE_DOCS;
         }
+        //System.out.println("  advance nextDoc=" + doc);
       } while (target > doc);
 
+      //System.out.println("  return doc=" + doc);
       return doc;
     }
 
@@ -577,7 +672,7 @@ public class SepPostingsReaderImpl exten
     public int nextPosition() throws IOException {
       if (posSeekPending) {
         posIndex.seek(posReader);
-        payloadIn.seek(payloadOffset);
+        payloadIn.seek(payloadFP);
         posSeekPending = false;
       }
 

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=1061979&r1=1061978&r2=1061979&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 Fri Jan 21 19:38:06 2011
@@ -27,6 +27,7 @@ import org.apache.lucene.index.SegmentWr
 import org.apache.lucene.index.codecs.PostingsWriterBase;
 import org.apache.lucene.index.codecs.TermStats;
 import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CodecUtil;
 
@@ -69,8 +70,7 @@ public final class SepPostingsWriterImpl
   boolean storePayloads;
   boolean omitTF;
 
-  // Starts a new term
-  long lastSkipStart;
+  long lastSkipFP;
 
   FieldInfo fieldInfo;
 
@@ -80,7 +80,10 @@ public final class SepPostingsWriterImpl
   long lastPayloadStart;
   int lastDocID;
   int df;
-  private boolean firstDoc;
+  private int pendingTermCount;
+
+  // Holds pending byte[] blob for the current terms block
+  private final RAMOutputStream indexBytesWriter = new RAMOutputStream();
 
   public SepPostingsWriterImpl(SegmentWriteState state, IntStreamFactory factory) throws IOException {
     super();
@@ -144,13 +147,9 @@ public final class SepPostingsWriterImpl
       payloadStart = payloadOut.getFilePointer();
       lastPayloadLength = -1;
     }
-    firstDoc = true;
     skipListWriter.resetSkip(docIndex, freqIndex, posIndex);
   }
 
-  // TODO: -- should we NOT reuse across fields?  would
-  // be cleaner
-
   // Currently, this instance is re-used across fields, so
   // our parent calls setField whenever the field changes
   @Override
@@ -161,27 +160,13 @@ public final class SepPostingsWriterImpl
     storePayloads = !omitTF && fieldInfo.storePayloads;
   }
 
-
   /** Adds a new doc in this term.  If this returns null
    *  then we just skip consuming positions/payloads. */
   @Override
   public void startDoc(int docID, int termDocFreq) throws IOException {
 
-    if (firstDoc) {
-      // TODO: we are writing absolute file pointers below,
-      // which is wasteful.  It'd be better compression to
-      // write the "baseline" into each indexed term, then
-      // write only the delta here.
-      if (!omitTF) {
-        freqIndex.write(docOut, true);
-        posIndex.write(docOut, true);
-        docOut.writeVLong(payloadStart);
-      }
-      docOut.writeVLong(skipOut.getFilePointer());
-      firstDoc = false;
-    }
-
     final int delta = docID - lastDocID;
+    //System.out.println("SepW startDoc: write doc=" + docID + " delta=" + delta);
 
     if (docID < 0 || (df > 0 && delta <= 0)) {
       throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " )");
@@ -190,6 +175,7 @@ public final class SepPostingsWriterImpl
     if ((++df % skipInterval) == 0) {
       // TODO: -- awkward we have to make these two
       // separate calls to skipper
+      //System.out.println("    buffer skip lastDocID=" + lastDocID);
       skipListWriter.setSkipData(lastDocID, storePayloads, lastPayloadLength);
       skipListWriter.bufferSkip(df);
     }
@@ -197,10 +183,20 @@ public final class SepPostingsWriterImpl
     lastDocID = docID;
     docOut.write(delta);
     if (!omitTF) {
+      //System.out.println("    sepw startDoc: write freq=" + termDocFreq);
       freqOut.write(termDocFreq);
     }
   }
 
+  @Override
+  public void flushTermsBlock() throws IOException {
+    //System.out.println("SepW.flushTermsBlock: pendingTermCount=" + pendingTermCount + " bytesUsed=" + indexBytesWriter.getFilePointer());
+    termsOut.writeVLong((int) indexBytesWriter.getFilePointer());
+    indexBytesWriter.writeTo(termsOut);
+    indexBytesWriter.reset();
+    pendingTermCount = 0;
+  }
+
   /** Add a new position & payload */
   @Override
   public void addPosition(int position, BytesRef payload) throws IOException {
@@ -240,20 +236,57 @@ public final class SepPostingsWriterImpl
 
   /** Called when we are done adding docs to this term */
   @Override
-  public void finishTerm(TermStats stats, boolean isIndexTerm) throws IOException {
-
+  public void finishTerm(TermStats stats) throws IOException {
     // TODO: -- wasteful we are counting this in two places?
     assert stats.docFreq > 0;
     assert stats.docFreq == df;
 
-    docIndex.write(termsOut, isIndexTerm);
+    final boolean isFirstTerm = pendingTermCount == 0;  
+    //System.out.println("SepW.finishTerm: isFirstTerm=" + isFirstTerm);
+
+    docIndex.write(indexBytesWriter, isFirstTerm);
+    //System.out.println("  docIndex=" + docIndex);
+
+    if (!omitTF) {
+      freqIndex.write(indexBytesWriter, isFirstTerm);
+      //System.out.println("  freqIndex=" + freqIndex);
+
+      posIndex.write(indexBytesWriter, isFirstTerm);
+      //System.out.println("  posIndex=" + posIndex);
+      if (storePayloads) {
+        if (isFirstTerm) {
+          indexBytesWriter.writeVLong(payloadStart);
+        } else {
+          indexBytesWriter.writeVLong(payloadStart - lastPayloadStart);
+        }
+        lastPayloadStart = payloadStart;
+        //System.out.println("  payloadFP=" + payloadStart);
+      }
+    }
 
     if (df >= skipInterval) {
+      //System.out.println("  skipFP=" + skipStart);
+      final long skipFP = skipOut.getFilePointer();
       skipListWriter.writeSkip(skipOut);
+      //System.out.println("   writeSkip @ " + indexBytesWriter.getFilePointer());
+      if (isFirstTerm) {
+        indexBytesWriter.writeVLong(skipFP);
+      } else {
+        indexBytesWriter.writeVLong(skipFP - lastSkipFP);
+      }
+      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;
     }
 
     lastDocID = 0;
     df = 0;
+    pendingTermCount++;
   }
 
   @Override

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=1061979&r1=1061978&r2=1061979&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 Fri Jan 21 19:38:06 2011
@@ -33,8 +33,8 @@ import org.apache.lucene.index.codecs.Te
 import org.apache.lucene.index.codecs.TermsIndexReaderBase;
 import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
 import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
-import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
-import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
+import org.apache.lucene.index.codecs.BlockTermsWriter;
+import org.apache.lucene.index.codecs.BlockTermsReader;
 import org.apache.lucene.store.Directory;
 
 /** Default codec. 
@@ -66,7 +66,7 @@ public class StandardCodec extends Codec
 
     success = false;
     try {
-      FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, docs, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, docs, BytesRef.getUTF8SortedAsUnicodeComparator());
       success = true;
       return ret;
     } finally {
@@ -103,15 +103,15 @@ public class StandardCodec extends Codec
 
     success = false;
     try {
-      FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
-                                                       state.dir,
-                                                       state.fieldInfos,
-                                                       state.segmentInfo.name,
-                                                       postings,
-                                                       state.readBufferSize,
-                                                       BytesRef.getUTF8SortedAsUnicodeComparator(),
-                                                       TERMS_CACHE_SIZE,
-                                                       state.codecId);
+      FieldsProducer ret = new BlockTermsReader(indexReader,
+                                                state.dir,
+                                                state.fieldInfos,
+                                                state.segmentInfo.name,
+                                                postings,
+                                                state.readBufferSize,
+                                                BytesRef.getUTF8SortedAsUnicodeComparator(),
+                                                TERMS_CACHE_SIZE,
+                                                state.codecId);
       success = true;
       return ret;
     } finally {
@@ -134,7 +134,7 @@ public class StandardCodec extends Codec
   @Override
   public void files(Directory dir, SegmentInfo segmentInfo, String id, Set<String> files) throws IOException {
     StandardPostingsReader.files(dir, segmentInfo, id, files);
-    PrefixCodedTermsReader.files(dir, segmentInfo, id, files);
+    BlockTermsReader.files(dir, segmentInfo, id, files);
     VariableGapTermsIndexReader.files(dir, segmentInfo, id, files);
   }
 
@@ -146,7 +146,7 @@ public class StandardCodec extends Codec
   public static void getStandardExtensions(Set<String> extensions) {
     extensions.add(FREQ_EXTENSION);
     extensions.add(PROX_EXTENSION);
-    PrefixCodedTermsReader.getExtensions(extensions);
+    BlockTermsReader.getExtensions(extensions);
     VariableGapTermsIndexReader.getIndexExtensions(extensions);
   }
 }

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=1061979&r1=1061978&r2=1061979&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 Fri Jan 21 19:38:06 2011
@@ -20,16 +20,18 @@ package org.apache.lucene.index.codecs.s
 import java.io.IOException;
 import java.util.Collection;
 
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.codecs.BlockTermState;
 import org.apache.lucene.index.codecs.PostingsReaderBase;
-import org.apache.lucene.index.codecs.PrefixCodedTermState;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CodecUtil;
@@ -46,9 +48,12 @@ public class StandardPostingsReader exte
   int skipInterval;
   int maxSkipLevels;
 
+  //private String segment;
+
   public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, int readBufferSize, String codecId) throws IOException {
     freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.FREQ_EXTENSION),
                            readBufferSize);
+    //this.segment = segmentInfo.name;
     if (segmentInfo.getHasProx()) {
       boolean success = false;
       try {
@@ -84,11 +89,16 @@ public class StandardPostingsReader exte
   }
 
   // Must keep final because we do non-standard clone
-  private final static class StandardTermState extends PrefixCodedTermState {
+  private final static class StandardTermState extends BlockTermState {
     long freqOffset;
     long proxOffset;
     int skipOffset;
 
+    // Only used by the "primary" TermState -- clones don't
+    // copy this (basically they are "transient"):
+    ByteArrayDataInput bytesReader;
+    byte[] bytes;
+
     public Object clone() {
       StandardTermState other = new StandardTermState();
       other.copyFrom(this);
@@ -101,6 +111,11 @@ public class StandardPostingsReader exte
       freqOffset = other.freqOffset;
       proxOffset = other.proxOffset;
       skipOffset = other.skipOffset;
+
+      // Do not copy bytes, bytesReader (else TermState is
+      // very heavy, ie drags around the entire block's
+      // byte[]).  On seek back, if next() is in fact used
+      // (rare!), they will be re-read from disk.
     }
 
     public String toString() {
@@ -109,7 +124,7 @@ public class StandardPostingsReader exte
   }
 
   @Override
-  public PrefixCodedTermState newTermState() {
+  public BlockTermState newTermState() {
     return new StandardTermState();
   }
 
@@ -126,34 +141,58 @@ public class StandardPostingsReader exte
     }
   }
 
+  /* Reads but does not decode the byte[] blob holding
+     metadata for the current terms block */
   @Override
-  public void readTerm(IndexInput termsIn, FieldInfo fieldInfo, PrefixCodedTermState termState, boolean isIndexTerm)
+  public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
+    final StandardTermState termState = (StandardTermState) _termState;
+
+    final int len = termsIn.readVInt();
+    //System.out.println("SPR.readTermsBlock termsIn.fp=" + termsIn.getFilePointer());
+    if (termState.bytes == null) {
+      termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
+      termState.bytesReader = new ByteArrayDataInput(null);
+    } else if (termState.bytes.length < len) {
+      termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
+    }
+
+    termsIn.readBytes(termState.bytes, 0, len);
+    termState.bytesReader.reset(termState.bytes, 0, len);
+  }
+
+  @Override
+  public void nextTerm(FieldInfo fieldInfo, BlockTermState _termState)
     throws IOException {
-    final StandardTermState docTermState = (StandardTermState) termState;
+    final StandardTermState termState = (StandardTermState) _termState;
+    //System.out.println("StandardR.nextTerm seg=" + segment);
+    final boolean isFirstTerm = termState.termCount == 0;
 
-    if (isIndexTerm) {
-      docTermState.freqOffset = termsIn.readVLong();
+    if (isFirstTerm) {
+      termState.freqOffset = termState.bytesReader.readVLong();
     } else {
-      docTermState.freqOffset += termsIn.readVLong();
+      termState.freqOffset += termState.bytesReader.readVLong();
     }
+    //System.out.println("  freqFP=" + termState.freqOffset);
 
-    if (docTermState.docFreq >= skipInterval) {
-      docTermState.skipOffset = termsIn.readVInt();
+    if (termState.docFreq >= skipInterval) {
+      termState.skipOffset = termState.bytesReader.readVInt();
+      //System.out.println("  skipOffset=" + termState.skipOffset);
     } else {
-      docTermState.skipOffset = 0;
+      // undefined
     }
 
     if (!fieldInfo.omitTermFreqAndPositions) {
-      if (isIndexTerm) {
-        docTermState.proxOffset = termsIn.readVLong();
+      if (isFirstTerm) {
+        termState.proxOffset = termState.bytesReader.readVLong();
       } else {
-        docTermState.proxOffset += termsIn.readVLong();
+        termState.proxOffset += termState.bytesReader.readVLong();
       }
+      //System.out.println("  proxFP=" + termState.proxOffset);
     }
   }
     
   @Override
-  public DocsEnum docs(FieldInfo fieldInfo, PrefixCodedTermState termState, Bits skipDocs, DocsEnum reuse) throws IOException {
+  public DocsEnum docs(FieldInfo fieldInfo, BlockTermState termState, Bits skipDocs, DocsEnum reuse) throws IOException {
     SegmentDocsEnum docsEnum;
     if (reuse == null || !(reuse instanceof SegmentDocsEnum)) {
       docsEnum = new SegmentDocsEnum(freqIn);
@@ -170,7 +209,7 @@ public class StandardPostingsReader exte
   }
 
   @Override
-  public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, PrefixCodedTermState termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
+  public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
     if (fieldInfo.omitTermFreqAndPositions) {
       return null;
     }
@@ -248,8 +287,10 @@ public class StandardPostingsReader exte
       // cases
       freqIn.seek(termState.freqOffset);
       limit = termState.docFreq;
+      assert limit > 0;
       ord = 0;
       doc = 0;
+      //System.out.println("  sde limit=" + limit + " freqFP=" + freqOffset);
 
       skipped = false;
 
@@ -420,6 +461,8 @@ public class StandardPostingsReader exte
       lazyProxPointer = termState.proxOffset;
 
       limit = termState.docFreq;
+      assert limit > 0;
+
       ord = 0;
       doc = 0;
       position = 0;
@@ -430,6 +473,7 @@ public class StandardPostingsReader exte
       freqOffset = termState.freqOffset;
       proxOffset = termState.proxOffset;
       skipOffset = termState.skipOffset;
+      //System.out.println("StandardR.D&PE reset seg=" + segment + " limit=" + limit + " freqFP=" + freqOffset + " proxFP=" + proxOffset);
 
       return this;
     }
@@ -438,6 +482,7 @@ public class StandardPostingsReader exte
     public int nextDoc() throws IOException {
       while(true) {
         if (ord == limit) {
+          //System.out.println("StandardR.D&PE seg=" + segment + " nextDoc return doc=END");
           return doc = NO_MORE_DOCS;
         }
 
@@ -461,6 +506,7 @@ public class StandardPostingsReader exte
 
       position = 0;
 
+      //System.out.println("StandardR.D&PE nextDoc seg=" + segment + " return doc=" + doc);
       return doc;
     }
 
@@ -477,6 +523,8 @@ public class StandardPostingsReader exte
     @Override
     public int advance(int target) throws IOException {
 
+      //System.out.println("StandardR.D&PE advance target=" + target);
+
       // TODO: jump right to next() if target is < X away
       // from where we are now?
 
@@ -622,6 +670,7 @@ public class StandardPostingsReader exte
       freqOffset = termState.freqOffset;
       proxOffset = termState.proxOffset;
       skipOffset = termState.skipOffset;
+      //System.out.println("StandardR.D&PE reset seg=" + segment + " limit=" + limit + " freqFP=" + freqOffset + " proxFP=" + proxOffset);
 
       return this;
     }
@@ -630,6 +679,7 @@ public class StandardPostingsReader exte
     public int nextDoc() throws IOException {
       while(true) {
         if (ord == limit) {
+          //System.out.println("StandardR.D&PE seg=" + segment + " nextDoc return doc=END");
           return doc = NO_MORE_DOCS;
         }
 
@@ -653,6 +703,7 @@ public class StandardPostingsReader exte
 
       position = 0;
 
+      //System.out.println("StandardR.D&PE nextDoc seg=" + segment + " return doc=" + doc);
       return doc;
     }
 
@@ -748,6 +799,7 @@ public class StandardPostingsReader exte
         posPendingCount--;
         position = 0;
         payloadPending = false;
+        //System.out.println("StandardR.D&PE skipPos");
       }
 
       // read next position
@@ -771,6 +823,7 @@ public class StandardPostingsReader exte
 
       assert posPendingCount >= 0: "nextPosition() was called too many times (more than freq() times) posPendingCount=" + posPendingCount;
 
+      //System.out.println("StandardR.D&PE nextPos   return pos=" + position);
       return position;
     }
 

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=1061979&r1=1061978&r2=1061979&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 Fri Jan 21 19:38:06 2011
@@ -22,13 +22,14 @@ package org.apache.lucene.index.codecs.s
 
 import java.io.IOException;
 
-import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.codecs.PostingsWriterBase;
 import org.apache.lucene.index.codecs.TermStats;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CodecUtil;
 
@@ -59,8 +60,15 @@ public final class StandardPostingsWrite
   int lastPayloadLength;
   int lastPosition;
 
+  private int pendingCount;
+
+  //private String segment;
+
+  private RAMOutputStream bytesWriter = new RAMOutputStream();
+
   public StandardPostingsWriter(SegmentWriteState state) throws IOException {
     super();
+    //this.segment = state.segmentName;
     String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.FREQ_EXTENSION);
     freqOut = state.directory.createOutput(fileName);
 
@@ -96,6 +104,7 @@ public final class StandardPostingsWrite
 
   @Override
   public void startTerm() {
+    //System.out.println("StandardW: startTerm seg=" + segment + " pendingCount=" + pendingCount);
     freqStart = freqOut.getFilePointer();
     if (proxOut != null) {
       proxStart = proxOut.getFilePointer();
@@ -109,9 +118,12 @@ public final class StandardPostingsWrite
   // our parent calls setField whenever the field changes
   @Override
   public void setField(FieldInfo fieldInfo) {
+    //System.out.println("SPW: setField");
     this.fieldInfo = fieldInfo;
     omitTermFreqAndPositions = fieldInfo.omitTermFreqAndPositions;
     storePayloads = fieldInfo.storePayloads;
+    //System.out.println("  set init blockFreqStart=" + freqStart);
+    //System.out.println("  set init blockProxStart=" + proxStart);
   }
 
   int lastDocID;
@@ -121,6 +133,7 @@ public final class StandardPostingsWrite
    *  then we just skip consuming positions/payloads. */
   @Override
   public void startDoc(int docID, int termDocFreq) throws IOException {
+    //System.out.println("StandardW:   startDoc seg=" + segment + " docID=" + docID + " tf=" + termDocFreq);
 
     final int delta = docID - lastDocID;
     
@@ -151,6 +164,7 @@ public final class StandardPostingsWrite
   /** Add a new position & payload */
   @Override
   public void addPosition(int position, BytesRef payload) throws IOException {
+    //System.out.println("StandardW:     addPos pos=" + position + " payload=" + (payload == null ? "null" : (payload.length + " bytes")) + " proxFP=" + proxOut.getFilePointer());
     assert !omitTermFreqAndPositions: "omitTermFreqAndPositions is true";
     assert proxOut != null;
 
@@ -185,40 +199,51 @@ public final class StandardPostingsWrite
 
   /** Called when we are done adding docs to this term */
   @Override
-  public void finishTerm(TermStats stats, boolean isIndexTerm) throws IOException {
+  public void finishTerm(TermStats stats) throws IOException {
+    //System.out.println("StandardW.finishTerm seg=" + segment);
     assert stats.docFreq > 0;
 
     // TODO: wasteful we are counting this (counting # docs
     // for this term) in two places?
     assert stats.docFreq == df;
 
-    if (isIndexTerm) {
-      // Write absolute at seek points
-      termsOut.writeVLong(freqStart);
+    final boolean isFirstTerm = pendingCount == 0;
+    //System.out.println("  isFirstTerm=" + isFirstTerm);
+
+    //System.out.println("  freqFP=" + freqStart);
+    if (isFirstTerm) {
+      bytesWriter.writeVLong(freqStart);
     } else {
-      // Write delta between seek points
-      termsOut.writeVLong(freqStart - lastFreqStart);
+      bytesWriter.writeVLong(freqStart-lastFreqStart);
     }
-
     lastFreqStart = freqStart;
 
     if (df >= skipInterval) {
-      termsOut.writeVInt((int) (skipListWriter.writeSkip(freqOut)-freqStart));
+      bytesWriter.writeVInt((int) (skipListWriter.writeSkip(freqOut)-freqStart));
     }
-     
+
     if (!omitTermFreqAndPositions) {
-      if (isIndexTerm) {
-        // Write absolute at seek points
-        termsOut.writeVLong(proxStart);
+      //System.out.println("  proxFP=" + proxStart);
+      if (isFirstTerm) {
+        bytesWriter.writeVLong(proxStart);
       } else {
-        // Write delta between seek points
-        termsOut.writeVLong(proxStart - lastProxStart);
+        bytesWriter.writeVLong(proxStart - lastProxStart);
       }
       lastProxStart = proxStart;
     }
-
+     
     lastDocID = 0;
     df = 0;
+    pendingCount++;
+  }
+
+  @Override
+  public void flushTermsBlock() throws IOException {
+    //System.out.println("SPW.flushBlock pendingCount=" + pendingCount);
+    termsOut.writeVInt((int) bytesWriter.getFilePointer());
+    bytesWriter.writeTo(termsOut);
+    bytesWriter.reset();
+    pendingCount = 0;
   }
 
   @Override

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BooleanQuery.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BooleanQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BooleanQuery.java Fri Jan 21 19:38:06 2011
@@ -169,7 +169,7 @@ public class BooleanQuery extends Query 
 
     public BooleanWeight(IndexSearcher searcher, boolean disableCoord)
       throws IOException {
-      this.similarity = getSimilarity(searcher);
+      this.similarity = searcher.getSimilarity();
       this.disableCoord = disableCoord;
       weights = new ArrayList<Weight>(clauses.size());
       for (int i = 0 ; i < clauses.size(); i++) {
@@ -201,6 +201,9 @@ public class BooleanQuery extends Query 
       return sum ;
     }
 
+    public float coord(int overlap, int maxOverlap) {
+      return similarity.coord(overlap, maxOverlap);
+    }
 
     @Override
     public void normalize(float norm) {
@@ -273,7 +276,7 @@ public class BooleanQuery extends Query 
       sumExpl.setMatch(0 < coord ? Boolean.TRUE : Boolean.FALSE);
       sumExpl.setValue(sum);
       
-      final float coordFactor = disableCoord ? 1.0f : similarity.coord(coord, maxCoord);
+      final float coordFactor = disableCoord ? 1.0f : coord(coord, maxCoord);
       if (coordFactor == 1.0f) {
         return sumExpl;                             // eliminate wrapper
       } else {
@@ -312,7 +315,7 @@ public class BooleanQuery extends Query 
       
       // Check if we can return a BooleanScorer
       if (!scorerContext.scoreDocsInOrder && scorerContext.topScorer && required.size() == 0 && prohibited.size() < 32) {
-        return new BooleanScorer(this, disableCoord, similarity, minNrShouldMatch, optional, prohibited, maxCoord);
+        return new BooleanScorer(this, disableCoord, minNrShouldMatch, optional, prohibited, maxCoord);
       }
       
       if (required.size() == 0 && optional.size() == 0) {
@@ -326,7 +329,7 @@ public class BooleanQuery extends Query 
       }
       
       // Return a BooleanScorer2
-      return new BooleanScorer2(this, disableCoord, similarity, minNrShouldMatch, required, prohibited, optional, maxCoord);
+      return new BooleanScorer2(this, disableCoord, minNrShouldMatch, required, prohibited, optional, maxCoord);
     }
     
     @Override

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BooleanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BooleanScorer.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BooleanScorer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BooleanScorer.java Fri Jan 21 19:38:06 2011
@@ -22,6 +22,7 @@ import java.util.List;
 
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BooleanQuery.BooleanWeight;
 
 /* Description from Doug Cutting (excerpted from
  * LUCENE-1483):
@@ -118,7 +119,7 @@ final class BooleanScorer extends Scorer
     int doc = NO_MORE_DOCS;
     int freq;
     
-    public BucketScorer() { super(null); }
+    public BucketScorer(Weight weight) { super(weight); }
     
     @Override
     public int advance(int target) throws IOException { return NO_MORE_DOCS; }
@@ -197,9 +198,9 @@ final class BooleanScorer extends Scorer
   private Bucket current;
   private int doc = -1;
   
-  BooleanScorer(Weight weight, boolean disableCoord, Similarity similarity, int minNrShouldMatch,
+  BooleanScorer(BooleanWeight weight, boolean disableCoord, int minNrShouldMatch,
       List<Scorer> optionalScorers, List<Scorer> prohibitedScorers, int maxCoord) throws IOException {
-    super(null, weight);   // Similarity not used
+    super(weight);
     this.minNrShouldMatch = minNrShouldMatch;
 
     if (optionalScorers != null && optionalScorers.size() > 0) {
@@ -223,7 +224,7 @@ final class BooleanScorer extends Scorer
 
     coordFactors = new float[optionalScorers.size() + 1];
     for (int i = 0; i < coordFactors.length; i++) {
-      coordFactors[i] = disableCoord ? 1.0f : similarity.coord(i, maxCoord); 
+      coordFactors[i] = disableCoord ? 1.0f : weight.coord(i, maxCoord); 
     }
   }
 
@@ -232,7 +233,7 @@ final class BooleanScorer extends Scorer
   public boolean score(Collector collector, int max, int firstDocID) throws IOException {
     boolean more;
     Bucket tmp;
-    BucketScorer bs = new BucketScorer();
+    BucketScorer bs = new BucketScorer(weight);
     // The internal loop will set the score and doc before calling collect.
     collector.setScorer(bs);
     do {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BooleanScorer2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BooleanScorer2.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BooleanScorer2.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BooleanScorer2.java Fri Jan 21 19:38:06 2011
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BooleanQuery.BooleanWeight;
 
 /* See the description in BooleanScorer.java, comparing
  * BooleanScorer & BooleanScorer2 */
@@ -42,10 +43,10 @@ class BooleanScorer2 extends Scorer {
     int maxCoord = 0; // to be increased for each non prohibited scorer
     int nrMatchers; // to be increased by score() of match counting scorers.
     
-    void init(Similarity sim, boolean disableCoord) { // use after all scorers have been added.
+    void init(boolean disableCoord) { // use after all scorers have been added.
       coordFactors = new float[optionalScorers.size() + requiredScorers.size() + 1];
       for (int i = 0; i < coordFactors.length; i++) {
-        coordFactors[i] = disableCoord ? 1.0f : sim.coord(i, maxCoord);
+        coordFactors[i] = disableCoord ? 1.0f : ((BooleanWeight)weight).coord(i, maxCoord);
       }
     }
   }
@@ -67,8 +68,11 @@ class BooleanScorer2 extends Scorer {
    * prohibited and optional scorers. In no required scorers are added, at least
    * one of the optional scorers will have to match during the search.
    * 
-   * @param similarity
-   *          The similarity to be used.
+   * @param weight
+   *          The BooleanWeight to be used.
+   * @param disableCoord
+   *          If this parameter is true, coordination level matching 
+   *          ({@link Similarity#coord(int, int)}) is not used.
    * @param minNrShouldMatch
    *          The minimum number of optional added scorers that should match
    *          during the search. In case no required scorers are added, at least
@@ -80,9 +84,9 @@ class BooleanScorer2 extends Scorer {
    * @param optional
    *          the list of optional scorers.
    */
-  public BooleanScorer2(Weight weight, boolean disableCoord, Similarity similarity, int minNrShouldMatch,
+  public BooleanScorer2(BooleanWeight weight, boolean disableCoord, int minNrShouldMatch,
       List<Scorer> required, List<Scorer> prohibited, List<Scorer> optional, int maxCoord) throws IOException {
-    super(null, weight);   // Similarity not used
+    super(weight);
     if (minNrShouldMatch < 0) {
       throw new IllegalArgumentException("Minimum number of optional scorers should not be negative");
     }
@@ -94,8 +98,8 @@ class BooleanScorer2 extends Scorer {
     requiredScorers = required;    
     prohibitedScorers = prohibited;
     
-    coordinator.init(similarity, disableCoord);
-    countingSumScorer = makeCountingSumScorer(disableCoord, similarity);
+    coordinator.init(disableCoord);
+    countingSumScorer = makeCountingSumScorer(disableCoord);
   }
   
   /** Count a scorer as a single match. */
@@ -107,7 +111,7 @@ class BooleanScorer2 extends Scorer {
     private float lastDocScore = Float.NaN;
 
     SingleMatchScorer(Scorer scorer) {
-      super(null); // No similarity used.
+      super(scorer.weight);
       this.scorer = scorer;
     }
 
@@ -143,7 +147,7 @@ class BooleanScorer2 extends Scorer {
   private Scorer countingDisjunctionSumScorer(final List<Scorer> scorers,
       int minNrShouldMatch) throws IOException {
     // each scorer from the list counted as a single matcher
-    return new DisjunctionSumScorer(scorers, minNrShouldMatch) {
+    return new DisjunctionSumScorer(weight, scorers, minNrShouldMatch) {
       private int lastScoredDoc = -1;
       // Save the score of lastScoredDoc, so that we don't compute it more than
       // once in score().
@@ -163,11 +167,10 @@ class BooleanScorer2 extends Scorer {
   }
 
   private Scorer countingConjunctionSumScorer(boolean disableCoord,
-                                              Similarity similarity,
                                               List<Scorer> requiredScorers) throws IOException {
     // each scorer from the list counted as a single matcher
     final int requiredNrMatchers = requiredScorers.size();
-    return new ConjunctionScorer(disableCoord ? 1.0f : similarity.coord(requiredScorers.size(), requiredScorers.size()), requiredScorers) {
+    return new ConjunctionScorer(weight, disableCoord ? 1.0f : ((BooleanWeight)weight).coord(requiredScorers.size(), requiredScorers.size()), requiredScorers) {
       private int lastScoredDoc = -1;
       // Save the score of lastScoredDoc, so that we don't compute it more than
       // once in score().
@@ -191,9 +194,8 @@ class BooleanScorer2 extends Scorer {
   }
 
   private Scorer dualConjunctionSumScorer(boolean disableCoord,
-                                          Similarity similarity,
                                           Scorer req1, Scorer req2) throws IOException { // non counting.
-    return new ConjunctionScorer(disableCoord ? 1.0f : similarity.coord(2, 2), req1, req2);
+    return new ConjunctionScorer(weight, disableCoord ? 1.0f : ((BooleanWeight)weight).coord(2, 2), req1, req2);
     // All scorers match, so defaultSimilarity always has 1 as
     // the coordination factor.
     // Therefore the sum of the scores of two scorers
@@ -203,14 +205,13 @@ class BooleanScorer2 extends Scorer {
   /** Returns the scorer to be used for match counting and score summing.
    * Uses requiredScorers, optionalScorers and prohibitedScorers.
    */
-  private Scorer makeCountingSumScorer(boolean disableCoord,
-                                       Similarity similarity) throws IOException { // each scorer counted as a single matcher
+  private Scorer makeCountingSumScorer(boolean disableCoord) throws IOException { // each scorer counted as a single matcher
     return (requiredScorers.size() == 0)
-      ? makeCountingSumScorerNoReq(disableCoord, similarity)
-      : makeCountingSumScorerSomeReq(disableCoord, similarity);
+      ? makeCountingSumScorerNoReq(disableCoord)
+      : makeCountingSumScorerSomeReq(disableCoord);
   }
 
-  private Scorer makeCountingSumScorerNoReq(boolean disableCoord, Similarity similarity) throws IOException { // No required scorers
+  private Scorer makeCountingSumScorerNoReq(boolean disableCoord) throws IOException { // No required scorers
     // minNrShouldMatch optional scorers are required, but at least 1
     int nrOptRequired = (minNrShouldMatch < 1) ? 1 : minNrShouldMatch;
     Scorer requiredCountingSumScorer;
@@ -219,26 +220,25 @@ class BooleanScorer2 extends Scorer {
     else if (optionalScorers.size() == 1)
       requiredCountingSumScorer = new SingleMatchScorer(optionalScorers.get(0));
     else {
-      requiredCountingSumScorer = countingConjunctionSumScorer(disableCoord, similarity, optionalScorers);
+      requiredCountingSumScorer = countingConjunctionSumScorer(disableCoord, optionalScorers);
     }
     return addProhibitedScorers(requiredCountingSumScorer);
   }
 
-  private Scorer makeCountingSumScorerSomeReq(boolean disableCoord, Similarity similarity) throws IOException { // At least one required scorer.
+  private Scorer makeCountingSumScorerSomeReq(boolean disableCoord) throws IOException { // At least one required scorer.
     if (optionalScorers.size() == minNrShouldMatch) { // all optional scorers also required.
       ArrayList<Scorer> allReq = new ArrayList<Scorer>(requiredScorers);
       allReq.addAll(optionalScorers);
-      return addProhibitedScorers(countingConjunctionSumScorer(disableCoord, similarity, allReq));
+      return addProhibitedScorers(countingConjunctionSumScorer(disableCoord, allReq));
     } else { // optionalScorers.size() > minNrShouldMatch, and at least one required scorer
       Scorer requiredCountingSumScorer =
             requiredScorers.size() == 1
             ? new SingleMatchScorer(requiredScorers.get(0))
-            : countingConjunctionSumScorer(disableCoord, similarity, requiredScorers);
+            : countingConjunctionSumScorer(disableCoord, requiredScorers);
       if (minNrShouldMatch > 0) { // use a required disjunction scorer over the optional scorers
         return addProhibitedScorers( 
                       dualConjunctionSumScorer( // non counting
                               disableCoord,
-                              similarity,
                               requiredCountingSumScorer,
                               countingDisjunctionSumScorer(
                                       optionalScorers,
@@ -265,7 +265,7 @@ class BooleanScorer2 extends Scorer {
           : new ReqExclScorer(requiredCountingSumScorer,
                               ((prohibitedScorers.size() == 1)
                                 ? prohibitedScorers.get(0)
-                                : new DisjunctionSumScorer(prohibitedScorers)));
+                                : new DisjunctionSumScorer(weight, prohibitedScorers)));
   }
 
   /** Scores and collects all matching documents.

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BoostAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BoostAttributeImpl.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BoostAttributeImpl.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BoostAttributeImpl.java Fri Jan 21 19:38:06 2011
@@ -37,20 +37,6 @@ public final class BoostAttributeImpl ex
   public void clear() {
     boost = 1.0f;
   }
-
-  @Override
-  public boolean equals(Object other) {
-    if (this == other)
-      return true;
-    if (other instanceof BoostAttributeImpl)
-      return ((BoostAttributeImpl) other).boost == boost;
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return Float.floatToIntBits(boost);
-  }
   
   @Override
   public void copyTo(AttributeImpl target) {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java Fri Jan 21 19:38:06 2011
@@ -29,12 +29,12 @@ class ConjunctionScorer extends Scorer {
   private final float coord;
   private int lastDoc = -1;
 
-  public ConjunctionScorer(float coord, Collection<Scorer> scorers) throws IOException {
-    this(coord, scorers.toArray(new Scorer[scorers.size()]));
+  public ConjunctionScorer(Weight weight, float coord, Collection<Scorer> scorers) throws IOException {
+    this(weight, coord, scorers.toArray(new Scorer[scorers.size()]));
   }
 
-  public ConjunctionScorer(float coord, Scorer... scorers) throws IOException {
-    super(null);
+  public ConjunctionScorer(Weight weight, float coord, Scorer... scorers) throws IOException {
+    super(weight);
     this.scorers = scorers;
     this.coord = coord;
     

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java Fri Jan 21 19:38:06 2011
@@ -97,12 +97,10 @@ public class ConstantScoreQuery extends 
 
   protected class ConstantWeight extends Weight {
     private final Weight innerWeight;
-    private final Similarity similarity;
     private float queryNorm;
     private float queryWeight;
     
     public ConstantWeight(IndexSearcher searcher) throws IOException {
-      this.similarity = getSimilarity(searcher);
       this.innerWeight = (query == null) ? null : query.createWeight(searcher);
     }
 
@@ -148,7 +146,7 @@ public class ConstantScoreQuery extends 
       }
       if (disi == null)
         return null;
-      return new ConstantScorer(similarity, disi, this);
+      return new ConstantScorer(disi, this);
     }
     
     @Override
@@ -181,8 +179,8 @@ public class ConstantScoreQuery extends 
     final DocIdSetIterator docIdSetIterator;
     final float theScore;
 
-    public ConstantScorer(Similarity similarity, DocIdSetIterator docIdSetIterator, Weight w) throws IOException {
-      super(similarity,w);
+    public ConstantScorer(DocIdSetIterator docIdSetIterator, Weight w) throws IOException {
+      super(w);
       theScore = w.getValue();
       this.docIdSetIterator = docIdSetIterator;
     }
@@ -212,8 +210,7 @@ public class ConstantScoreQuery extends 
         @Override
         public void setScorer(Scorer scorer) throws IOException {
           // we must wrap again here, but using the scorer passed in as parameter:
-          collector.setScorer(new ConstantScorer(ConstantScorer.this.getSimilarity(),
-            scorer, ConstantScorer.this.weight));
+          collector.setScorer(new ConstantScorer(scorer, ConstantScorer.this.weight));
         }
         
         @Override

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java Fri Jan 21 19:38:06 2011
@@ -95,29 +95,26 @@ public class DisjunctionMaxQuery extends
    * change suddenly in the next release.</p>
    */
   protected class DisjunctionMaxWeight extends Weight {
-    /** The Similarity implementation. */
-    protected Similarity similarity;
 
     /** The Weights for our subqueries, in 1-1 correspondence with disjuncts */
     protected ArrayList<Weight> weights = new ArrayList<Weight>();  // The Weight's for our subqueries, in 1-1 correspondence with disjuncts
 
-    /* Construct the Weight for this Query searched by searcher.  Recursively construct subquery weights. */
+    /** Construct the Weight for this Query searched by searcher.  Recursively construct subquery weights. */
     public DisjunctionMaxWeight(IndexSearcher searcher) throws IOException {
-      this.similarity = searcher.getSimilarity();
       for (Query disjunctQuery : disjuncts) {
         weights.add(disjunctQuery.createWeight(searcher));
       }
     }
 
-    /* Return our associated DisjunctionMaxQuery */
+    /** Return our associated DisjunctionMaxQuery */
     @Override
     public Query getQuery() { return DisjunctionMaxQuery.this; }
 
-    /* Return our boost */
+    /** Return our boost */
     @Override
     public float getValue() { return getBoost(); }
 
-    /* Compute the sub of squared weights of us applied to our subqueries.  Used for normalization. */
+    /** Compute the sub of squared weights of us applied to our subqueries.  Used for normalization. */
     @Override
     public float sumOfSquaredWeights() throws IOException {
       float max = 0.0f, sum = 0.0f;
@@ -131,7 +128,7 @@ public class DisjunctionMaxQuery extends
       return (((sum - max) * tieBreakerMultiplier * tieBreakerMultiplier) + max) * boost * boost;
     }
 
-    /* Apply the computed normalization factor to our subqueries */
+    /** Apply the computed normalization factor to our subqueries */
     @Override
     public void normalize(float norm) {
       norm *= getBoost();  // Incorporate our boost
@@ -140,7 +137,7 @@ public class DisjunctionMaxQuery extends
       }
     }
 
-    /* Create the scorer used to score our associated DisjunctionMaxQuery */
+    /** Create the scorer used to score our associated DisjunctionMaxQuery */
     @Override
     public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
       Scorer[] scorers = new Scorer[weights.size()];
@@ -152,11 +149,11 @@ public class DisjunctionMaxQuery extends
         }
       }
       if (idx == 0) return null; // all scorers did not have documents
-      DisjunctionMaxScorer result = new DisjunctionMaxScorer(tieBreakerMultiplier, similarity, scorers, idx);
+      DisjunctionMaxScorer result = new DisjunctionMaxScorer(this, tieBreakerMultiplier, scorers, idx);
       return result;
     }
 
-    /* Explain the score we computed for doc */
+    /** Explain the score we computed for doc */
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
       if (disjuncts.size() == 1) return weights.get(0).explain(context,doc);
@@ -178,7 +175,7 @@ public class DisjunctionMaxQuery extends
     
   }  // end of DisjunctionMaxWeight inner class
 
-  /* Create the Weight used to score us */
+  /** Create the Weight used to score us */
   @Override
   public Weight createWeight(IndexSearcher searcher) throws IOException {
     return new DisjunctionMaxWeight(searcher);

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java Fri Jan 21 19:38:06 2011
@@ -40,22 +40,20 @@ class DisjunctionMaxScorer extends Score
   /**
    * Creates a new instance of DisjunctionMaxScorer
    * 
+   * @param weight
+   *          The Weight to be used.
    * @param tieBreakerMultiplier
    *          Multiplier applied to non-maximum-scoring subqueries for a
    *          document as they are summed into the result.
-   * @param similarity
-   *          -- not used since our definition involves neither coord nor terms
-   *          directly
    * @param subScorers
    *          The sub scorers this Scorer should iterate on
    * @param numScorers
    *          The actual number of scorers to iterate on. Note that the array's
    *          length may be larger than the actual number of scorers.
    */
-  public DisjunctionMaxScorer(float tieBreakerMultiplier,
-      Similarity similarity, Scorer[] subScorers, int numScorers) throws IOException {
-    super(similarity);
-
+  public DisjunctionMaxScorer(Weight weight, float tieBreakerMultiplier,
+      Scorer[] subScorers, int numScorers) throws IOException {
+    super(weight);
     this.tieBreakerMultiplier = tieBreakerMultiplier;
     // The passed subScorers array includes only scorers which have documents
     // (DisjunctionMaxQuery takes care of that), and their nextDoc() was already

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DisjunctionSumScorer.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DisjunctionSumScorer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DisjunctionSumScorer.java Fri Jan 21 19:38:06 2011
@@ -58,6 +58,7 @@ class DisjunctionSumScorer extends Score
   private float currentScore = Float.NaN;
   
   /** Construct a <code>DisjunctionScorer</code>.
+   * @param weight The weight to be used.
    * @param subScorers A collection of at least two subscorers.
    * @param minimumNrMatchers The positive minimum number of subscorers that should
    * match to match this query.
@@ -67,8 +68,8 @@ class DisjunctionSumScorer extends Score
    * <br>When minimumNrMatchers equals the number of subScorers,
    * it more efficient to use <code>ConjunctionScorer</code>.
    */
-  public DisjunctionSumScorer( List<Scorer> subScorers, int minimumNrMatchers) throws IOException {
-    super(null);
+  public DisjunctionSumScorer(Weight weight, List<Scorer> subScorers, int minimumNrMatchers) throws IOException {
+    super(weight);
     
     nrScorers = subScorers.size();
 
@@ -88,8 +89,8 @@ class DisjunctionSumScorer extends Score
   /** Construct a <code>DisjunctionScorer</code>, using one as the minimum number
    * of matching subscorers.
    */
-  public DisjunctionSumScorer(List<Scorer> subScorers) throws IOException {
-    this(subScorers, 1);
+  public DisjunctionSumScorer(Weight weight, List<Scorer> subScorers) throws IOException {
+    this(weight, subScorers, 1);
   }
 
   /** Called the first time nextDoc() or advance() is called to

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ExactPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ExactPhraseScorer.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ExactPhraseScorer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ExactPhraseScorer.java Fri Jan 21 19:38:06 2011
@@ -60,9 +60,12 @@ final class ExactPhraseScorer extends Sc
   private int docID = -1;
   private int freq;
 
+  private final Similarity similarity;
+  
   ExactPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
                     Similarity similarity, byte[] norms) throws IOException {
-    super(similarity, weight);
+    super(weight);
+    this.similarity = similarity;
     this.norms = norms;
     this.value = weight.getValue();
 
@@ -87,7 +90,7 @@ final class ExactPhraseScorer extends Sc
     }
 
     for (int i = 0; i < SCORE_CACHE_SIZE; i++) {
-      scoreCache[i] = getSimilarity().tf((float) i) * value;
+      scoreCache[i] = similarity.tf((float) i) * value;
     }
   }
 
@@ -207,9 +210,9 @@ final class ExactPhraseScorer extends Sc
     if (freq < SCORE_CACHE_SIZE) {
       raw = scoreCache[freq];
     } else {
-      raw = getSimilarity().tf((float) freq) * value;
+      raw = similarity.tf((float) freq) * value;
     }
-    return norms == null ? raw : raw * getSimilarity().decodeNormValue(norms[docID]); // normalize
+    return norms == null ? raw : raw * similarity.decodeNormValue(norms[docID]); // normalize
   }
 
   private int phraseFreq() throws IOException {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldComparator.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldComparator.java Fri Jan 21 19:38:06 2011
@@ -82,7 +82,7 @@ import org.apache.lucene.util.packed.Pac
  *       priority queue.  The {@link FieldValueHitQueue}
  *       calls this method when a new hit is competitive.
  *
- *  <li> {@link #setNextReader(AtomicReaderContext)} Invoked
+ *  <li> {@link #setNextReader(IndexReader.AtomicReaderContext)} Invoked
  *       when the search is switching to the next segment.
  *       You may need to update internal state of the
  *       comparator, for example retrieving new values from

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Filter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Filter.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Filter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Filter.java Fri Jan 21 19:38:06 2011
@@ -19,6 +19,7 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
+import org.apache.lucene.index.IndexReader; // javadocs
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.util.DocIdBitSet;
 

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FilteredQuery.java Fri Jan 21 19:38:06 2011
@@ -62,7 +62,6 @@ extends Query {
   @Override
   public Weight createWeight(final IndexSearcher searcher) throws IOException {
     final Weight weight = query.createWeight (searcher);
-    final Similarity similarity = query.getSimilarity(searcher);
     return new Weight() {
       private float value;
         
@@ -127,7 +126,7 @@ extends Query {
           return null;
         }
 
-        return new Scorer(similarity, this) {
+        return new Scorer(this) {
 
           private int doc = -1;
           

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java Fri Jan 21 19:38:06 2011
@@ -122,12 +122,12 @@ public abstract class FilteredTermsEnum 
   }
     
   @Override
-  public int docFreq() {
+  public int docFreq() throws IOException {
     return tenum.docFreq();
   }
 
   @Override
-  public long totalTermFreq() {
+  public long totalTermFreq() throws IOException {
     return tenum.totalTermFreq();
   }
 
@@ -166,7 +166,7 @@ public abstract class FilteredTermsEnum 
    * @throws UnsupportedOperationException
    */
   @Override
-  public SeekStatus seek(BytesRef term, TermState state) throws IOException {
+  public void seek(BytesRef term, TermState state) throws IOException {
     throw new UnsupportedOperationException(getClass().getName()+" does not support seeking");
   }
   

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Fri Jan 21 19:38:06 2011
@@ -241,12 +241,12 @@ public final class FuzzyTermsEnum extend
   
   // proxy all other enum calls to the actual enum
   @Override
-  public int docFreq() {
+  public int docFreq() throws IOException {
     return actualEnum.docFreq();
   }
 
   @Override
-  public long totalTermFreq() {
+  public long totalTermFreq() throws IOException {
     return actualEnum.totalTermFreq();
   }
   
@@ -261,8 +261,8 @@ public final class FuzzyTermsEnum extend
     return actualEnum.docsAndPositions(skipDocs, reuse);
   }
   
-  public SeekStatus seek(BytesRef term, TermState state) throws IOException {
-    return actualEnum.seek(term, state);
+  public void seek(BytesRef term, TermState state) throws IOException {
+    actualEnum.seek(term, state);
   }
   
   @Override

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MatchAllDocsQuery.java Fri Jan 21 19:38:06 2011
@@ -51,10 +51,12 @@ public class MatchAllDocsQuery extends Q
     private int doc = -1;
     private final int maxDoc;
     private final Bits delDocs;
+    private final Similarity similarity;
     
     MatchAllScorer(IndexReader reader, Similarity similarity, Weight w,
         byte[] norms) throws IOException {
-      super(similarity,w);
+      super(w);
+      this.similarity = similarity;
       delDocs = reader.getDeletedDocs();
       score = w.getValue();
       maxDoc = reader.maxDoc();
@@ -80,7 +82,7 @@ public class MatchAllDocsQuery extends Q
     
     @Override
     public float score() {
-      return norms == null ? score : score * getSimilarity().decodeNormValue(norms[docID()]);
+      return norms == null ? score : score * similarity.decodeNormValue(norms[docID()]);
     }
 
     @Override

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttributeImpl.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttributeImpl.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MaxNonCompetitiveBoostAttributeImpl.java Fri Jan 21 19:38:06 2011
@@ -48,25 +48,6 @@ public final class MaxNonCompetitiveBoos
     maxNonCompetitiveBoost = Float.NEGATIVE_INFINITY;
     competitiveTerm = null;
   }
-
-  @Override
-  public boolean equals(Object other) {
-    if (this == other)
-      return true;
-    if (other instanceof MaxNonCompetitiveBoostAttributeImpl) {
-      final MaxNonCompetitiveBoostAttributeImpl o = (MaxNonCompetitiveBoostAttributeImpl) other;
-      return (o.maxNonCompetitiveBoost == maxNonCompetitiveBoost)
-        && (o.competitiveTerm == null ? competitiveTerm == null : o.competitiveTerm.equals(competitiveTerm));
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    int hash = Float.floatToIntBits(maxNonCompetitiveBoost);
-    if (competitiveTerm != null) hash = 31 * hash + competitiveTerm.hashCode();
-    return hash;
-  }
   
   @Override
   public void copyTo(AttributeImpl target) {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java Fri Jan 21 19:38:06 2011
@@ -137,7 +137,7 @@ public class MultiPhraseQuery extends Qu
 
     public MultiPhraseWeight(IndexSearcher searcher)
       throws IOException {
-      this.similarity = getSimilarity(searcher);
+      this.similarity = searcher.getSimilarity();
 
       // compute idf
       final int maxDoc = searcher.maxDoc();

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseQuery.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseQuery.java Fri Jan 21 19:38:06 2011
@@ -146,7 +146,7 @@ public class PhraseQuery extends Query {
 
     public PhraseWeight(IndexSearcher searcher)
       throws IOException {
-      this.similarity = getSimilarity(searcher);
+      this.similarity = searcher.getSimilarity();
 
       idfExp = similarity.idfExplain(terms, searcher);
       idf = idfExp.getIdf();

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseScorer.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseScorer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseScorer.java Fri Jan 21 19:38:06 2011
@@ -40,9 +40,12 @@ abstract class PhraseScorer extends Scor
 
   private float freq; //phrase frequency in current doc as computed by phraseFreq().
 
+  protected final Similarity similarity;
+
   PhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
       Similarity similarity, byte[] norms) {
-    super(similarity, weight);
+    super(weight);
+    this.similarity = similarity;
     this.norms = norms;
     this.value = weight.getValue();
 
@@ -105,8 +108,8 @@ abstract class PhraseScorer extends Scor
   @Override
   public float score() throws IOException {
     //System.out.println("scoring " + first.doc);
-    float raw = getSimilarity().tf(freq) * value; // raw score
-    return norms == null ? raw : raw * getSimilarity().decodeNormValue(norms[first.doc]); // normalize
+    float raw = similarity.tf(freq) * value; // raw score
+    return norms == null ? raw : raw * similarity.decodeNormValue(norms[first.doc]); // normalize
   }
 
   @Override

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Query.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Query.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Query.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Query.java Fri Jan 21 19:38:06 2011
@@ -98,7 +98,7 @@ public abstract class Query implements j
     Query query = searcher.rewrite(this);
     Weight weight = query.createWeight(searcher);
     float sum = weight.sumOfSquaredWeights();
-    float norm = getSimilarity(searcher).queryNorm(sum);
+    float norm = searcher.getSimilarity().queryNorm(sum);
     if (Float.isInfinite(norm) || Float.isNaN(norm))
       norm = 1.0f;
     weight.normalize(norm);
@@ -124,15 +124,6 @@ public abstract class Query implements j
     // needs to be implemented by query subclasses
     throw new UnsupportedOperationException();
   }
-  
-
-  /** Expert: Returns the Similarity implementation to be used for this query.
-   * Subclasses may override this method to specify their own Similarity
-   * implementation, perhaps one that delegates through that of the Searcher.
-   * By default the Searcher's Similarity implementation is returned.*/
-  public Similarity getSimilarity(IndexSearcher searcher) {
-    return searcher.getSimilarity();
-  }
 
   /** Returns a clone of this query. */
   @Override

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ReqExclScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ReqExclScorer.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ReqExclScorer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ReqExclScorer.java Fri Jan 21 19:38:06 2011
@@ -36,7 +36,7 @@ class ReqExclScorer extends Scorer {
    * @param exclDisi indicates exclusion.
    */
   public ReqExclScorer(Scorer reqScorer, DocIdSetIterator exclDisi) {
-    super(null); // No similarity used.
+    super(reqScorer.weight);
     this.reqScorer = reqScorer;
     this.exclDisi = exclDisi;
   }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ReqOptSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ReqOptSumScorer.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ReqOptSumScorer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ReqOptSumScorer.java Fri Jan 21 19:38:06 2011
@@ -38,7 +38,7 @@ class ReqOptSumScorer extends Scorer {
       Scorer reqScorer,
       Scorer optScorer)
   {
-    super(null); // No similarity used.
+    super(reqScorer.weight);
     this.reqScorer = reqScorer;
     this.optScorer = optScorer;
   }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java Fri Jan 21 19:38:06 2011
@@ -38,7 +38,7 @@ public class ScoreCachingWrappingScorer 
   
   /** Creates a new instance by wrapping the given scorer. */
   public ScoreCachingWrappingScorer(Scorer scorer) {
-    super(scorer.getSimilarity());
+    super(scorer.weight);
     this.scorer = scorer;
   }
 
@@ -46,11 +46,6 @@ public class ScoreCachingWrappingScorer 
   public boolean score(Collector collector, int max, int firstDocID) throws IOException {
     return scorer.score(collector, max, firstDocID);
   }
-
-  @Override
-  public Similarity getSimilarity() {
-    return scorer.getSimilarity();
-  }
   
   @Override
   public float score() throws IOException {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Scorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Scorer.java?rev=1061979&r1=1061978&r2=1061979&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Scorer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Scorer.java Fri Jan 21 19:38:06 2011
@@ -40,31 +40,16 @@ import org.apache.lucene.search.BooleanC
  * with these scores.
  */
 public abstract class Scorer extends DocIdSetIterator {
-  private final Similarity similarity;
   protected final Weight weight;
 
-  /** Constructs a Scorer.
-   * @param similarity The <code>Similarity</code> implementation used by this scorer.
-   */
-  protected Scorer(Similarity similarity) {
-    this(similarity, null);
-  }
-  
   /**
    * Constructs a Scorer
-   * @param similarity The <code>Similarity</code> implementation used by this scorer.
-   * @param weight The scorers <code>Weight</code>
+   * @param weight The scorers <code>Weight</code>.
    */
-  protected Scorer(Similarity similarity, Weight weight) {
-    this.similarity = similarity;
+  protected Scorer(Weight weight) {
     this.weight = weight;
   }
 
-  /** Returns the Similarity implementation used by this scorer. */
-  public Similarity getSimilarity() {
-    return this.similarity;
-  }
-
   /** Scores and collects all matching documents.
    * @param collector The collector to which all matching documents are passed.
    */
@@ -172,7 +157,7 @@ public abstract class Scorer extends Doc
    * <p>
    * Note: this method will throw {@link UnsupportedOperationException} if no
    * associated {@link Weight} instance is provided to
-   * {@link #Scorer(Similarity, Weight)}
+   * {@link #Scorer(Weight)}
    * </p>
    * 
    * @lucene.experimental