You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2012/11/23 13:01:26 UTC

svn commit: r1412849 [3/13] - in /lucene/dev/branches/LUCENE-2878: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/idea/solr/contrib/dataimporthandler/ dev-tools/maven/ dev-tools/maven/solr/contrib/dataimporthandler/ dev-tool...

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocValues.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocValues.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocValues.java Fri Nov 23 12:00:32 2012
@@ -33,8 +33,8 @@ import org.apache.lucene.document.Packed
 import org.apache.lucene.document.ShortDocValuesField; // javadocs
 import org.apache.lucene.document.SortedBytesDocValuesField; // javadocs
 import org.apache.lucene.document.StraightBytesDocValuesField; // javadocs
-import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CloseableThreadLocal;
 import org.apache.lucene.util.packed.PackedInts;
 
 /**
@@ -95,7 +95,6 @@ public abstract class DocValues implemen
 
   private volatile SourceCache cache = new SourceCache.DirectSourceCache();
   private final Object cacheLock = new Object();
-  
   /** Sole constructor. (For invocation by subclass 
    *  constructors, typically implicit.) */
   protected DocValues() {
@@ -112,12 +111,12 @@ public abstract class DocValues implemen
    * @see #getSource()
    * @see #setCache(SourceCache)
    */
-  public abstract Source load() throws IOException;
+  protected abstract Source loadSource() throws IOException;
 
   /**
    * Returns a {@link Source} instance through the current {@link SourceCache}.
    * Iff no {@link Source} has been loaded into the cache so far the source will
-   * be loaded through {@link #load()} and passed to the {@link SourceCache}.
+   * be loaded through {@link #loadSource()} and passed to the {@link SourceCache}.
    * The caller of this method should not close the obtained {@link Source}
    * instance unless it is not needed for the rest of its life time.
    * <p>
@@ -129,12 +128,30 @@ public abstract class DocValues implemen
   public Source getSource() throws IOException {
     return cache.load(this);
   }
+  
+  /**
+   * Returns a disk resident {@link Source} instance through the current
+   * {@link SourceCache}. Direct Sources are cached per thread in the
+   * {@link SourceCache}. The obtained instance should not be shared with other
+   * threads.
+   */
+  public Source getDirectSource() throws IOException {
+    return this.cache.loadDirect(this);
+  }
+  
 
   /**
-   * Returns a disk resident {@link Source} instance. Direct Sources are not
-   * cached in the {@link SourceCache} and should not be shared between threads.
+   * Loads a new {@link Source direct source} instance from this {@link DocValues} field
+   * instance. Source instances returned from this method are not cached. It is
+   * the callers responsibility to maintain the instance and release its
+   * resources once the source is not needed anymore.
+   * <p>
+   * For managed {@link Source direct source} instances see {@link #getDirectSource()}.
+   * 
+   * @see #getDirectSource()
+   * @see #setCache(SourceCache)
    */
-  public abstract Source getDirectSource() throws IOException;
+  protected abstract Source loadDirectSource() throws IOException;
 
   /**
    * Returns the {@link Type} of this {@link DocValues} instance
@@ -163,10 +180,10 @@ public abstract class DocValues implemen
 
   /**
    * Sets the {@link SourceCache} used by this {@link DocValues} instance. This
-   * method should be called before {@link #load()} is called. All {@link Source} instances in the currently used cache will be closed
+   * method should be called before {@link #loadSource()} is called. All {@link Source} instances in the currently used cache will be closed
    * before the new cache is installed.
    * <p>
-   * Note: All instances previously obtained from {@link #load()} will be lost.
+   * Note: All instances previously obtained from {@link #loadSource()} will be lost.
    * 
    * @throws IllegalArgumentException
    *           if the given cache is <code>null</code>
@@ -181,6 +198,14 @@ public abstract class DocValues implemen
       toClose.close(this);
     }
   }
+  /**
+   * Returns the currently used cache instance;
+   * @see #setCache(SourceCache)
+   */
+  // for tests
+  SourceCache getCache() {
+    return cache;
+  }
 
   /**
    * Source of per document values like long, double or {@link BytesRef}
@@ -687,9 +712,9 @@ public abstract class DocValues implemen
   /**
    * Abstract base class for {@link DocValues} {@link Source} cache.
    * <p>
-   * {@link Source} instances loaded via {@link DocValues#load()} are entirely memory resident
+   * {@link Source} instances loaded via {@link DocValues#loadSource()} are entirely memory resident
    * and need to be maintained by the caller. Each call to
-   * {@link DocValues#load()} will cause an entire reload of
+   * {@link DocValues#loadSource()} will cause an entire reload of
    * the underlying data. Source instances obtained from
    * {@link DocValues#getSource()} and {@link DocValues#getSource()}
    * respectively are maintained by a {@link SourceCache} that is closed (
@@ -721,6 +746,15 @@ public abstract class DocValues implemen
      * This method will not return <code>null</code>
      */
     public abstract Source load(DocValues values) throws IOException;
+    
+    /**
+     * Atomically loads a {@link Source direct source} into the per-thread cache from the given
+     * {@link DocValues} and returns it iff no other {@link Source direct source} has already
+     * been cached. Otherwise the cached source is returned.
+     * <p>
+     * This method will not return <code>null</code>
+     */
+    public abstract Source loadDirect(DocValues values) throws IOException;
 
     /**
      * Atomically invalidates the cached {@link Source} 
@@ -744,20 +778,34 @@ public abstract class DocValues implemen
      */
     public static final class DirectSourceCache extends SourceCache {
       private Source ref;
-
+      private final CloseableThreadLocal<Source> directSourceCache = new CloseableThreadLocal<Source>();
+      
       /** Sole constructor. */
       public DirectSourceCache() {
       }
 
       public synchronized Source load(DocValues values) throws IOException {
         if (ref == null) {
-          ref = values.load();
+          ref = values.loadSource();
         }
         return ref;
       }
 
       public synchronized void invalidate(DocValues values) {
         ref = null;
+        directSourceCache.close();
+      }
+
+      @Override
+      public synchronized Source loadDirect(DocValues values) throws IOException {
+        final Source source = directSourceCache.get();
+        if (source == null) {
+          final Source loadDirectSource = values.loadDirectSource();
+          directSourceCache.set(loadDirectSource);
+          return loadDirectSource;
+        } else {
+          return source;
+        }
       }
     }
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java Fri Nov 23 12:00:32 2012
@@ -46,6 +46,7 @@ final class DocumentsWriterFlushControl 
   private long activeBytes = 0;
   private long flushBytes = 0;
   private volatile int numPending = 0;
+  private int numDocsSinceStalled = 0; // only with assert
   final AtomicBoolean flushDeletes = new AtomicBoolean(false);
   private boolean fullFlush = false;
   private final Queue<DocumentsWriterPerThread> flushQueue = new LinkedList<DocumentsWriterPerThread>();
@@ -104,8 +105,8 @@ final class DocumentsWriterFlushControl 
       // 2 * ramBufferBytes -> before we stall we need to cross the 2xRAM Buffer border this is still a valid limit
       // (numPending + numFlushingDWPT() + numBlockedFlushes()) * peakDelta) -> those are the total number of DWPT that are not active but not yet fully fluhsed
       // all of them could theoretically be taken out of the loop once they crossed the RAM buffer and the last document was the peak delta
-      // (perThreadPool.getActiveThreadState() * peakDelta) -> at any given time there could be n threads in flight that crossed the stall control before we reached the limit and each of them could hold a peak document
-      final long expected = (2 * (ramBufferBytes)) + ((numPending + numFlushingDWPT() + numBlockedFlushes()) * peakDelta) + (perThreadPool.getActiveThreadState() * peakDelta);
+      // (numDocsSinceStalled * peakDelta) -> at any given time there could be n threads in flight that crossed the stall control before we reached the limit and each of them could hold a peak document
+      final long expected = (2 * (ramBufferBytes)) + ((numPending + numFlushingDWPT() + numBlockedFlushes()) * peakDelta) + (numDocsSinceStalled * peakDelta);
       // the expected ram consumption is an upper bound at this point and not really the expected consumption
       if (peakDelta < (ramBufferBytes >> 1)) {
         /*
@@ -183,10 +184,26 @@ final class DocumentsWriterFlushControl 
       }
       return flushingDWPT;
     } finally {
-      updateStallState();
-      assert assertMemory();
+      boolean stalled = updateStallState();
+      assert assertNumDocsSinceStalled(stalled) && assertMemory();
     }
   }
+  
+  private boolean assertNumDocsSinceStalled(boolean stalled) {
+    /*
+     *  updates the number of documents "finished" while we are in a stalled state.
+     *  this is important for asserting memory upper bounds since it corresponds 
+     *  to the number of threads that are in-flight and crossed the stall control
+     *  check before we actually stalled.
+     *  see #assertMemory()
+     */
+    if (stalled) { 
+      numDocsSinceStalled++;
+    } else {
+      numDocsSinceStalled = 0;
+    }
+    return true;
+  }
 
   synchronized void doAfterFlush(DocumentsWriterPerThread dwpt) {
     assert flushingWriters.containsKey(dwpt);
@@ -204,7 +221,7 @@ final class DocumentsWriterFlushControl 
     }
   }
   
-  private final void updateStallState() {
+  private final boolean updateStallState() {
     
     assert Thread.holdsLock(this);
     final long limit = stallLimitBytes();
@@ -219,6 +236,7 @@ final class DocumentsWriterFlushControl 
                           (activeBytes < limit) &&
                           !closed;
     stallControl.updateStalled(stall);
+    return stall;
   }
   
   public synchronized void waitForFlush() {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Fri Nov 23 12:00:32 2012
@@ -36,6 +36,7 @@ import org.apache.lucene.util.ByteBlockP
 import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.InfoStream;
+import org.apache.lucene.util.IntBlockPool;
 import org.apache.lucene.util.MutableBits;
 import org.apache.lucene.util.RamUsageEstimator;
 
@@ -186,6 +187,7 @@ class DocumentsWriterPerThread {
   DeleteSlice deleteSlice;
   private final NumberFormat nf = NumberFormat.getInstance(Locale.ROOT);
   final Allocator byteBlockAllocator;
+  final IntBlockPool.Allocator intBlockAllocator;
 
   
   public DocumentsWriterPerThread(Directory directory, DocumentsWriter parent,
@@ -201,9 +203,12 @@ class DocumentsWriterPerThread {
     this.docState.similarity = parent.indexWriter.getConfig().getSimilarity();
     bytesUsed = Counter.newCounter();
     byteBlockAllocator = new DirectTrackingAllocator(bytesUsed);
-    consumer = indexingChain.getChain(this);
     pendingDeletes = new BufferedDeletes();
+    intBlockAllocator = new IntBlockAllocator(bytesUsed);
     initialize();
+    // this should be the last call in the ctor 
+    // it really sucks that we need to pull this within the ctor and pass this ref to the chain!
+    consumer = indexingChain.getChain(this);
   }
   
   public DocumentsWriterPerThread(DocumentsWriterPerThread other, FieldInfos.Builder fieldInfos) {
@@ -577,6 +582,9 @@ class DocumentsWriterPerThread {
           infoStream.message("DWPT", "flush: write " + delCount + " deletes gen=" + flushedSegment.segmentInfo.getDelGen());
         }
 
+        // TODO: we should prune the segment if it's 100%
+        // deleted... but merge will also catch it.
+
         // TODO: in the NRT case it'd be better to hand
         // this del vector over to the
         // shortly-to-be-opened SegmentReader and let it
@@ -619,23 +627,28 @@ class DocumentsWriterPerThread {
    * getTerms/getTermsIndex requires <= 32768 */
   final static int MAX_TERM_LENGTH_UTF8 = BYTE_BLOCK_SIZE-2;
 
-  /* Initial chunks size of the shared int[] blocks used to
-     store postings data */
-  final static int INT_BLOCK_SHIFT = 13;
-  final static int INT_BLOCK_SIZE = 1 << INT_BLOCK_SHIFT;
-  final static int INT_BLOCK_MASK = INT_BLOCK_SIZE - 1;
-
-  /* Allocate another int[] from the shared pool */
-  int[] getIntBlock() {
-    int[] b = new int[INT_BLOCK_SIZE];
-    bytesUsed.addAndGet(INT_BLOCK_SIZE*RamUsageEstimator.NUM_BYTES_INT);
-    return b;
-  }
-  
-  void recycleIntBlocks(int[][] blocks, int offset, int length) {
-    bytesUsed.addAndGet(-(length *(INT_BLOCK_SIZE*RamUsageEstimator.NUM_BYTES_INT)));
-  }
 
+  private static class IntBlockAllocator extends IntBlockPool.Allocator {
+    private final Counter bytesUsed;
+    
+    public IntBlockAllocator(Counter bytesUsed) {
+      super(IntBlockPool.INT_BLOCK_SIZE);
+      this.bytesUsed = bytesUsed;
+    }
+    
+    /* Allocate another int[] from the shared pool */
+    public int[] getIntBlock() {
+      int[] b = new int[IntBlockPool.INT_BLOCK_SIZE];
+      bytesUsed.addAndGet(IntBlockPool.INT_BLOCK_SIZE
+          * RamUsageEstimator.NUM_BYTES_INT);
+      return b;
+    }
+    
+    public void recycleIntBlocks(int[][] blocks, int offset, int length) {
+      bytesUsed.addAndGet(-(length * (IntBlockPool.INT_BLOCK_SIZE * RamUsageEstimator.NUM_BYTES_INT)));
+    }
+    
+  }
   PerDocWriteState newPerDocWriteState(String segmentSuffix) {
     assert segmentInfo != null;
     return new PerDocWriteState(infoStream, directory, segmentInfo, bytesUsed, segmentSuffix, IOContext.DEFAULT);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java Fri Nov 23 12:00:32 2012
@@ -376,6 +376,7 @@ final class FreqProxTermsWriterPerField 
     long sumTotalTermFreq = 0;
     long sumDocFreq = 0;
 
+    Term protoTerm = new Term(fieldName);
     for (int i = 0; i < numTerms; i++) {
       final int termID = termIDs[i];
       //System.out.println("term=" + termID);
@@ -398,7 +399,8 @@ final class FreqProxTermsWriterPerField 
 
       final int delDocLimit;
       if (segDeletes != null) {
-        final Integer docIDUpto = segDeletes.get(new Term(fieldName, text));
+        protoTerm.bytes = text;
+        final Integer docIDUpto = segDeletes.get(protoTerm);
         if (docIDUpto != null) {
           delDocLimit = docIDUpto;
         } else {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexDocument.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexDocument.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexDocument.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexDocument.java Fri Nov 23 12:00:32 2012
@@ -24,8 +24,8 @@ package org.apache.lucene.index;
 public interface IndexDocument {
 
   /** Obtains all indexable fields in document */
-  public Iterable<? extends IndexableField> indexableFields();
+  public Iterable<IndexableField> indexableFields();
   
   /** Obtains all storable fields in document */
-  public Iterable<? extends StorableField> storableFields();
+  public Iterable<StorableField> storableFields();
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Fri Nov 23 12:00:32 2012
@@ -1317,6 +1317,10 @@ public class IndexWriter implements Clos
                 checkpoint();
               }
             }
+
+            // Must bump changeCount so if no other changes
+            // happened, we still commit this change:
+            changeCount++;
           }
           //System.out.println("  yes " + info.info.name + " " + docID);
           return true;
@@ -1887,6 +1891,15 @@ public class IndexWriter implements Clos
   }
 
   /**
+   * Expert: returns true if there are merges waiting to be scheduled.
+   * 
+   * @lucene.experimental
+   */
+  public synchronized boolean hasPendingMerges() {
+    return pendingMerges.size() != 0;
+  }
+
+  /**
    * Close the <code>IndexWriter</code> without committing
    * any changes that have occurred since the last commit
    * (or since it was opened, if commit hasn't been called).
@@ -2069,7 +2082,7 @@ public class IndexWriter implements Clos
       // they are aborted.
       while(runningMerges.size() > 0) {
         if (infoStream.isEnabled("IW")) {
-          infoStream.message("IW", "now wait for " + runningMerges.size() + " running merge to abort");
+          infoStream.message("IW", "now wait for " + runningMerges.size() + " running merge/s to abort");
         }
         doWait();
       }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java Fri Nov 23 12:00:32 2012
@@ -555,7 +555,7 @@ public class LiveIndexWriterConfig {
     sb.append("commit=").append(commit == null ? "null" : commit).append("\n");
     sb.append("openMode=").append(getOpenMode()).append("\n");
     sb.append("similarity=").append(getSimilarity().getClass().getName()).append("\n");
-    sb.append("mergeScheduler=").append(getMergeScheduler().getClass().getName()).append("\n");
+    sb.append("mergeScheduler=").append(getMergeScheduler()).append("\n");
     sb.append("default WRITE_LOCK_TIMEOUT=").append(IndexWriterConfig.WRITE_LOCK_TIMEOUT).append("\n");
     sb.append("writeLockTimeout=").append(getWriteLockTimeout()).append("\n");
     sb.append("codec=").append(getCodec()).append("\n");

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java Fri Nov 23 12:00:32 2012
@@ -185,7 +185,7 @@ class MultiDocValues extends DocValues {
   }
 
   @Override
-  public Source load() throws IOException {
+  protected Source loadSource() throws IOException {
     return new MultiSource(slices, starts, false, type);
   }
 
@@ -199,7 +199,7 @@ class MultiDocValues extends DocValues {
     }
 
     @Override
-    public Source load() throws IOException {
+    protected Source loadSource() throws IOException {
       return emptySource;
     }
 
@@ -209,7 +209,7 @@ class MultiDocValues extends DocValues {
     }
 
     @Override
-    public Source getDirectSource() throws IOException {
+    protected Source loadDirectSource() throws IOException {
       return emptySource;
     }
   }
@@ -226,7 +226,7 @@ class MultiDocValues extends DocValues {
     }
 
     @Override
-    public Source load() throws IOException {
+    protected Source loadSource() throws IOException {
       return emptyFixedSource;
     }
 
@@ -241,7 +241,7 @@ class MultiDocValues extends DocValues {
     }
 
     @Override
-    public Source getDirectSource() throws IOException {
+    protected Source loadDirectSource() throws IOException {
       return emptyFixedSource;
     }
   }
@@ -594,7 +594,7 @@ class MultiDocValues extends DocValues {
   }
 
   @Override
-  public Source getDirectSource() throws IOException {
+  protected Source loadDirectSource() throws IOException {
     return new MultiSource(slices, starts, true, type);
   }
   

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Norm.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Norm.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Norm.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Norm.java Fri Nov 23 12:00:32 2012
@@ -115,6 +115,15 @@ public final class Norm  {
     setType(Type.FIXED_INTS_64);
     this.field.setLongValue(norm);
   }
+  
+  /**
+   * Sets a packed long norm value.
+   * @lucene.experimental
+   */
+  public void setPackedLong(long norm) {
+    setType(Type.VAR_INTS);
+    this.field.setLongValue(norm);
+  }
 
   /**
    * Sets a byte norm value

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java Fri Nov 23 12:00:32 2012
@@ -136,9 +136,7 @@ public final class SegmentReader extends
   
   @Override
   public void document(int docID, StoredFieldVisitor visitor) throws IOException {
-    if (docID < 0 || docID >= maxDoc()) {       
-      throw new IllegalArgumentException("docID must be >= 0 and < maxDoc=" + maxDoc() + " (got docID=" + docID + ")");
-    }
+    checkBounds(docID);
     getFieldsReader().visitDocument(docID, visitor);
   }
 
@@ -174,8 +172,15 @@ public final class SegmentReader extends
     if (termVectorsReader == null) {
       return null;
     }
+    checkBounds(docID);
     return termVectorsReader.get(docID);
   }
+  
+  private void checkBounds(int docID) {
+    if (docID < 0 || docID >= maxDoc()) {       
+      throw new IndexOutOfBoundsException("docID must be >= 0 and < maxDoc=" + maxDoc() + " (got docID=" + docID + ")");
+    }
+  }
 
   @Override
   public String toString() {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Term.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Term.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Term.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/Term.java Fri Nov 23 12:00:32 2012
@@ -37,8 +37,8 @@ public final class Term implements Compa
    *
    * <p>WARNING: the provided BytesRef is not copied, but used directly.
    * Therefore the bytes should not be modified after construction, for
-   * example, you should clone a copy rather than pass reused bytes from
-   * a TermsEnum.
+   * example, you should clone a copy by {@link BytesRef#deepCopyOf}
+   * rather than pass reused bytes from a TermsEnum.
    */
   public Term(String fld, BytesRef bytes) {
     field = fld;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermsHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermsHash.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermsHash.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermsHash.java Fri Nov 23 12:00:32 2012
@@ -23,6 +23,8 @@ import java.util.Map;
 
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.IntBlockPool;
 
 /** This class implements {@link InvertedDocConsumer}, which
  *  is passed each token produced by the analyzer on each
@@ -36,11 +38,11 @@ final class TermsHash extends InvertedDo
 
   final TermsHashConsumer consumer;
   final TermsHash nextTermsHash;
-  final DocumentsWriterPerThread docWriter;
 
   final IntBlockPool intPool;
   final ByteBlockPool bytePool;
   ByteBlockPool termBytePool;
+  final Counter bytesUsed;
 
   final boolean primary;
   final DocumentsWriterPerThread.DocState docState;
@@ -56,11 +58,11 @@ final class TermsHash extends InvertedDo
 
   public TermsHash(final DocumentsWriterPerThread docWriter, final TermsHashConsumer consumer, boolean trackAllocations, final TermsHash nextTermsHash) {
     this.docState = docWriter.docState;
-    this.docWriter = docWriter;
     this.consumer = consumer;
     this.trackAllocations = trackAllocations; 
     this.nextTermsHash = nextTermsHash;
-    intPool = new IntBlockPool(docWriter);
+    this.bytesUsed = trackAllocations ? docWriter.bytesUsed : Counter.newCounter();
+    intPool = new IntBlockPool(docWriter.intBlockAllocator);
     bytePool = new ByteBlockPool(docWriter.byteBlockAllocator);
 
     if (nextTermsHash != null) {
@@ -87,12 +89,9 @@ final class TermsHash extends InvertedDo
 
   // Clear all state
   void reset() {
-    intPool.reset();
-    bytePool.reset();
-
-    if (primary) {
-      bytePool.reset();
-    }
+    // we don't reuse so we drop everything and don't fill with 0
+    intPool.reset(false, false); 
+    bytePool.reset(false, false);
   }
 
   @Override

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java Fri Nov 23 12:00:32 2012
@@ -25,6 +25,7 @@ import org.apache.lucene.util.ByteBlockP
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.IntBlockPool;
 import org.apache.lucene.util.BytesRefHash.BytesStartArray;
 import org.apache.lucene.util.BytesRefHash.MaxBytesLengthExceededException;
 
@@ -62,8 +63,7 @@ final class TermsHashPerField extends In
     termBytePool = termsHash.termBytePool;
     docState = termsHash.docState;
     this.termsHash = termsHash;
-    bytesUsed = termsHash.trackAllocations ? termsHash.docWriter.bytesUsed
-        : Counter.newCounter();
+    bytesUsed = termsHash.bytesUsed;
     fieldState = docInverterPerField.fieldState;
     this.consumer = termsHash.consumer.addField(this, fieldInfo);
     PostingsBytesStartArray byteStarts = new PostingsBytesStartArray(this, bytesUsed);
@@ -99,8 +99,8 @@ final class TermsHashPerField extends In
   public void initReader(ByteSliceReader reader, int termID, int stream) {
     assert stream < streamCount;
     int intStart = postingsArray.intStarts[termID];
-    final int[] ints = intPool.buffers[intStart >> DocumentsWriterPerThread.INT_BLOCK_SHIFT];
-    final int upto = intStart & DocumentsWriterPerThread.INT_BLOCK_MASK;
+    final int[] ints = intPool.buffers[intStart >> IntBlockPool.INT_BLOCK_SHIFT];
+    final int upto = intStart & IntBlockPool.INT_BLOCK_MASK;
     reader.init(bytePool,
                 postingsArray.byteStarts[termID]+stream*ByteBlockPool.FIRST_LEVEL_SIZE,
                 ints[upto+stream]);
@@ -143,7 +143,7 @@ final class TermsHashPerField extends In
       // First time we are seeing this token since we last
       // flushed the hash.
       // Init stream slices
-      if (numPostingInt + intPool.intUpto > DocumentsWriterPerThread.INT_BLOCK_SIZE)
+      if (numPostingInt + intPool.intUpto > IntBlockPool.INT_BLOCK_SIZE)
         intPool.nextBuffer();
 
       if (ByteBlockPool.BYTE_BLOCK_SIZE - bytePool.byteUpto < numPostingInt*ByteBlockPool.FIRST_LEVEL_SIZE) {
@@ -167,8 +167,8 @@ final class TermsHashPerField extends In
     } else {
       termID = (-termID)-1;
       int intStart = postingsArray.intStarts[termID];
-      intUptos = intPool.buffers[intStart >> DocumentsWriterPerThread.INT_BLOCK_SHIFT];
-      intUptoStart = intStart & DocumentsWriterPerThread.INT_BLOCK_MASK;
+      intUptos = intPool.buffers[intStart >> IntBlockPool.INT_BLOCK_SHIFT];
+      intUptoStart = intStart & IntBlockPool.INT_BLOCK_MASK;
       consumer.addTerm(termID);
     }
   }
@@ -205,7 +205,7 @@ final class TermsHashPerField extends In
     if (termID >= 0) {// New posting
       bytesHash.byteStart(termID);
       // Init stream slices
-      if (numPostingInt + intPool.intUpto > DocumentsWriterPerThread.INT_BLOCK_SIZE) {
+      if (numPostingInt + intPool.intUpto > IntBlockPool.INT_BLOCK_SIZE) {
         intPool.nextBuffer();
       }
 
@@ -230,8 +230,8 @@ final class TermsHashPerField extends In
     } else {
       termID = (-termID)-1;
       final int intStart = postingsArray.intStarts[termID];
-      intUptos = intPool.buffers[intStart >> DocumentsWriterPerThread.INT_BLOCK_SHIFT];
-      intUptoStart = intStart & DocumentsWriterPerThread.INT_BLOCK_MASK;
+      intUptos = intPool.buffers[intStart >> IntBlockPool.INT_BLOCK_SHIFT];
+      intUptoStart = intStart & IntBlockPool.INT_BLOCK_MASK;
       consumer.addTerm(termID);
     }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java Fri Nov 23 12:00:32 2012
@@ -127,7 +127,7 @@ final class BooleanScorer extends Scorer
     public int docID() { return doc; }
 
     @Override
-    public float freq() { return freq; }
+    public int freq() { return freq; }
 
     @Override
     public int nextDoc() { return NO_MORE_DOCS; }
@@ -332,7 +332,7 @@ final class BooleanScorer extends Scorer
   }
 
   @Override
-  public float freq() throws IOException {
+  public int freq() throws IOException {
     throw new UnsupportedOperationException();
   }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java Fri Nov 23 12:00:32 2012
@@ -130,7 +130,7 @@ class BooleanScorer2 extends Scorer {
     }
 
     @Override
-    public float freq() throws IOException {
+    public int freq() throws IOException {
       return 1;
     }
 
@@ -319,7 +319,7 @@ class BooleanScorer2 extends Scorer {
   }
 
   @Override
-  public float freq() throws IOException {
+  public int freq() throws IOException {
     return countingSumScorer.freq();
   }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java Fri Nov 23 12:00:32 2012
@@ -86,7 +86,7 @@ public abstract class CachingCollector e
     public final int docID() { return doc; }
     
     @Override
-    public final float freq() { throw new UnsupportedOperationException(); }
+    public final int freq() { throw new UnsupportedOperationException(); }
     
     @Override
     public final int nextDoc() { throw new UnsupportedOperationException(); }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java Fri Nov 23 12:00:32 2012
@@ -154,7 +154,7 @@ class ConjunctionScorer extends Scorer {
 
 
   @Override
-  public float freq() throws IOException {
+  public int freq() throws IOException {
     return scorers.length;
   }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java Fri Nov 23 12:00:32 2012
@@ -108,7 +108,7 @@ class ConjunctionTermScorer extends Scor
   }
   
   @Override
-  public float freq() {
+  public int freq() {
     return docsAndFreqs.length;
   }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java Fri Nov 23 12:00:32 2012
@@ -197,7 +197,7 @@ public class ConstantScoreQuery extends 
     }
 
     @Override
-    public float freq() throws IOException {
+    public int freq() throws IOException {
       return 1;
     }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java Fri Nov 23 12:00:32 2012
@@ -104,7 +104,7 @@ class DisjunctionMaxScorer extends Disju
   }
 
   @Override
-  public float freq() throws IOException {
+  public int freq() throws IOException {
     int doc = subScorers[0].docID();
     int size = numScorers;
     return 1 + freq(1, size, doc) + freq(2, size, doc);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java Fri Nov 23 12:00:32 2012
@@ -134,7 +134,7 @@ class DisjunctionSumScorer extends Disju
   }
 
   @Override
-  public float freq() throws IOException {
+  public int freq() throws IOException {
     return nrMatchers;
   }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java Fri Nov 23 12:00:32 2012
@@ -193,7 +193,7 @@ final class ExactPhraseScorer extends Sc
   }
   
   @Override
-  public float freq() {
+  public int freq() {
     return freq;
   }
   
@@ -333,6 +333,6 @@ final class ExactPhraseScorer extends Sc
     for (int i = 0; i < chunkStates.length; i++) {
       posIters[i] = new TermIntervalIterator(this, enums[i] = chunkStates[i].factory.docsAndPositionsEnum(), false, collectIntervals);
     }
-    return new PhraseScorer.AdvancingIntervalIterator(this, collectIntervals, enums, new BlockIntervalIterator(this, collectIntervals, posIters));
+    return new SloppyPhraseScorer.AdvancingIntervalIterator(this, collectIntervals, enums, new BlockIntervalIterator(this, collectIntervals, posIters));
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java Fri Nov 23 12:00:32 2012
@@ -205,7 +205,7 @@ public class FilteredQuery extends Query
     }
     
     @Override
-    public float freq() throws IOException { return scorer.freq(); }
+    public int freq() throws IOException { return scorer.freq(); }
     
     @Override
     public Collection<ChildScorer> getChildren() {
@@ -305,7 +305,7 @@ public class FilteredQuery extends Query
     }
     
     @Override
-    public final float freq() throws IOException { return scorer.freq(); }
+    public final int freq() throws IOException { return scorer.freq(); }
     
     @Override
     public final Collection<ChildScorer> getChildren() {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java Fri Nov 23 12:00:32 2012
@@ -502,7 +502,7 @@ public class IndexSearcher {
                                 boolean doDocScores, boolean doMaxScore)
       throws IOException {
 
-    if (sort == null) throw new NullPointerException();
+    if (sort == null) throw new NullPointerException("Sort must not be null");
     
     if (executor == null) {
       // use all leaves here!
@@ -750,7 +750,7 @@ public class IndexSearcher {
     
       @Override
       public int advance(int target) {
-        throw new UnsupportedOperationException();
+        throw new UnsupportedOperationException("FakeScorer doesn't support advance(int)");
       }
 
       @Override
@@ -759,13 +759,13 @@ public class IndexSearcher {
       }
 
       @Override
-      public float freq() {
-        throw new UnsupportedOperationException();
+      public int freq() {
+        throw new UnsupportedOperationException("FakeScorer doesn't support freq()");
       }
 
       @Override
       public int nextDoc() {
-        throw new UnsupportedOperationException();
+        throw new UnsupportedOperationException("FakeScorer doesn't support nextDoc()");
       }
     
       @Override
@@ -833,8 +833,8 @@ public class IndexSearcher {
     }
 
     public T next() {
-      if(!this.hasNext())
-        throw new NoSuchElementException();
+      if(!this.hasNext()) 
+        throw new NoSuchElementException("next() is called but hasNext() returned false");
       try {
         return service.take().get();
       } catch (InterruptedException e) {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java Fri Nov 23 12:00:32 2012
@@ -69,7 +69,7 @@ public class MatchAllDocsQuery extends Q
     }
 
     @Override
-    public float freq() {
+    public int freq() {
       return 1;
     }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java Fri Nov 23 12:00:32 2012
@@ -252,7 +252,7 @@ public class MultiPhraseQuery extends Qu
       if (scorer != null) {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {
-          float freq = scorer.freq();
+          float freq = slop == 0 ? scorer.freq() : ((SloppyPhraseScorer)scorer).sloppyFreq();
           SloppySimScorer docScorer = similarity.sloppySimScorer(stats, context);
           ComplexExplanation result = new ComplexExplanation();
           result.setDescription("weight("+getQuery()+" in "+doc+") [" + similarity.getClass().getSimpleName() + "], result of:");

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java Fri Nov 23 12:00:32 2012
@@ -306,7 +306,7 @@ public class PhraseQuery extends Query {
       if (scorer != null) {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {
-          float freq = scorer.freq();
+          float freq = slop == 0 ? scorer.freq() : ((SloppyPhraseScorer)scorer).sloppyFreq();
           SloppySimScorer docScorer = similarity.sloppySimScorer(stats, context);
           ComplexExplanation result = new ComplexExplanation();
           result.setDescription("weight("+getQuery()+" in "+doc+") [" + similarity.getClass().getSimpleName() + "], result of:");

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReferenceManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReferenceManager.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReferenceManager.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReferenceManager.java Fri Nov 23 12:00:32 2012
@@ -58,18 +58,24 @@ public abstract class ReferenceManager<G
     release(oldReference);
   }
 
-  /** Decrement reference counting on the given reference. */
+  /**
+   * Decrement reference counting on the given reference. 
+   * @throws IOException if reference decrement on the given resource failed.
+   * */
   protected abstract void decRef(G reference) throws IOException;
   
   /**
    * Refresh the given reference if needed. Returns {@code null} if no refresh
    * was needed, otherwise a new refreshed reference.
+   * @throws AlreadyClosedException if the reference manager has been {@link #close() closed}.
+   * @throws IOException if the refresh operation failed
    */
   protected abstract G refreshIfNeeded(G referenceToRefresh) throws IOException;
 
   /**
    * Try to increment reference counting on the given reference. Return true if
    * the operation was successful.
+   * @throws AlreadyClosedException if the reference manager has been {@link #close() closed}. 
    */
   protected abstract boolean tryIncRef(G reference);
 
@@ -78,6 +84,7 @@ public abstract class ReferenceManager<G
    * call to {@link #release}; it's best to do so in a finally clause, and set
    * the reference to {@code null} to prevent accidental usage after it has been
    * released.
+   * @throws AlreadyClosedException if the reference manager has been {@link #close() closed}. 
    */
   public final G acquire() {
     G ref;
@@ -88,12 +95,27 @@ public abstract class ReferenceManager<G
     } while (!tryIncRef(ref));
     return ref;
   }
-
+  
   /**
-   * Close this ReferenceManager to future {@link #acquire() acquiring}. Any
-   * references that were previously {@link #acquire() acquired} won't be
-   * affected, and they should still be {@link #release released} when they are
-   * not needed anymore.
+    * <p>
+    * Closes this ReferenceManager to prevent future {@link #acquire() acquiring}. A
+    * reference manager should be closed if the reference to the managed resource
+    * should be disposed or the application using the {@link ReferenceManager}
+    * is shutting down. The managed resource might not be released immediately,
+    * if the {@link ReferenceManager} user is holding on to a previously
+    * {@link #acquire() acquired} reference. The resource will be released once
+    * when the last reference is {@link #release(Object) released}. Those
+    * references can still be used as if the manager was still active.
+    * </p>
+    * <p>
+    * Applications should not {@link #acquire() acquire} new references from this
+    * manager once this method has been called. {@link #acquire() Acquiring} a
+    * resource on a closed {@link ReferenceManager} will throw an
+    * {@link AlreadyClosedException}.
+    * </p>
+    * 
+    * @throws IOException
+    *           if the underlying reader of the current reference could not be closed
    */
   public final synchronized void close() throws IOException {
     if (current != null) {
@@ -105,7 +127,10 @@ public abstract class ReferenceManager<G
     }
   }
 
-  /** Called after close(), so subclass can free any resources. */
+  /**
+   *  Called after close(), so subclass can free any resources.
+   *  @throws IOException if the after close operation in a sub-class throws an {@link IOException} 
+   * */
   protected void afterClose() throws IOException {
   }
 
@@ -158,6 +183,9 @@ public abstract class ReferenceManager<G
    * If this method returns true it means the calling thread either refreshed or
    * that there were no changes to refresh. If it returns false it means another
    * thread is currently refreshing.
+   * </p>
+   * @throws IOException if refreshing the resource causes an {@link IOException}
+   * @throws AlreadyClosedException if the reference manager has been {@link #close() closed}. 
    */
   public final boolean maybeRefresh() throws IOException {
     ensureOpen();
@@ -185,6 +213,8 @@ public abstract class ReferenceManager<G
    * useful if you want to guarantee that the next call to {@link #acquire()}
    * will return a refreshed instance. Otherwise, consider using the
    * non-blocking {@link #maybeRefresh()}.
+   * @throws IOException if refreshing the resource causes an {@link IOException}
+   * @throws AlreadyClosedException if the reference manager has been {@link #close() closed}. 
    */
   public final void maybeRefreshBlocking() throws IOException {
     ensureOpen();
@@ -199,14 +229,17 @@ public abstract class ReferenceManager<G
   }
 
   /** Called after swapReference has installed a new
-   *  instance. */
+   *  instance.
+   *  @throws IOException if a low level I/O exception occurs  
+   **/
   protected void afterRefresh() throws IOException {
   }
   
   /**
-   * Release the refernce previously obtained via {@link #acquire()}.
+   * Release the reference previously obtained via {@link #acquire()}.
    * <p>
    * <b>NOTE:</b> it's safe to call this after {@link #close()}.
+   * @throws IOException if the release operation on the given resource throws an {@link IOException}
    */
   public final void release(G reference) throws IOException {
     assert reference != null;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java Fri Nov 23 12:00:32 2012
@@ -108,7 +108,7 @@ class ReqExclScorer extends Scorer {
   }
   
   @Override
-  public float freq() throws IOException {
+  public int freq() throws IOException {
     return reqScorer.freq();
   }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java Fri Nov 23 12:00:32 2012
@@ -94,7 +94,7 @@ class ReqOptSumScorer extends Scorer {
   }
 
   @Override
-  public float freq() throws IOException {
+  public int freq() throws IOException {
     // we might have deferred advance()
     score();
     return (optScorer != null && optScorer.docID() == reqScorer.docID()) ? 2 : 1;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java Fri Nov 23 12:00:32 2012
@@ -63,7 +63,7 @@ public class ScoreCachingWrappingScorer 
   }
 
   @Override
-  public float freq() throws IOException {
+  public int freq() throws IOException {
     return scorer.freq();
   }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java Fri Nov 23 12:00:32 2012
@@ -23,6 +23,8 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
 
+import org.apache.lucene.index.DocsEnum;
+
 /**
  * Expert: Common scoring functionality for different types of queries.
  *
@@ -41,7 +43,7 @@ import java.util.Collections;
  * TopScoreDocCollector}) will not properly collect hits
  * with these scores.
  */
-public abstract class Scorer extends DocIdSetIterator {
+public abstract class Scorer extends DocsEnum {
   /** the Scorer's parent Weight. in some cases this may be null */
   // TODO can we clean this up?
   protected final Weight weight;
@@ -139,14 +141,6 @@ public abstract class Scorer extends Doc
    * {@link Collector#collect}.
    */
   public abstract float score() throws IOException;
-
-  /** Returns number of matches for the current document.
-   *  This returns a float (not int) because
-   *  SloppyPhraseScorer discounts its freq according to how
-   *  "sloppy" the match was.
-   *
-   * @lucene.experimental */
-  public abstract float freq() throws IOException;
   
   /** returns parent Weight
    * @lucene.experimental

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java Fri Nov 23 12:00:32 2012
@@ -30,16 +30,22 @@ import java.util.Map;
 
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.search.intervals.ConjunctionIntervalIterator;
 import org.apache.lucene.search.intervals.Interval;
+import org.apache.lucene.search.intervals.IntervalCollector;
 import org.apache.lucene.search.intervals.IntervalIterator;
 import org.apache.lucene.search.intervals.SloppyIntervalIterator;
 import org.apache.lucene.search.intervals.TermIntervalIterator;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.OpenBitSet;
 
-final class SloppyPhraseScorer extends PhraseScorer {
-//  private MaxLengthPositionIntervalIterator iter;
+final class SloppyPhraseScorer extends Scorer {
+  private PhrasePositions min, max;
+
+  private float sloppyFreq; //phrase frequency in current doc as computed by phraseFreq().
+
+  private final Similarity.SloppySimScorer docScorer;
+  private final PhraseQuery.PostingsAndFreq[] postings;
+  
   private final int slop;
   private final int numPostings;
   private final PhraseQueue pq; // for advancing min position
@@ -52,23 +58,34 @@ final class SloppyPhraseScorer extends P
   private PhrasePositions[][] rptGroups; // in each group are PPs that repeats each other (i.e. same term), sorted by (query) offset 
   private PhrasePositions[] rptStack; // temporary stack for switching colliding repeating pps 
   
+  private int numMatches;
+  
   SloppyPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
-      int slop, Similarity.SloppySimScorer docScorer) throws IOException {
-    super(weight, postings, docScorer);
+      int slop, Similarity.SloppySimScorer docScorer) {
+    super(weight);
+    this.docScorer = docScorer;
+    this.postings = postings;
     this.slop = slop;
     this.numPostings = postings==null ? 0 : postings.length;
     pq = new PhraseQueue(postings.length);
-//    iter = (MaxLengthPositionIntervalIterator) positions(false, false, false);
+    // convert tps to a list of phrase positions.
+    // note: phrase-position differs from term-position in that its position
+    // reflects the phrase offset: pp.pos = tp.pos - offset.
+    // this allows to easily identify a matching (exact) phrase 
+    // when all PhrasePositions have exactly the same position.
+    if (postings.length > 0) {
+      min = new PhrasePositions(postings[0].postings, postings[0].position, 0, postings[0].terms);
+      max = min;
+      max.doc = -1;
+      for (int i = 1; i < postings.length; i++) {
+        PhrasePositions pp = new PhrasePositions(postings[i].postings, postings[i].position, i, postings[i].terms);
+        max.next = pp;
+        max = pp;
+        max.doc = -1;
+      }
+      max.next = min; // make it cyclic for easier manipulation
+    }
   }
-  
-//  String current() {
-//    StringBuilder b = new StringBuilder();
-//    int i = 0;
-//    for (PhrasePositions phrasePositions : postings) {
-//      b.append(i++).append(phrasePositions).append("\n");
-//    }
-//    return b.toString();
-//  }
 
   /**
    * Score a candidate doc for all slop-valid position-combinations (matches) 
@@ -88,76 +105,39 @@ final class SloppyPhraseScorer extends P
    * would get same score as "g f"~2, although "c b"~2 could be matched twice.
    * We may want to fix this in the future (currently not, for performance reasons).
    */
-  @Override
-  protected float phraseFreq() throws IOException {
-//    float freq = 0.0f;
-//
-//    if (1 ==1) {
-//    if (iter.docID() != this.docID()) {
-//      iter.advanceTo(this.docID());
-//    }
-//    while(iter.next() != null) {
-//      freq += docScorer.computeSlopFactor(iter.matchLength()); // score match
-//    }
-//    
-//    return freq;
-//    }
-//    freq = 0.0f;
-
+  private float phraseFreq() throws IOException {
     if (!initPhrasePositions()) {
       return 0.0f;
     }
-   
     float freq = 0.0f;
+    numMatches = 0;
     PhrasePositions pp = pq.pop();
     int matchLength = end - pp.position;
     int next = pq.top().position; 
-    
-//    int _lPos = pp.position;
-//    int _lend = end;
-//    String _s = current();
-//    Term[] _lTerms = pp.terms;
     while (advancePP(pp)) {
       if (hasRpts && !advanceRpts(pp)) {
         break; // pps exhausted
       }
       if (pp.position > next) { // done minimizing current match-length 
         if (matchLength <= slop) {
-//          System.out.println("match: " + _lPos + " " + _lend + " " + Arrays.toString(_lTerms)  + " " + matchLength);
-//          System.out.println(_s);
-//          System.out.println( docScorer.computeSlopFactor(matchLength));
           freq += docScorer.computeSlopFactor(matchLength); // score match
+          numMatches++;
         }      
         pq.add(pp);
         pp = pq.pop();
         next = pq.top().position;
         matchLength = end - pp.position;
-//        _lPos = pp.position;
-//        _lend = end;
-//        _lTerms = pp.terms;
-//        _s = current();
       } else {
         int matchLength2 = end - pp.position;
-        
         if (matchLength2 < matchLength) {
-//          _lPos = pp.position;
-//          _lend = end;
-//          _lTerms = pp.terms;
-//          _s = current();
           matchLength = matchLength2;
         }
       }
     }
     if (matchLength <= slop) {
-//      System.out.println("match: " + _lPos + " " + _lend + " " + Arrays.toString(_lTerms) + " " + matchLength);
-//      System.out.println(_s);
-//      System.out.println( docScorer.computeSlopFactor(matchLength));
-
       freq += docScorer.computeSlopFactor(matchLength); // score match
-
+      numMatches++;
     }    
-//    System.out.println("res: " + freq + " doc: " + this.docID());
-    
     return freq;
   }
 
@@ -542,6 +522,87 @@ final class SloppyPhraseScorer extends P
   }
 
   @Override
+  public int freq() {
+    return numMatches;
+  }
+  
+  float sloppyFreq() {
+    return sloppyFreq;
+  }
+  
+//  private void printQueue(PrintStream ps, PhrasePositions ext, String title) {
+//    //if (min.doc != ?) return;
+//    ps.println();
+//    ps.println("---- "+title);
+//    ps.println("EXT: "+ext);
+//    PhrasePositions[] t = new PhrasePositions[pq.size()];
+//    if (pq.size()>0) {
+//      t[0] = pq.pop();
+//      ps.println("  " + 0 + "  " + t[0]);
+//      for (int i=1; i<t.length; i++) {
+//        t[i] = pq.pop();
+//        assert t[i-1].position <= t[i].position;
+//        ps.println("  " + i + "  " + t[i]);
+//      }
+//      // add them back
+//      for (int i=t.length-1; i>=0; i--) {
+//        pq.add(t[i]);
+//      }
+//    }
+//  }
+  
+  private boolean advanceMin(int target) throws IOException {
+    if (!min.skipTo(target)) { 
+      max.doc = NO_MORE_DOCS; // for further calls to docID() 
+      return false;
+    }
+    min = min.next; // cyclic
+    max = max.next; // cyclic
+    return true;
+  }
+  
+  @Override
+  public int docID() {
+    return max.doc; 
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    return advance(max.doc);
+  }
+  
+  @Override
+  public float score() {
+    return docScorer.score(max.doc, sloppyFreq);
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    sloppyFreq = 0.0f;
+    if (!advanceMin(target)) {
+      return NO_MORE_DOCS;
+    }        
+    boolean restart=false;
+    while (sloppyFreq == 0.0f) {
+      while (min.doc < max.doc || restart) {
+        restart = false;
+        if (!advanceMin(max.doc)) {
+          return NO_MORE_DOCS;
+        }        
+      }
+      // found a doc with all of the terms
+      sloppyFreq = phraseFreq(); // check for phrase
+      restart = true;
+    } 
+
+    // found a match
+    return max.doc;
+  }
+  
+  @Override
+  public String toString() { return "scorer(" + weight + ")"; }
+
+  @Override
   public IntervalIterator intervals(boolean collectIntervals) throws IOException {
     Map<Term, IterAndOffsets> map = new HashMap<Term, IterAndOffsets>();
     List<DocsAndPositionsEnum> enums = new ArrayList<DocsAndPositionsEnum>();
@@ -572,15 +633,15 @@ final class SloppyPhraseScorer extends P
     }
     return new AdvancingIntervalIterator(this, collectIntervals, enums.toArray(new DocsAndPositionsEnum[enums.size()]), new SloppyIntervalIterator(this, slop, collectIntervals, iters));
   }
-  
+
   private final static class IterAndOffsets {
     final List<Integer> offsets = new ArrayList<Integer>();
     final IntervalIterator iter;
-    
+
     IterAndOffsets(IntervalIterator iter) {
       this.iter = iter;
     }
-    
+
     int[] toIntArray() {
       int[] array = new int[offsets.size()];
       for (int i = 0; i < array.length; i++) {
@@ -589,6 +650,47 @@ final class SloppyPhraseScorer extends P
       return array;
     }
   }
-  
 
+  final static class AdvancingIntervalIterator extends IntervalIterator {
+
+    public AdvancingIntervalIterator(Scorer scorer, boolean collectIntervals, final DocsAndPositionsEnum[] enums, final IntervalIterator delegate) {
+      super(scorer, collectIntervals);
+      this.enums = enums;
+      this.delegate = delegate;
+    }
+
+    private final DocsAndPositionsEnum[] enums;
+    private final IntervalIterator delegate;
+    @Override
+    public int scorerAdvanced(int docId) throws IOException {
+      assert docId == docID();
+      for (DocsAndPositionsEnum oneEnum : enums) {
+        int advance = oneEnum.advance(docId);
+        assert advance == docId;
+      }
+      delegate.scorerAdvanced(docId);
+      return docId;
+    }
+
+    @Override
+    public Interval next() throws IOException {
+      return delegate.next();
+    }
+
+    @Override
+    public void collect(IntervalCollector collector) {
+      delegate.collect(collector);
+    }
+
+    @Override
+    public IntervalIterator[] subs(boolean inOrder) {
+      return delegate.subs(inOrder);
+    }
+
+    @Override
+    public int matchDistance() {
+      return delegate.matchDistance();
+    }
+
+  }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TermScorer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TermScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/TermScorer.java Fri Nov 23 12:00:32 2012
@@ -58,7 +58,7 @@ final class TermScorer extends Scorer {
   }
 
   @Override
-  public float freq() throws IOException {
+  public int freq() throws IOException {
     return docsEnum.freq();
   }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/intervals/IntervalFilterQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/intervals/IntervalFilterQuery.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/intervals/IntervalFilterQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/intervals/IntervalFilterQuery.java Fri Nov 23 12:00:32 2012
@@ -334,7 +334,11 @@ public class IntervalFilterQuery extends
     }
 
     @Override
-    public float freq() throws IOException {
+    public int freq() throws IOException {
+      return 1; // nocommit how to calculate frequency?
+    }
+
+    public float sloppyFreq() throws IOException {
       float freq = 0.0f;
       do {
         int d = filter.matchDistance();

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/intervals/NonOverlappingQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/intervals/NonOverlappingQuery.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/intervals/NonOverlappingQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/intervals/NonOverlappingQuery.java Fri Nov 23 12:00:32 2012
@@ -318,7 +318,7 @@ public final class NonOverlappingQuery e
     }
 
     @Override
-    public float freq() throws IOException {
+    public int freq() throws IOException {
       return minuend.freq();
     }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java Fri Nov 23 12:00:32 2012
@@ -103,12 +103,14 @@ public class PayloadTermQuery extends Sp
         }
         doc = spans.doc();
         freq = 0.0f;
+        numMatches = 0;
         payloadScore = 0;
         payloadsSeen = 0;
         while (more && doc == spans.doc()) {
           int matchLength = spans.end() - spans.start();
 
           freq += docScorer.computeSlopFactor(matchLength);
+          numMatches++;
           processPayload(similarity);
 
           more = spans.next();// this moves positions to the next match in this
@@ -179,7 +181,7 @@ public class PayloadTermQuery extends Sp
       if (scorer != null) {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {
-          float freq = scorer.freq();
+          float freq = scorer.sloppyFreq();
           SloppySimScorer docScorer = similarity.sloppySimScorer(stats, context);
           Explanation expl = new Explanation();
           expl.setDescription("weight("+getQuery()+" in "+doc+") [" + similarity.getClass().getSimpleName() + "], result of:");

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/similarities/DefaultSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/similarities/DefaultSimilarity.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/similarities/DefaultSimilarity.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/similarities/DefaultSimilarity.java Fri Nov 23 12:00:32 2012
@@ -46,13 +46,13 @@ public class DefaultSimilarity extends T
    *
    *  @lucene.experimental */
   @Override
-  public void computeNorm(FieldInvertState state, Norm norm) {
+  public float lengthNorm(FieldInvertState state) {
     final int numTerms;
     if (discountOverlaps)
       numTerms = state.getLength() - state.getNumOverlap();
     else
       numTerms = state.getLength();
-    norm.setByte(encodeNormValue(state.getBoost() * ((float) (1.0 / Math.sqrt(numTerms)))));
+   return state.getBoost() * ((float) (1.0 / Math.sqrt(numTerms)));
   }
 
   /** Implemented as <code>sqrt(freq)</code>. */

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java Fri Nov 23 12:00:32 2012
@@ -22,6 +22,8 @@ import java.io.IOException;
 
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.FieldInvertState;
+import org.apache.lucene.index.Norm;
 import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
@@ -668,6 +670,23 @@ public abstract class TFIDFSimilarity ex
    */
   public abstract float idf(long docFreq, long numDocs);
 
+  /**
+   * Compute an index-time normalization value for this field instance.
+   * <p>
+   * This value will be stored in a single byte lossy representation by 
+   * {@link #encodeNormValue(float)}.
+   * 
+   * @param state statistics of the current field (such as length, boost, etc)
+   * @return an index-time normalization value
+   */
+  public abstract float lengthNorm(FieldInvertState state);
+  
+  @Override
+  public final void computeNorm(FieldInvertState state, Norm norm) {
+    float normValue = lengthNorm(state);
+    norm.setByte(encodeNormValue(normValue));
+  }
+  
   /** Cache of decoded bytes. */
   private static final float[] NORM_TABLE = new float[256];
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java Fri Nov 23 12:00:32 2012
@@ -34,6 +34,7 @@ public class SpanScorer extends Scorer {
 
   protected int doc;
   protected float freq;
+  protected int numMatches;
   protected final Similarity.SloppySimScorer docScorer;
   
   protected SpanScorer(Spans spans, Weight weight, Similarity.SloppySimScorer docScorer)
@@ -78,9 +79,11 @@ public class SpanScorer extends Scorer {
     }
     doc = spans.doc();
     freq = 0.0f;
+    numMatches = 0;
     do {
       int matchLength = spans.end() - spans.start();
       freq += docScorer.computeSlopFactor(matchLength);
+      numMatches++;
       more = spans.next();
     } while (more && (doc == spans.doc()));
     return true;
@@ -95,7 +98,14 @@ public class SpanScorer extends Scorer {
   }
   
   @Override
-  public float freq() throws IOException {
+  public int freq() throws IOException {
+    return numMatches;
+  }
+  
+  /** Returns the intermediate "sloppy freq" adjusted for edit distance 
+   *  @lucene.internal */
+  // only public so .payloads can see it.
+  public float sloppyFreq() throws IOException {
     return freq;
   }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java Fri Nov 23 12:00:32 2012
@@ -92,11 +92,11 @@ public class SpanWeight extends Weight {
 
   @Override
   public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-    Scorer scorer = scorer(context, true, false, PostingFeatures.POSITIONS, context.reader().getLiveDocs());
+    SpanScorer scorer = (SpanScorer) scorer(context, true, false, PostingFeatures.POSITIONS, context.reader().getLiveDocs());
     if (scorer != null) {
       int newDoc = scorer.advance(doc);
       if (newDoc == doc) {
-        float freq = scorer.freq();
+        float freq = scorer.sloppyFreq();
         SloppySimScorer docScorer = similarity.sloppySimScorer(stats, context);
         ComplexExplanation result = new ComplexExplanation();
         result.setDescription("weight("+getQuery()+" in "+doc+") [" + similarity.getClass().getSimpleName() + "], result of:");

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java Fri Nov 23 12:00:32 2012
@@ -123,9 +123,6 @@ public abstract class FSDirectory extend
   protected final Set<String> staleFiles = synchronizedSet(new HashSet<String>()); // Files written, but not yet sync'ed
   private int chunkSize = DEFAULT_READ_CHUNK_SIZE; // LUCENE-1566
 
-  // null means no limit
-  private volatile RateLimiter mergeWriteRateLimiter;
-
   // returns the canonical version of the directory, creating it if it doesn't exist.
   private static File getCanonicalPath(File file) throws IOException {
     return new File(file.getCanonicalPath());
@@ -286,51 +283,7 @@ public abstract class FSDirectory extend
     ensureOpen();
 
     ensureCanWrite(name);
-    return new FSIndexOutput(this, name, context.context == IOContext.Context.MERGE ? mergeWriteRateLimiter : null);
-  }
-
-  /** Sets the maximum (approx) MB/sec allowed by all write
-   *  IO performed by merging.  Pass null to have no limit.
-   *
-   *  <p><b>NOTE</b>: if merges are already running there is
-   *  no guarantee this new rate will apply to them; it will
-   *  only apply for certain to new merges.
-   *
-   * @lucene.experimental */
-  public void setMaxMergeWriteMBPerSec(Double mbPerSec) {
-    RateLimiter limiter = mergeWriteRateLimiter;
-    if (mbPerSec == null) {
-      if (limiter != null) {
-        limiter.setMbPerSec(Double.MAX_VALUE);
-        mergeWriteRateLimiter = null;
-      }
-    } else if (limiter != null) {
-      limiter.setMbPerSec(mbPerSec);
-    } else {
-      mergeWriteRateLimiter = new RateLimiter(mbPerSec);
-    }
-  }
-
-  /**
-   * Sets the rate limiter to be used to limit (approx) MB/sec allowed
-   * by all IO performed when merging. Pass null to have no limit.
-   *
-   * <p>Passing an instance of rate limiter compared to setting it using
-   * {@link #setMaxMergeWriteMBPerSec(Double)} allows to use the same limiter
-   * instance across several directories globally limiting IO when merging
-   * across them.
-   *
-   * @lucene.experimental */
-  public void setMaxMergeWriteLimiter(RateLimiter mergeWriteRateLimiter) {
-    this.mergeWriteRateLimiter = mergeWriteRateLimiter;
-  }
-
-  /** See {@link #setMaxMergeWriteMBPerSec}.
-   *
-   * @lucene.experimental */
-  public Double getMaxMergeWriteMBPerSec() {
-    RateLimiter limiter = mergeWriteRateLimiter;
-    return limiter == null ? null : limiter.getMbPerSec();
+    return new FSIndexOutput(this, name);
   }
 
   protected void ensureCanWrite(String name) throws IOException {
@@ -504,23 +457,18 @@ public abstract class FSDirectory extend
     private final String name;
     private final RandomAccessFile file;
     private volatile boolean isOpen; // remember if the file is open, so that we don't try to close it more than once
-    private final RateLimiter rateLimiter;
     
-    public FSIndexOutput(FSDirectory parent, String name, RateLimiter rateLimiter) throws IOException {
+    public FSIndexOutput(FSDirectory parent, String name) throws IOException {
       this.parent = parent;
       this.name = name;
       file = new RandomAccessFile(new File(parent.directory, name), "rw");
       isOpen = true;
-      this.rateLimiter = rateLimiter;
     }
 
     /** output methods: */
     @Override
     public void flushBuffer(byte[] b, int offset, int size) throws IOException {
       assert isOpen;
-      if (rateLimiter != null) {
-        rateLimiter.pause(size);
-      }
       file.write(b, offset, size);
     }
     

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/store/RateLimiter.java Fri Nov 23 12:00:32 2012
@@ -19,75 +19,102 @@ package org.apache.lucene.store;
 
 import org.apache.lucene.util.ThreadInterruptedException;
 
-/** Simple class to rate limit IO.  Typically it's shared
- *  across multiple IndexInputs or IndexOutputs (for example
+/** Abstract base class to rate limit IO.  Typically implementations are
+ *  shared across multiple IndexInputs or IndexOutputs (for example
  *  those involved all merging).  Those IndexInputs and
  *  IndexOutputs would call {@link #pause} whenever they
  *  want to read bytes or write bytes. */
-
-public class RateLimiter {
-  private volatile double mbPerSec;
-  private volatile double nsPerByte;
-  private volatile long lastNS;
-
-  // TODO: we could also allow eg a sub class to dynamically
-  // determine the allowed rate, eg if an app wants to
-  // change the allowed rate over time or something
-
-  /** mbPerSec is the MB/sec max IO rate */
-  public RateLimiter(double mbPerSec) {
-    setMbPerSec(mbPerSec);
-  }
+public abstract class RateLimiter {
 
   /**
    * Sets an updated mb per second rate limit.
    */
-  public void setMbPerSec(double mbPerSec) {
-    this.mbPerSec = mbPerSec;
-    nsPerByte = 1000000000. / (1024*1024*mbPerSec);
-  }
-
+  public abstract void setMbPerSec(double mbPerSec);
   /**
    * The current mb per second rate limit.
    */
-  public double getMbPerSec() {
-    return this.mbPerSec;
-  }
-
+  public abstract double getMbPerSec();
+  
   /** Pauses, if necessary, to keep the instantaneous IO
-   *  rate at or below the target. NOTE: multiple threads
-   *  may safely use this, however the implementation is
-   *  not perfectly thread safe but likely in practice this
-   *  is harmless (just means in some rare cases the rate
-   *  might exceed the target).  It's best to call this
-   *  with a biggish count, not one byte at a time. */
-  public void pause(long bytes) {
-    if (bytes == 1) {
-      return;
+   *  rate at or below the target. 
+   *  <p>
+   *  Note: the implementation is thread-safe
+   *  </p>
+   *  @return the pause time in nano seconds 
+   * */
+  public abstract long pause(long bytes);
+  
+  /**
+   * Simple class to rate limit IO.
+   */
+  public static class SimpleRateLimiter extends RateLimiter {
+    private volatile double mbPerSec;
+    private volatile double nsPerByte;
+    private volatile long lastNS;
+
+    // TODO: we could also allow eg a sub class to dynamically
+    // determine the allowed rate, eg if an app wants to
+    // change the allowed rate over time or something
+
+    /** mbPerSec is the MB/sec max IO rate */
+    public SimpleRateLimiter(double mbPerSec) {
+      setMbPerSec(mbPerSec);
+    }
+
+    /**
+     * Sets an updated mb per second rate limit.
+     */
+    public void setMbPerSec(double mbPerSec) {
+      this.mbPerSec = mbPerSec;
+      nsPerByte = 1000000000. / (1024*1024*mbPerSec);
+      
     }
 
-    // TODO: this is purely instantaneous rate; maybe we
-    // should also offer decayed recent history one?
-    final long targetNS = lastNS = lastNS + ((long) (bytes * nsPerByte));
-    long curNS = System.nanoTime();
-    if (lastNS < curNS) {
-      lastNS = curNS;
+    /**
+     * The current mb per second rate limit.
+     */
+    public double getMbPerSec() {
+      return this.mbPerSec;
     }
+    
+    /** Pauses, if necessary, to keep the instantaneous IO
+     *  rate at or below the target. NOTE: multiple threads
+     *  may safely use this, however the implementation is
+     *  not perfectly thread safe but likely in practice this
+     *  is harmless (just means in some rare cases the rate
+     *  might exceed the target).  It's best to call this
+     *  with a biggish count, not one byte at a time.
+     *  @return the pause time in nano seconds 
+     * */
+    public long pause(long bytes) {
+      if (bytes == 1) {
+        return 0;
+      }
+
+      // TODO: this is purely instantaneous rate; maybe we
+      // should also offer decayed recent history one?
+      final long targetNS = lastNS = lastNS + ((long) (bytes * nsPerByte));
+      long curNS = System.nanoTime();
+      if (lastNS < curNS) {
+        lastNS = curNS;
+      }
 
-    // While loop because Thread.sleep doesn't always sleep
-    // enough:
-    while(true) {
-      final long pauseNS = targetNS - curNS;
-      if (pauseNS > 0) {
-        try {
-          Thread.sleep((int) (pauseNS/1000000), (int) (pauseNS % 1000000));
-        } catch (InterruptedException ie) {
-          throw new ThreadInterruptedException(ie);
+      // While loop because Thread.sleep doesn't always sleep
+      // enough:
+      while(true) {
+        final long pauseNS = targetNS - curNS;
+        if (pauseNS > 0) {
+          try {
+            Thread.sleep((int) (pauseNS/1000000), (int) (pauseNS % 1000000));
+          } catch (InterruptedException ie) {
+            throw new ThreadInterruptedException(ie);
+          }
+          curNS = System.nanoTime();
+          continue;
         }
-        curNS = System.nanoTime();
-        continue;
+        break;
       }
-      break;
+      return targetNS;
     }
   }
 }