You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2013/10/21 20:58:44 UTC

svn commit: r1534320 [15/39] - in /lucene/dev/branches/lucene4956: ./ dev-tools/ dev-tools/idea/.idea/ dev-tools/idea/lucene/expressions/ dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/expressions/...

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java Mon Oct 21 18:58:24 2013
@@ -61,8 +61,9 @@ final class StandardDirectoryReader exte
           } catch(IOException ex) {
             prior = ex;
           } finally {
-            if (!success)
+            if (!success) {
               IOUtils.closeWhileHandlingException(prior, readers);
+            }
           }
         }
         return new StandardDirectoryReader(directory, readers, null, sis, false);
@@ -82,10 +83,9 @@ final class StandardDirectoryReader exte
 
     final SegmentInfos segmentInfos = infos.clone();
     int infosUpto = 0;
-    for (int i=0;i<numSegments;i++) {
-      IOException prior = null;
-      boolean success = false;
-      try {
+    boolean success = false;
+    try {
+      for (int i = 0; i < numSegments; i++) {
         // NOTE: important that we use infos not
         // segmentInfos here, so that we are passing the
         // actual instance of SegmentInfoPerCommit in
@@ -100,23 +100,30 @@ final class StandardDirectoryReader exte
             readers.add(reader);
             infosUpto++;
           } else {
-            reader.close();
+            reader.decRef();
             segmentInfos.remove(infosUpto);
           }
         } finally {
           writer.readerPool.release(rld);
         }
-        success = true;
-      } catch(IOException ex) {
-        prior = ex;
-      } finally {
-        if (!success) {
-          IOUtils.closeWhileHandlingException(prior, readers);
+      }
+      StandardDirectoryReader result = new StandardDirectoryReader(dir,
+          readers.toArray(new SegmentReader[readers.size()]), writer,
+          segmentInfos, applyAllDeletes);
+      success = true;
+      return result;
+    } finally {
+      if (!success) {
+        for (SegmentReader r : readers) {
+          try {
+            r.decRef();
+          } catch (Throwable th) {
+            // ignore any exception that is thrown here to not mask any original
+            // exception. 
+          }
         }
       }
     }
-    return new StandardDirectoryReader(dir, readers.toArray(new SegmentReader[readers.size()]),
-      writer, segmentInfos, applyAllDeletes);
   }
 
   /** This constructor is only used for {@link #doOpenIfChanged(SegmentInfos)} */
@@ -162,18 +169,26 @@ final class StandardDirectoryReader exte
           readerShared[i] = false;
           newReaders[i] = newReader;
         } else {
-          if (newReaders[i].getSegmentInfo().getDelGen() == infos.info(i).getDelGen()) {
+          if (newReaders[i].getSegmentInfo().getDelGen() == infos.info(i).getDelGen()
+              && newReaders[i].getSegmentInfo().getFieldInfosGen() == infos.info(i).getFieldInfosGen()) {
             // No change; this reader will be shared between
             // the old and the new one, so we must incRef
             // it:
             readerShared[i] = true;
             newReaders[i].incRef();
           } else {
+            // there are changes to the reader, either liveDocs or DV updates
             readerShared[i] = false;
             // Steal the ref returned by SegmentReader ctor:
             assert infos.info(i).info.dir == newReaders[i].getSegmentInfo().info.dir;
-            assert infos.info(i).hasDeletions();
-            newReaders[i] = new SegmentReader(infos.info(i), newReaders[i].core, IOContext.READ);
+            assert infos.info(i).hasDeletions() || infos.info(i).hasFieldUpdates();
+            if (newReaders[i].getSegmentInfo().getDelGen() == infos.info(i).getDelGen()) {
+              // only DV updates
+              newReaders[i] = new SegmentReader(infos.info(i), newReaders[i], newReaders[i].getLiveDocs(), newReaders[i].numDocs());
+            } else {
+              // both DV and liveDocs have changed
+              newReaders[i] = new SegmentReader(infos.info(i), newReaders[i]);
+            }
           }
         }
         success = true;
@@ -200,12 +215,7 @@ final class StandardDirectoryReader exte
           }
         }
         // throw the first exception
-        if (prior != null) {
-          if (prior instanceof IOException) throw (IOException) prior;
-          if (prior instanceof RuntimeException) throw (RuntimeException) prior;
-          if (prior instanceof Error) throw (Error) prior;
-          throw new RuntimeException(prior);
-        }
+        IOUtils.reThrow(prior);
       }
     }    
     return new StandardDirectoryReader(directory, newReaders, null, infos, false);
@@ -345,7 +355,9 @@ final class StandardDirectoryReader exte
       try {
         r.decRef();
       } catch (Throwable t) {
-        if (firstExc == null) firstExc = t;
+        if (firstExc == null) {
+          firstExc = t;
+        }
       }
     }
 
@@ -356,12 +368,7 @@ final class StandardDirectoryReader exte
     }
 
     // throw the first exception
-    if (firstExc != null) {
-      if (firstExc instanceof IOException) throw (IOException) firstExc;
-      if (firstExc instanceof RuntimeException) throw (RuntimeException) firstExc;
-      if (firstExc instanceof Error) throw (Error) firstExc;
-      throw new RuntimeException(firstExc);
-    }
+    IOUtils.reThrow(firstExc);
   }
 
   @Override

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java Mon Oct 21 18:58:24 2013
@@ -66,12 +66,6 @@ final class TermVectorsConsumer extends 
         hasVectors = false;
       }
     }
-
-    for (final TermsHashConsumerPerField field : fieldsToFlush.values() ) {
-      TermVectorsConsumerPerField perField = (TermVectorsConsumerPerField) field;
-      perField.termsHashPerField.reset();
-      perField.shrinkHash();
-    }
   }
 
   /** Fills in no-term-vectors for all docs we haven't seen

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java Mon Oct 21 18:58:24 2013
@@ -156,7 +156,7 @@ final class TermVectorsConsumerPerField 
     TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
     final TermVectorsWriter tv = termsWriter.writer;
 
-    final int[] termIDs = termsHashPerField.sortPostings(tv.getComparator());
+    final int[] termIDs = termsHashPerField.sortPostings();
 
     tv.startField(fieldInfo, numPostings, doVectorPositions, doVectorOffsets, hasPayloads);
     
@@ -191,11 +191,6 @@ final class TermVectorsConsumerPerField 
     fieldInfo.setStoreTermVectors();
   }
 
-  void shrinkHash() {
-    termsHashPerField.shrinkHash(maxNumPostings);
-    maxNumPostings = 0;
-  }
-
   @Override
   void start(IndexableField f) {
     if (doVectorOffsets) {

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/Terms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/Terms.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/Terms.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/Terms.java Mon Oct 21 18:58:24 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.util.Comparator;
 
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
@@ -75,13 +74,6 @@ public abstract class Terms {
     }
   }
 
-  /** Return the BytesRef Comparator used to sort terms
-   *  provided by the iterator.  This method may return null
-   *  if there are no terms.  This method may be invoked
-   *  many times; it's best to cache a single instance &
-   *  reuse it. */
-  public abstract Comparator<BytesRef> getComparator();
-
   /** Returns the number of terms for this field, or -1 if this 
    *  measure isn't stored by the codec. Note that, just like 
    *  other term measures, this measure does not take deleted 
@@ -109,7 +101,11 @@ public abstract class Terms {
    *  measures, this measure does not take deleted documents
    *  into account. */
   public abstract int getDocCount() throws IOException;
-  
+
+  /** Returns true if documents in this field store
+   *  per-document term frequency ({@link DocsEnum#freq}). */
+  public abstract boolean hasFreqs();
+
   /** Returns true if documents in this field store offsets. */
   public abstract boolean hasOffsets();
   

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java Mon Oct 21 18:58:24 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.util.Comparator;
 
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.Bits;
@@ -33,8 +32,9 @@ import org.apache.lucene.util.BytesRefIt
  * #docs}.
  * 
  * <p>Term enumerations are always ordered by
- * {@link #getComparator}.  Each term in the enumeration is
- * greater than the one before it.</p>
+ * BytesRef.compareTo, which is Unicode sort
+ * order if the terms are UTF-8 bytes.  Each term in the
+ * enumeration is greater than the one before it.</p>
  *
  * <p>The TermsEnum is unpositioned when you first obtain it
  * and you must first successfully call {@link #next} or one
@@ -230,11 +230,6 @@ public abstract class TermsEnum implemen
     }
 
     @Override
-    public Comparator<BytesRef> getComparator() {
-      return null;
-    }
-      
-    @Override
     public int docFreq() {
       throw new IllegalStateException("this method should never be called");
     }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java Mon Oct 21 18:58:24 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.util.Comparator;
 
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.util.ByteBlockPool;
@@ -77,13 +76,7 @@ final class TermsHashPerField extends In
       nextPerField = null;
   }
 
-  void shrinkHash(int targetSize) {
-    // Fully free the bytesHash on each flush but keep the pool untouched
-    // bytesHash.clear will clear the ByteStartArray and in turn the ParallelPostingsArray too
-    bytesHash.clear(false);
-  }
-
-  public void reset() {
+  void reset() {
     bytesHash.clear(false);
     if (nextPerField != null)
       nextPerField.reset();
@@ -107,8 +100,8 @@ final class TermsHashPerField extends In
   }
 
   /** Collapse the hash table & sort in-place. */
-  public int[] sortPostings(Comparator<BytesRef> termComp) {
-   return bytesHash.sort(termComp);
+  public int[] sortPostings() {
+    return bytesHash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
   }
 
   private boolean doCall;
@@ -136,7 +129,8 @@ final class TermsHashPerField extends In
 
   // Secondary entry point (for 2nd & subsequent TermsHash),
   // because token text has already been "interned" into
-  // textStart, so we hash by textStart
+  // textStart, so we hash by textStart.  term vectors use
+  // this API.
   public void add(int textStart) throws IOException {
     int termID = bytesHash.addByPoolOffset(textStart);
     if (termID >= 0) {      // New posting
@@ -173,7 +167,8 @@ final class TermsHashPerField extends In
     }
   }
 
-  // Primary entry point (for first TermsHash)
+  // Primary entry point (for first TermsHash); postings use
+  // this API.
   @Override
   void add() throws IOException {
 

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/CachingWrapperFilter.java Mon Oct 21 18:58:24 2013
@@ -27,8 +27,8 @@ import java.util.WeakHashMap;
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.WAH8DocIdSet;
 
 /**
  * Wraps another {@link Filter}'s result and caches it.  The purpose is to allow
@@ -36,9 +36,6 @@ import org.apache.lucene.util.RamUsageEs
  * to add caching.
  */
 public class CachingWrapperFilter extends Filter {
-  // TODO: make this filter aware of ReaderContext. a cached filter could 
-  // specify the actual readers key or something similar to indicate on which
-  // level of the readers hierarchy it should be cached.
   private final Filter filter;
   private final Map<Object,DocIdSet> cache = Collections.synchronizedMap(new WeakHashMap<Object,DocIdSet>());
 
@@ -49,11 +46,19 @@ public class CachingWrapperFilter extend
     this.filter = filter;
   }
 
+  /**
+   * Gets the contained filter.
+   * @return the contained filter.
+   */
+  public Filter getFilter() {
+    return filter;
+  }
+
   /** 
    *  Provide the DocIdSet to be cached, using the DocIdSet provided
    *  by the wrapped Filter. <p>This implementation returns the given {@link DocIdSet},
-   *  if {@link DocIdSet#isCacheable} returns <code>true</code>, else it copies the 
-   *  {@link DocIdSetIterator} into a {@link FixedBitSet}.
+   *  if {@link DocIdSet#isCacheable} returns <code>true</code>, else it calls
+   *  {@link #cacheImpl(DocIdSetIterator,AtomicReader)}
    *  <p>Note: This method returns {@linkplain #EMPTY_DOCIDSET} if the given docIdSet
    *  is <code>null</code> or if {@link DocIdSet#iterator()} return <code>null</code>. The empty
    *  instance is use as a placeholder in the cache instead of the <code>null</code> value.
@@ -72,12 +77,19 @@ public class CachingWrapperFilter extend
       if (it == null) {
         return EMPTY_DOCIDSET;
       } else {
-        final FixedBitSet bits = new FixedBitSet(reader.maxDoc());
-        bits.or(it);
-        return bits;
+        return cacheImpl(it, reader);
       }
     }
   }
+  
+  /**
+   * Default cache implementation: uses {@link WAH8DocIdSet}.
+   */
+  protected DocIdSet cacheImpl(DocIdSetIterator iterator, AtomicReader reader) throws IOException {
+    WAH8DocIdSet.Builder builder = new WAH8DocIdSet.Builder();
+    builder.add(iterator);
+    return builder.build();
+  }
 
   // for testing
   int hitCount, missCount;
@@ -93,6 +105,7 @@ public class CachingWrapperFilter extend
     } else {
       missCount++;
       docIdSet = docIdSetToCache(filter.getDocIdSet(context, null), reader);
+      assert docIdSet.isCacheable();
       cache.put(key, docIdSet);
     }
 
@@ -101,19 +114,19 @@ public class CachingWrapperFilter extend
   
   @Override
   public String toString() {
-    return "CachingWrapperFilter("+filter+")";
+    return getClass().getSimpleName() + "("+filter+")";
   }
 
   @Override
   public boolean equals(Object o) {
-    if (!(o instanceof CachingWrapperFilter)) return false;
+    if (o == null || !getClass().equals(o.getClass())) return false;
     final CachingWrapperFilter other = (CachingWrapperFilter) o;
     return this.filter.equals(other.filter);
   }
 
   @Override
   public int hashCode() {
-    return (filter.hashCode() ^ 0x1117BF25);
+    return (filter.hashCode() ^ getClass().hashCode());
   }
   
   /** An empty {@code DocIdSet} instance */

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java Mon Oct 21 18:58:24 2013
@@ -96,17 +96,17 @@ class ConstantScoreAutoRewrite extends T
     final int size = col.pendingTerms.size();
     if (col.hasCutOff) {
       return MultiTermQuery.CONSTANT_SCORE_FILTER_REWRITE.rewrite(reader, query);
-    } else if (size == 0) {
-      return getTopLevelQuery();
     } else {
       final BooleanQuery bq = getTopLevelQuery();
-      final BytesRefHash pendingTerms = col.pendingTerms;
-      final int sort[] = pendingTerms.sort(col.termsEnum.getComparator());
-      for(int i = 0; i < size; i++) {
-        final int pos = sort[i];
-        // docFreq is not used for constant score here, we pass 1
-        // to explicitely set a fake value, so it's not calculated
-        addClause(bq, new Term(query.field, pendingTerms.get(pos, new BytesRef())), 1, 1.0f, col.array.termState[pos]);
+      if (size > 0) {
+        final BytesRefHash pendingTerms = col.pendingTerms;
+        final int sort[] = pendingTerms.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
+        for(int i = 0; i < size; i++) {
+          final int pos = sort[i];
+          // docFreq is not used for constant score here, we pass 1
+          // to explicitely set a fake value, so it's not calculated
+          addClause(bq, new Term(query.field, pendingTerms.get(pos, new BytesRef())), 1, 1.0f, col.array.termState[pos]);
+        }
       }
       // Strip scores
       final Query result = new ConstantScoreQuery(bq);

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java Mon Oct 21 18:58:24 2013
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.lucene.index.AtomicReaderContext;
@@ -154,17 +155,20 @@ public class DisjunctionMaxQuery extends
     @Override
     public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
         boolean topScorer, Bits acceptDocs) throws IOException {
-      Scorer[] scorers = new Scorer[weights.size()];
-      int idx = 0;
+      List<Scorer> scorers = new ArrayList<Scorer>();
       for (Weight w : weights) {
         // we will advance() subscorers
         Scorer subScorer = w.scorer(context, true, false, acceptDocs);
         if (subScorer != null) {
-          scorers[idx++] = subScorer;
+          scorers.add(subScorer);
+
         }
       }
-      if (idx == 0) return null; // all scorers did not have documents
-      DisjunctionMaxScorer result = new DisjunctionMaxScorer(this, tieBreakerMultiplier, scorers, idx);
+      if (scorers.isEmpty()) {
+        // no sub-scorers had any documents
+        return null;
+      }
+      DisjunctionMaxScorer result = new DisjunctionMaxScorer(this, tieBreakerMultiplier, scorers.toArray(new Scorer[scorers.size()]));
       return result;
     }
 

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java Mon Oct 21 18:58:24 2013
@@ -27,7 +27,6 @@ import java.io.IOException;
 class DisjunctionMaxScorer extends DisjunctionScorer {
   /* Multiplier applied to non-maximum-scoring subqueries for a document as they are summed into the result. */
   private final float tieBreakerMultiplier;
-  private int doc = -1;
   private int freq = -1;
 
   /* Used when scoring currently matching doc. */
@@ -44,40 +43,13 @@ class DisjunctionMaxScorer extends Disju
    *          document as they are summed into the result.
    * @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(Weight weight, float tieBreakerMultiplier,
-      Scorer[] subScorers, int numScorers) {
-    super(weight, subScorers, numScorers);
+      Scorer[] subScorers) {
+    super(weight, subScorers);
     this.tieBreakerMultiplier = tieBreakerMultiplier;
   }
 
-  @Override
-  public int nextDoc() throws IOException {
-    assert doc != NO_MORE_DOCS;
-    while(true) {
-      if (subScorers[0].nextDoc() != NO_MORE_DOCS) {
-        heapAdjust(0);
-      } else {
-        heapRemoveRoot();
-        if (numScorers == 0) {
-          return doc = NO_MORE_DOCS;
-        }
-      }
-      if (subScorers[0].docID() != doc) {
-        afterNext();
-        return doc;
-      }
-    }
-  }
-
-  @Override
-  public int docID() {
-    return doc;
-  }
-
   /** Determine the current document score.  Initially invalid, until {@link #nextDoc()} is called the first time.
    * @return the score of the current generated document
    */
@@ -86,7 +58,8 @@ class DisjunctionMaxScorer extends Disju
     return scoreMax + (scoreSum - scoreMax) * tieBreakerMultiplier;
   }
   
-  private void afterNext() throws IOException {
+  @Override
+  protected void afterNext() throws IOException {
     doc = subScorers[0].docID();
     if (doc != NO_MORE_DOCS) {
       scoreSum = scoreMax = subScorers[0].score();
@@ -112,23 +85,4 @@ class DisjunctionMaxScorer extends Disju
   public int freq() throws IOException {
     return freq;
   }
-
-  @Override
-  public int advance(int target) throws IOException {
-    assert doc != NO_MORE_DOCS;
-    while(true) {
-      if (subScorers[0].advance(target) != NO_MORE_DOCS) {
-        heapAdjust(0);
-      } else {
-        heapRemoveRoot();
-        if (numScorers == 0) {
-          return doc = NO_MORE_DOCS;
-        }
-      }
-      if (subScorers[0].docID() >= target) {
-        afterNext();
-        return doc;
-      }
-    }
-  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java Mon Oct 21 18:58:24 2013
@@ -17,6 +17,7 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 
@@ -26,12 +27,14 @@ import java.util.Collection;
  */
 abstract class DisjunctionScorer extends Scorer {
   protected final Scorer subScorers[];
+  /** The document number of the current match. */
+  protected int doc = -1;
   protected int numScorers;
   
-  protected DisjunctionScorer(Weight weight, Scorer subScorers[], int numScorers) {
+  protected DisjunctionScorer(Weight weight, Scorer subScorers[]) {
     super(weight);
     this.subScorers = subScorers;
-    this.numScorers = numScorers;
+    this.numScorers = subScorers.length;
     heapify();
   }
   
@@ -114,4 +117,59 @@ abstract class DisjunctionScorer extends
     }
     return sum;
   } 
+  
+  @Override
+  public int docID() {
+   return doc;
+  }
+ 
+  @Override
+  public int nextDoc() throws IOException {
+    assert doc != NO_MORE_DOCS;
+    while(true) {
+      if (subScorers[0].nextDoc() != NO_MORE_DOCS) {
+        heapAdjust(0);
+      } else {
+        heapRemoveRoot();
+        if (numScorers == 0) {
+          return doc = NO_MORE_DOCS;
+        }
+      }
+      if (subScorers[0].docID() != doc) {
+        afterNext();
+        return doc;
+      }
+    }
+  }
+  
+  @Override
+  public int advance(int target) throws IOException {
+    assert doc != NO_MORE_DOCS;
+    while(true) {
+      if (subScorers[0].advance(target) != NO_MORE_DOCS) {
+        heapAdjust(0);
+      } else {
+        heapRemoveRoot();
+        if (numScorers == 0) {
+          return doc = NO_MORE_DOCS;
+        }
+      }
+      if (subScorers[0].docID() >= target) {
+        afterNext();
+        return doc;
+      }
+    }
+  }
+  
+  /** 
+   * Called after next() or advance() land on a new document.
+   * <p>
+   * {@code subScorers[0]} will be positioned to the new docid,
+   * which could be {@code NO_MORE_DOCS} (subclass must handle this).
+   * <p>
+   * implementations should assign {@code doc} appropriately, and do any
+   * other work necessary to implement {@code score()} and {@code freq()}
+   */
+  // TODO: make this less horrible
+  protected abstract void afterNext() throws IOException;
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java Mon Oct 21 18:58:24 2013
@@ -23,8 +23,6 @@ import java.io.IOException;
  * This Scorer implements {@link Scorer#advance(int)} and uses advance() on the given Scorers. 
  */
 class DisjunctionSumScorer extends DisjunctionScorer { 
-  /** The document number of the current match. */
-  private int doc = -1;
 
   /** The number of subscorers that provide the current match. */
   protected int nrMatchers = -1;
@@ -38,34 +36,16 @@ class DisjunctionSumScorer extends Disju
    * @param coord Table of coordination factors
    */
   DisjunctionSumScorer(Weight weight, Scorer[] subScorers, float[] coord) throws IOException {
-    super(weight, subScorers, subScorers.length);
+    super(weight, subScorers);
 
     if (numScorers <= 1) {
       throw new IllegalArgumentException("There must be at least 2 subScorers");
     }
     this.coord = coord;
   }
-
-  @Override
-  public int nextDoc() throws IOException {
-    assert doc != NO_MORE_DOCS;
-    while(true) {
-      if (subScorers[0].nextDoc() != NO_MORE_DOCS) {
-        heapAdjust(0);
-      } else {
-        heapRemoveRoot();
-        if (numScorers == 0) {
-          return doc = NO_MORE_DOCS;
-        }
-      }
-      if (subScorers[0].docID() != doc) {
-        afterNext();
-        return doc;
-      }
-    }
-  }
   
-  private void afterNext() throws IOException {
+  @Override
+  protected void afterNext() throws IOException {
     final Scorer sub = subScorers[0];
     doc = sub.docID();
     if (doc != NO_MORE_DOCS) {
@@ -96,43 +76,9 @@ class DisjunctionSumScorer extends Disju
   public float score() throws IOException { 
     return (float)score * coord[nrMatchers]; 
   }
-   
-  @Override
-  public int docID() {
-    return doc;
-  }
 
   @Override
   public int freq() throws IOException {
     return nrMatchers;
   }
-
-  /**
-   * Advances to the first match beyond the current whose document number is
-   * greater than or equal to a given target. <br>
-   * The implementation uses the advance() method on the subscorers.
-   * 
-   * @param target
-   *          The target document number.
-   * @return the document whose number is greater than or equal to the given
-   *         target, or -1 if none exist.
-   */
-  @Override
-  public int advance(int target) throws IOException {
-    assert doc != NO_MORE_DOCS;
-    while(true) {
-      if (subScorers[0].advance(target) != NO_MORE_DOCS) {
-        heapAdjust(0);
-      } else {
-        heapRemoveRoot();
-        if (numScorers == 0) {
-          return doc = NO_MORE_DOCS;
-        }
-      }
-      if (subScorers[0].docID() >= target) {
-        afterNext();
-        return doc;
-      }
-    }
-  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DocIdSet.java Mon Oct 21 18:58:24 2013
@@ -50,7 +50,7 @@ public abstract class DocIdSet {
 
   /**
    * This method is a hint for {@link CachingWrapperFilter}, if this <code>DocIdSet</code>
-   * should be cached without copying it into a BitSet. The default is to return
+   * should be cached without copying it. The default is to return
    * <code>false</code>. If you have an own <code>DocIdSet</code> implementation
    * that does its iteration very effective and fast without doing disk I/O,
    * override this method and return <code>true</code>.

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRewriteMethod.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRewriteMethod.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRewriteMethod.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/DocTermOrdsRewriteMethod.java Mon Oct 21 18:58:24 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.util.Comparator;
 
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
@@ -26,7 +25,6 @@ import org.apache.lucene.index.SortedSet
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.OpenBitSet;
 
 /**
@@ -91,11 +89,6 @@ public final class DocTermOrdsRewriteMet
       TermsEnum termsEnum = query.getTermsEnum(new Terms() {
         
         @Override
-        public Comparator<BytesRef> getComparator() {
-          return BytesRef.getUTF8SortedAsUnicodeComparator();
-        }
-        
-        @Override
         public TermsEnum iterator(TermsEnum reuse) {
           return docTermOrds.termsEnum();
         }
@@ -121,6 +114,11 @@ public final class DocTermOrdsRewriteMet
         }
 
         @Override
+        public boolean hasFreqs() {
+          return false;
+        }
+
+        @Override
         public boolean hasOffsets() {
           return false;
         }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FieldCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FieldCache.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FieldCache.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FieldCache.java Mon Oct 21 18:58:24 2013
@@ -20,14 +20,16 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.io.PrintStream;
 
-import org.apache.lucene.analysis.NumericTokenStream; // for javadocs
-import org.apache.lucene.document.DoubleField; // for javadocs
-import org.apache.lucene.document.FloatField; // for javadocs
-import org.apache.lucene.document.IntField; // for javadocs
-import org.apache.lucene.document.LongField; // for javadocs
+import org.apache.lucene.analysis.NumericTokenStream;
+import org.apache.lucene.document.DoubleField;
+import org.apache.lucene.document.FloatField;
+import org.apache.lucene.document.IntField;
+import org.apache.lucene.document.LongField;
+import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DocTermOrds;
+import org.apache.lucene.index.IndexReader; // javadocs
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.Terms;
@@ -104,26 +106,6 @@ public interface FieldCache {
       }
     };
   }
-  
-  /** Returns MISSING/-1 ordinal for every document */
-  public static final SortedDocValues EMPTY_TERMSINDEX = new SortedDocValues() {
-    @Override
-    public int getOrd(int docID) {
-      return -1;
-    }
-
-    @Override
-    public void lookupOrd(int ord, BytesRef result) {
-      result.bytes = MISSING;
-      result.offset = 0;
-      result.length = 0;
-    }
-
-    @Override
-    public int getValueCount() {
-      return 0;
-    }
-  };
 
   /**
    * Placeholder indicating creation of this cache is currently in-progress.
@@ -266,132 +248,146 @@ public interface FieldCache {
     }
   };
   
- 
   /** Checks the internal cache for an appropriate entry, and if none is found,
    *  reads the terms in <code>field</code> and returns a bit set at the size of
    *  <code>reader.maxDoc()</code>, with turned on bits for each docid that 
-   *  does have a value for this field.  Note that if the field was only indexed
-   *  as DocValues then this method will not work (it will return a Bits stating
-   *  that no documents contain the field).
+   *  does have a value for this field.
    */
   public Bits getDocsWithField(AtomicReader reader, String field) throws IOException;
 
-  /** Checks the internal cache for an appropriate entry, and if none is
-   * found, reads the terms in <code>field</code> as integers and returns an array
-   * of size <code>reader.maxDoc()</code> of the value each document
-   * has in the given field.
-   * @param reader  Used to get field values.
-   * @param field   Which field contains the integers.
-   * @param setDocsWithField  If true then {@link #getDocsWithField} will
-   *        also be computed and stored in the FieldCache.
-   * @return The values in the given field for each document.
-   * @throws IOException  If any error occurs.
-   */
-  public Ints getInts (AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
-
-  /** Checks the internal cache for an appropriate entry, and if none is found,
-   * reads the terms in <code>field</code> as integers and returns an array of
-   * size <code>reader.maxDoc()</code> of the value each document has in the
-   * given field.
-   * @param reader  Used to get field values.
-   * @param field   Which field contains the integers.
-   * @param parser  Computes integer for string values.
-   * @param setDocsWithField  If true then {@link #getDocsWithField} will
-   *        also be computed and stored in the FieldCache.
-   * @return The values in the given field for each document.
-   * @throws IOException  If any error occurs.
+  /**
+   * Returns an {@link Ints} over the values found in documents in the given
+   * field.
+   *
+   * @see #getInts(AtomicReader, String, IntParser, boolean)
    */
-  public Ints getInts (AtomicReader reader, String field, IntParser parser, boolean setDocsWithField) throws IOException;
+  public Ints getInts(AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
 
-  /** Checks the internal cache for an appropriate entry, and if
-   * none is found, reads the terms in <code>field</code> as floats and returns an array
-   * of size <code>reader.maxDoc()</code> of the value each document
+  /**
+   * Returns an {@link Ints} over the values found in documents in the given
+   * field. If the field was indexed as {@link NumericDocValuesField}, it simply
+   * uses {@link AtomicReader#getNumericDocValues(String)} to read the values.
+   * Otherwise, it checks the internal cache for an appropriate entry, and if
+   * none is found, reads the terms in <code>field</code> as ints and returns
+   * an array of size <code>reader.maxDoc()</code> of the value each document
    * has in the given field.
-   * @param reader  Used to get field values.
-   * @param field   Which field contains the floats.
-   * @param setDocsWithField  If true then {@link #getDocsWithField} will
-   *        also be computed and stored in the FieldCache.
+   * 
+   * @param reader
+   *          Used to get field values.
+   * @param field
+   *          Which field contains the longs.
+   * @param parser
+   *          Computes int for string values. May be {@code null} if the
+   *          requested field was indexed as {@link NumericDocValuesField} or
+   *          {@link IntField}.
+   * @param setDocsWithField
+   *          If true then {@link #getDocsWithField} will also be computed and
+   *          stored in the FieldCache.
    * @return The values in the given field for each document.
-   * @throws IOException  If any error occurs.
+   * @throws IOException
+   *           If any error occurs.
    */
-  public Floats getFloats (AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
+  public Ints getInts(AtomicReader reader, String field, IntParser parser, boolean setDocsWithField) throws IOException;
 
-  /** Checks the internal cache for an appropriate entry, and if
-   * none is found, reads the terms in <code>field</code> as floats and returns an array
-   * of size <code>reader.maxDoc()</code> of the value each document
-   * has in the given field.
-   * @param reader  Used to get field values.
-   * @param field   Which field contains the floats.
-   * @param parser  Computes float for string values.
-   * @param setDocsWithField  If true then {@link #getDocsWithField} will
-   *        also be computed and stored in the FieldCache.
-   * @return The values in the given field for each document.
-   * @throws IOException  If any error occurs.
+  /**
+   * Returns a {@link Floats} over the values found in documents in the given
+   * field.
+   *
+   * @see #getFloats(AtomicReader, String, FloatParser, boolean)
    */
-  public Floats getFloats (AtomicReader reader, String field, FloatParser parser, boolean setDocsWithField) throws IOException;
+  public Floats getFloats(AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
 
   /**
-   * Checks the internal cache for an appropriate entry, and if none is
-   * found, reads the terms in <code>field</code> as longs and returns an array
-   * of size <code>reader.maxDoc()</code> of the value each document
+   * Returns a {@link Floats} over the values found in documents in the given
+   * field. If the field was indexed as {@link NumericDocValuesField}, it simply
+   * uses {@link AtomicReader#getNumericDocValues(String)} to read the values.
+   * Otherwise, it checks the internal cache for an appropriate entry, and if
+   * none is found, reads the terms in <code>field</code> as floats and returns
+   * an array of size <code>reader.maxDoc()</code> of the value each document
    * has in the given field.
-   *
-   * @param reader Used to get field values.
-   * @param field  Which field contains the longs.
-   * @param setDocsWithField  If true then {@link #getDocsWithField} will
-   *        also be computed and stored in the FieldCache.
+   * 
+   * @param reader
+   *          Used to get field values.
+   * @param field
+   *          Which field contains the floats.
+   * @param parser
+   *          Computes float for string values. May be {@code null} if the
+   *          requested field was indexed as {@link NumericDocValuesField} or
+   *          {@link FloatField}.
+   * @param setDocsWithField
+   *          If true then {@link #getDocsWithField} will also be computed and
+   *          stored in the FieldCache.
    * @return The values in the given field for each document.
-   * @throws java.io.IOException If any error occurs.
+   * @throws IOException
+   *           If any error occurs.
    */
-  public Longs getLongs(AtomicReader reader, String field, boolean setDocsWithField)
-          throws IOException;
+  public Floats getFloats(AtomicReader reader, String field, FloatParser parser, boolean setDocsWithField) throws IOException;
 
   /**
-   * Checks the internal cache for an appropriate entry, and if none is found,
-   * reads the terms in <code>field</code> as longs and returns an array of
-   * size <code>reader.maxDoc()</code> of the value each document has in the
-   * given field.
+   * Returns a {@link Longs} over the values found in documents in the given
+   * field.
    *
-   * @param reader Used to get field values.
-   * @param field  Which field contains the longs.
-   * @param parser Computes integer for string values.
-   * @param setDocsWithField  If true then {@link #getDocsWithField} will
-   *        also be computed and stored in the FieldCache.
-   * @return The values in the given field for each document.
-   * @throws IOException If any error occurs.
+   * @see #getLongs(AtomicReader, String, LongParser, boolean)
    */
-  public Longs getLongs(AtomicReader reader, String field, LongParser parser, boolean setDocsWithField)
-          throws IOException;
+  public Longs getLongs(AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
 
   /**
-   * Checks the internal cache for an appropriate entry, and if none is
-   * found, reads the terms in <code>field</code> as integers and returns an array
-   * of size <code>reader.maxDoc()</code> of the value each document
+   * Returns a {@link Longs} over the values found in documents in the given
+   * field. If the field was indexed as {@link NumericDocValuesField}, it simply
+   * uses {@link AtomicReader#getNumericDocValues(String)} to read the values.
+   * Otherwise, it checks the internal cache for an appropriate entry, and if
+   * none is found, reads the terms in <code>field</code> as longs and returns
+   * an array of size <code>reader.maxDoc()</code> of the value each document
    * has in the given field.
-   *
-   * @param reader Used to get field values.
-   * @param field  Which field contains the doubles.
-   * @param setDocsWithField  If true then {@link #getDocsWithField} will
-   *        also be computed and stored in the FieldCache.
+   * 
+   * @param reader
+   *          Used to get field values.
+   * @param field
+   *          Which field contains the longs.
+   * @param parser
+   *          Computes long for string values. May be {@code null} if the
+   *          requested field was indexed as {@link NumericDocValuesField} or
+   *          {@link LongField}.
+   * @param setDocsWithField
+   *          If true then {@link #getDocsWithField} will also be computed and
+   *          stored in the FieldCache.
    * @return The values in the given field for each document.
-   * @throws IOException If any error occurs.
+   * @throws IOException
+   *           If any error occurs.
    */
-  public Doubles getDoubles(AtomicReader reader, String field, boolean setDocsWithField)
-          throws IOException;
+  public Longs getLongs(AtomicReader reader, String field, LongParser parser, boolean setDocsWithField) throws IOException;
 
   /**
-   * Checks the internal cache for an appropriate entry, and if none is found,
-   * reads the terms in <code>field</code> as doubles and returns an array of
-   * size <code>reader.maxDoc()</code> of the value each document has in the
-   * given field.
+   * Returns a {@link Doubles} over the values found in documents in the given
+   * field.
    *
-   * @param reader Used to get field values.
-   * @param field  Which field contains the doubles.
-   * @param parser Computes integer for string values.
-   * @param setDocsWithField  If true then {@link #getDocsWithField} will
-   *        also be computed and stored in the FieldCache.
+   * @see #getDoubles(AtomicReader, String, DoubleParser, boolean)
+   */
+  public Doubles getDoubles(AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
+
+  /**
+   * Returns a {@link Doubles} over the values found in documents in the given
+   * field. If the field was indexed as {@link NumericDocValuesField}, it simply
+   * uses {@link AtomicReader#getNumericDocValues(String)} to read the values.
+   * Otherwise, it checks the internal cache for an appropriate entry, and if
+   * none is found, reads the terms in <code>field</code> as doubles and returns
+   * an array of size <code>reader.maxDoc()</code> of the value each document
+   * has in the given field.
+   * 
+   * @param reader
+   *          Used to get field values.
+   * @param field
+   *          Which field contains the longs.
+   * @param parser
+   *          Computes double for string values. May be {@code null} if the
+   *          requested field was indexed as {@link NumericDocValuesField} or
+   *          {@link DoubleField}.
+   * @param setDocsWithField
+   *          If true then {@link #getDocsWithField} will also be computed and
+   *          stored in the FieldCache.
    * @return The values in the given field for each document.
-   * @throws IOException If any error occurs.
+   * @throws IOException
+   *           If any error occurs.
    */
   public Doubles getDoubles(AtomicReader reader, String field, DoubleParser parser, boolean setDocsWithField) throws IOException;
 
@@ -401,18 +397,19 @@ public interface FieldCache {
    * method to retrieve the term (as a BytesRef) per document.
    * @param reader  Used to get field values.
    * @param field   Which field contains the strings.
+   * @param setDocsWithField  If true then {@link #getDocsWithField} will
+   *        also be computed and stored in the FieldCache.
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public BinaryDocValues getTerms (AtomicReader reader, String field)
-  throws IOException;
+  public BinaryDocValues getTerms(AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
 
-  /** Expert: just like {@link #getTerms(AtomicReader,String)},
+  /** Expert: just like {@link #getTerms(AtomicReader,String,boolean)},
    *  but you can specify whether more RAM should be consumed in exchange for
    *  faster lookups (default is "true").  Note that the
    *  first call for a given reader and field "wins",
    *  subsequent calls will share the same cache entry. */
-  public BinaryDocValues getTerms (AtomicReader reader, String field, float acceptableOverheadRatio) throws IOException;
+  public BinaryDocValues getTerms(AtomicReader reader, String field, boolean setDocsWithField, float acceptableOverheadRatio) throws IOException;
 
   /** Checks the internal cache for an appropriate entry, and if none
    * is found, reads the term values in <code>field</code>
@@ -424,7 +421,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public SortedDocValues getTermsIndex (AtomicReader reader, String field) throws IOException;
+  public SortedDocValues getTermsIndex(AtomicReader reader, String field) throws IOException;
 
   /** Expert: just like {@link
    *  #getTermsIndex(AtomicReader,String)}, but you can specify
@@ -432,7 +429,7 @@ public interface FieldCache {
    *  faster lookups (default is "true").  Note that the
    *  first call for a given reader and field "wins",
    *  subsequent calls will share the same cache entry. */
-  public SortedDocValues getTermsIndex (AtomicReader reader, String field, float acceptableOverheadRatio) throws IOException;
+  public SortedDocValues getTermsIndex(AtomicReader reader, String field, float acceptableOverheadRatio) throws IOException;
 
   /**
    * Checks the internal cache for an appropriate entry, and if none is found, reads the term values
@@ -537,7 +534,7 @@ public interface FieldCache {
    * </p>
    * @lucene.experimental
    */
-  public abstract CacheEntry[] getCacheEntries();
+  public CacheEntry[] getCacheEntries();
 
   /**
    * <p>
@@ -550,16 +547,17 @@ public interface FieldCache {
    * </p>
    * @lucene.experimental
    */
-  public abstract void purgeAllCaches();
+  public void purgeAllCaches();
 
   /**
    * Expert: drops all cache entries associated with this
-   * reader.  NOTE: this reader must precisely match the
-   * reader that the cache entry is keyed on. If you pass a
-   * top-level reader, it usually will have no effect as
-   * Lucene now caches at the segment reader level.
+   * reader {@link IndexReader#getCoreCacheKey}.  NOTE: this cache key must
+   * precisely match the reader that the cache entry is
+   * keyed on. If you pass a top-level reader, it usually
+   * will have no effect as Lucene now caches at the segment
+   * reader level.
    */
-  public abstract void purge(AtomicReader r);
+  public void purgeByCacheKey(Object coreCacheKey);
 
   /**
    * If non-null, FieldCacheImpl will warn whenever

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java Mon Oct 21 18:58:24 2013
@@ -78,9 +78,9 @@ class FieldCacheImpl implements FieldCac
   }
 
   @Override
-  public synchronized void purge(AtomicReader r) {
+  public synchronized void purgeByCacheKey(Object coreCacheKey) {
     for(Cache c : caches.values()) {
-      c.purge(r);
+      c.purgeByCacheKey(coreCacheKey);
     }
   }
 
@@ -110,8 +110,8 @@ class FieldCacheImpl implements FieldCac
   // per-segment fieldcaches don't purge until the shared core closes.
   final SegmentReader.CoreClosedListener purgeCore = new SegmentReader.CoreClosedListener() {
     @Override
-    public void onClose(SegmentReader owner) {
-      FieldCacheImpl.this.purge(owner);
+    public void onClose(Object ownerCoreCacheKey) {
+      FieldCacheImpl.this.purgeByCacheKey(ownerCoreCacheKey);
     }
   };
 
@@ -120,7 +120,7 @@ class FieldCacheImpl implements FieldCac
     @Override
     public void onClose(IndexReader owner) {
       assert owner instanceof AtomicReader;
-      FieldCacheImpl.this.purge((AtomicReader) owner);
+      FieldCacheImpl.this.purgeByCacheKey(((AtomicReader) owner).getCoreCacheKey());
     }
   };
   
@@ -155,10 +155,9 @@ class FieldCacheImpl implements FieldCac
         throws IOException;
 
     /** Remove this reader from the cache, if present. */
-    public void purge(AtomicReader r) {
-      Object readerKey = r.getCoreCacheKey();
+    public void purgeByCacheKey(Object coreCacheKey) {
       synchronized(readerCache) {
-        readerCache.remove(readerKey);
+        readerCache.remove(coreCacheKey);
       }
     }
 
@@ -353,12 +352,12 @@ class FieldCacheImpl implements FieldCac
     caches.get(DocsWithFieldCache.class).put(reader, new CacheKey(field, null), bits);
   }
 
-  // inherit javadocs
+  @Override
   public Ints getInts (AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
     return getInts(reader, field, null, setDocsWithField);
   }
 
-  // inherit javadocs
+  @Override
   public Ints getInts(AtomicReader reader, String field, IntParser parser, boolean setDocsWithField)
       throws IOException {
     final NumericDocValues valuesIn = reader.getNumericDocValues(field);
@@ -501,8 +500,7 @@ class FieldCacheImpl implements FieldCac
       // field does not exist or has no value
       return new Bits.MatchNoBits(reader.maxDoc());
     } else if (fieldInfo.hasDocValues()) {
-      // doc values are dense
-      return new Bits.MatchAllBits(reader.maxDoc());
+      return reader.getDocsWithField(field);
     } else if (!fieldInfo.isIndexed()) {
       return new Bits.MatchNoBits(reader.maxDoc());
     }
@@ -566,13 +564,13 @@ class FieldCacheImpl implements FieldCac
     }
   }
 
-  // inherit javadocs
+  @Override
   public Floats getFloats (AtomicReader reader, String field, boolean setDocsWithField)
     throws IOException {
     return getFloats(reader, field, null, setDocsWithField);
   }
 
-  // inherit javadocs
+  @Override
   public Floats getFloats(AtomicReader reader, String field, FloatParser parser, boolean setDocsWithField)
     throws IOException {
     final NumericDocValues valuesIn = reader.getNumericDocValues(field);
@@ -672,12 +670,12 @@ class FieldCacheImpl implements FieldCac
     }
   }
 
-  // inherit javadocs
+  @Override
   public Longs getLongs(AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
     return getLongs(reader, field, null, setDocsWithField);
   }
   
-  // inherit javadocs
+  @Override
   public Longs getLongs(AtomicReader reader, String field, FieldCache.LongParser parser, boolean setDocsWithField)
       throws IOException {
     final NumericDocValues valuesIn = reader.getNumericDocValues(field);
@@ -791,13 +789,13 @@ class FieldCacheImpl implements FieldCac
     }
   }
 
-  // inherit javadocs
+  @Override
   public Doubles getDoubles(AtomicReader reader, String field, boolean setDocsWithField)
     throws IOException {
     return getDoubles(reader, field, null, setDocsWithField);
   }
 
-  // inherit javadocs
+  @Override
   public Doubles getDoubles(AtomicReader reader, String field, FieldCache.DoubleParser parser, boolean setDocsWithField)
       throws IOException {
     final NumericDocValues valuesIn = reader.getNumericDocValues(field);
@@ -944,13 +942,13 @@ class FieldCacheImpl implements FieldCac
     } else {
       final FieldInfo info = reader.getFieldInfos().fieldInfo(field);
       if (info == null) {
-        return EMPTY_TERMSINDEX;
+        return SortedDocValues.EMPTY;
       } else if (info.hasDocValues()) {
         // we don't try to build a sorted instance from numeric/binary doc
         // values because dedup can be very costly
         throw new IllegalStateException("Type mismatch: " + field + " was indexed as " + info.getDocValuesType());
       } else if (!info.isIndexed()) {
-        return EMPTY_TERMSINDEX;
+        return SortedDocValues.EMPTY;
       }
       return (SortedDocValues) caches.get(SortedDocValues.class).get(reader, new CacheKey(field, acceptableOverheadRatio), false);
     }
@@ -1057,7 +1055,7 @@ class FieldCacheImpl implements FieldCac
     public void get(int docID, BytesRef ret) {
       final int pointer = (int) docToOffset.get(docID);
       if (pointer == 0) {
-        ret.bytes = MISSING;
+        ret.bytes = BytesRef.EMPTY_BYTES;
         ret.offset = 0;
         ret.length = 0;
       } else {
@@ -1068,11 +1066,11 @@ class FieldCacheImpl implements FieldCac
 
   // TODO: this if DocTermsIndex was already created, we
   // should share it...
-  public BinaryDocValues getTerms(AtomicReader reader, String field) throws IOException {
-    return getTerms(reader, field, PackedInts.FAST);
+  public BinaryDocValues getTerms(AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
+    return getTerms(reader, field, setDocsWithField, PackedInts.FAST);
   }
 
-  public BinaryDocValues getTerms(AtomicReader reader, String field, float acceptableOverheadRatio) throws IOException {
+  public BinaryDocValues getTerms(AtomicReader reader, String field, boolean setDocsWithField, float acceptableOverheadRatio) throws IOException {
     BinaryDocValues valuesIn = reader.getBinaryDocValues(field);
     if (valuesIn == null) {
       valuesIn = reader.getSortedDocValues(field);
@@ -1093,7 +1091,7 @@ class FieldCacheImpl implements FieldCac
       return BinaryDocValues.EMPTY;
     }
 
-    return (BinaryDocValues) caches.get(BinaryDocValues.class).get(reader, new CacheKey(field, acceptableOverheadRatio), false);
+    return (BinaryDocValues) caches.get(BinaryDocValues.class).get(reader, new CacheKey(field, acceptableOverheadRatio), setDocsWithField);
   }
 
   static final class BinaryDocValuesCache extends Cache {
@@ -1102,7 +1100,7 @@ class FieldCacheImpl implements FieldCac
     }
 
     @Override
-    protected Object createValue(AtomicReader reader, CacheKey key, boolean setDocsWithField /* ignored */)
+    protected Object createValue(AtomicReader reader, CacheKey key, boolean setDocsWithField)
         throws IOException {
 
       // TODO: would be nice to first check if DocTermsIndex
@@ -1171,8 +1169,22 @@ class FieldCacheImpl implements FieldCac
         }
       }
 
+      final PackedInts.Reader offsetReader = docToOffset.getMutable();
+      if (setDocsWithField) {
+        wrapper.setDocsWithField(reader, key.field, new Bits() {
+          @Override
+          public boolean get(int index) {
+            return offsetReader.get(index) != 0;
+          }
+
+          @Override
+          public int length() {
+            return maxDoc;
+          }
+        });
+      }
       // maybe an int-only impl?
-      return new BinaryDocValuesImpl(bytes.freeze(true), docToOffset.getMutable());
+      return new BinaryDocValuesImpl(bytes.freeze(true), offsetReader);
     }
   }
 

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FieldCacheRewriteMethod.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FieldCacheRewriteMethod.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FieldCacheRewriteMethod.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FieldCacheRewriteMethod.java Mon Oct 21 18:58:24 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.util.Comparator;
 
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
@@ -26,7 +25,6 @@ import org.apache.lucene.index.SortedDoc
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.OpenBitSet;
 
 /**
@@ -91,11 +89,6 @@ public final class FieldCacheRewriteMeth
       TermsEnum termsEnum = query.getTermsEnum(new Terms() {
         
         @Override
-        public Comparator<BytesRef> getComparator() {
-          return BytesRef.getUTF8SortedAsUnicodeComparator();
-        }
-        
-        @Override
         public TermsEnum iterator(TermsEnum reuse) {
           return fcsi.termsEnum();
         }
@@ -121,6 +114,11 @@ public final class FieldCacheRewriteMeth
         }
 
         @Override
+        public boolean hasFreqs() {
+          return false;
+        }
+
+        @Override
         public boolean hasOffsets() {
           return false;
         }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java Mon Oct 21 18:58:24 2013
@@ -893,6 +893,9 @@ public abstract class FieldComparator<T>
       return values[slot];
     }
   }
+  
+  // just used internally in this comparator
+  private static final byte[] MISSING_BYTES = new byte[0];
 
   /** Sorts by field's natural Term sort order.  All
    *  comparisons are done using BytesRef.compareTo, which is
@@ -902,6 +905,7 @@ public abstract class FieldComparator<T>
 
     private BytesRef[] values;
     private BinaryDocValues docTerms;
+    private Bits docsWithField;
     private final String field;
     private BytesRef bottom;
     private final BytesRef tempBR = new BytesRef();
@@ -930,12 +934,15 @@ public abstract class FieldComparator<T>
     @Override
     public int compareBottom(int doc) {
       docTerms.get(doc, tempBR);
-      if (bottom.bytes == BinaryDocValues.MISSING) {
-        if (tempBR.bytes == BinaryDocValues.MISSING) {
+      if (tempBR.length == 0 && docsWithField.get(doc) == false) {
+        tempBR.bytes = MISSING_BYTES;
+      }
+      if (bottom.bytes == MISSING_BYTES) {
+        if (tempBR.bytes == MISSING_BYTES) {
           return 0;
         }
         return -1;
-      } else if (tempBR.bytes == BinaryDocValues.MISSING) {
+      } else if (tempBR.bytes == MISSING_BYTES) {
         return 1;
       }
       return bottom.compareTo(tempBR);
@@ -947,11 +954,15 @@ public abstract class FieldComparator<T>
         values[slot] = new BytesRef();
       }
       docTerms.get(doc, values[slot]);
+      if (values[slot].length == 0 && docsWithField.get(doc) == false) {
+        values[slot].bytes = MISSING_BYTES;
+      }
     }
 
     @Override
     public FieldComparator<BytesRef> setNextReader(AtomicReaderContext context) throws IOException {
-      docTerms = FieldCache.DEFAULT.getTerms(context.reader(), field);
+      docTerms = FieldCache.DEFAULT.getTerms(context.reader(), field, true);
+      docsWithField = FieldCache.DEFAULT.getDocsWithField(context.reader(), field);
       return this;
     }
     
@@ -981,6 +992,9 @@ public abstract class FieldComparator<T>
     @Override
     public int compareDocToValue(int doc, BytesRef value) {
       docTerms.get(doc, tempBR);
+      if (tempBR.length == 0 && docsWithField.get(doc) == false) {
+        tempBR.bytes = MISSING_BYTES;
+      }
       return tempBR.compareTo(value);
     }
   }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FuzzyQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FuzzyQuery.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FuzzyQuery.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FuzzyQuery.java Mon Oct 21 18:58:24 2013
@@ -43,6 +43,13 @@ import org.apache.lucene.util.automaton.
  * will match a significant amount of the term dictionary. If you really want this, consider
  * using an n-gram indexing technique (such as the SpellChecker in the 
  * <a href="{@docRoot}/../suggest/overview-summary.html">suggest module</a>) instead.
+ *
+ * <p>NOTE: terms of length 1 or 2 will sometimes not match because of how the scaled
+ * distance between two terms is computed.  For a term to match, the edit distance between
+ * the terms must be less than the minimum length term (either the input term, or
+ * the candidate term).  For example, FuzzyQuery on term "abcd" with maxEdits=2 will
+ * not match an indexed term "ab", and FuzzyQuery on term "a" with maxEdits=2 will not
+ * match an indexed term "abc".
  */
 public class FuzzyQuery extends MultiTermQuery {
   

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Mon Oct 21 18:58:24 2013
@@ -46,7 +46,7 @@ import org.apache.lucene.util.automaton.
  * to the specified filter term.
  *
  * <p>Term enumerations are always ordered by
- * {@link #getComparator}.  Each term in the enumeration is
+ * {@link BytesRef#compareTo}.  Each term in the enumeration is
  * greater than all that precede it.</p>
  */
 public class FuzzyTermsEnum extends TermsEnum {
@@ -293,11 +293,6 @@ public class FuzzyTermsEnum extends Term
   }
   
   @Override
-  public Comparator<BytesRef> getComparator() {
-    return actualEnum.getComparator();
-  }
-  
-  @Override
   public long ord() throws IOException {
     return actualEnum.ord();
   }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java Mon Oct 21 18:58:24 2013
@@ -506,12 +506,6 @@ public class IndexSearcher {
    * Just like {@link #search(Weight, int, Sort, boolean, boolean)}, but you choose
    * whether or not the fields in the returned {@link FieldDoc} instances should
    * be set by specifying fillFields.
-   *
-   * <p>NOTE: this does not compute scores by default.  If you
-   * need scores, create a {@link TopFieldCollector}
-   * instance by calling {@link TopFieldCollector#create} and
-   * then pass that to {@link #search(List, Weight,
-   * Collector)}.</p>
    */
   protected TopFieldDocs search(Weight weight, FieldDoc after, int nDocs,
                                 Sort sort, boolean fillFields,

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java Mon Oct 21 18:58:24 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.util.Comparator;
 import java.util.LinkedList;
 
 import org.apache.lucene.analysis.NumericTokenStream; // for javadocs
@@ -392,7 +391,6 @@ public final class NumericRangeQuery<T e
     private BytesRef currentLowerBound, currentUpperBound;
 
     private final LinkedList<BytesRef> rangeBounds = new LinkedList<BytesRef>();
-    private final Comparator<BytesRef> termComp;
 
     NumericRangeTermsEnum(final TermsEnum tenum) {
       super(tenum);
@@ -481,15 +479,13 @@ public final class NumericRangeQuery<T e
           // should never happen
           throw new IllegalArgumentException("Invalid NumericType");
       }
-
-      termComp = getComparator();
     }
     
     private void nextRange() {
       assert rangeBounds.size() % 2 == 0;
 
       currentLowerBound = rangeBounds.removeFirst();
-      assert currentUpperBound == null || termComp.compare(currentUpperBound, currentLowerBound) <= 0 :
+      assert currentUpperBound == null || currentUpperBound.compareTo(currentLowerBound) <= 0 :
         "The current upper bound must be <= the new lower bound";
       
       currentUpperBound = rangeBounds.removeFirst();
@@ -501,10 +497,10 @@ public final class NumericRangeQuery<T e
         nextRange();
         
         // if the new upper bound is before the term parameter, the sub-range is never a hit
-        if (term != null && termComp.compare(term, currentUpperBound) > 0)
+        if (term != null && term.compareTo(currentUpperBound) > 0)
           continue;
         // never seek backwards, so use current term if lower bound is smaller
-        return (term != null && termComp.compare(term, currentLowerBound) > 0) ?
+        return (term != null && term.compareTo(currentLowerBound) > 0) ?
           term : currentLowerBound;
       }
       
@@ -516,11 +512,11 @@ public final class NumericRangeQuery<T e
     
     @Override
     protected final AcceptStatus accept(BytesRef term) {
-      while (currentUpperBound == null || termComp.compare(term, currentUpperBound) > 0) {
+      while (currentUpperBound == null || term.compareTo(currentUpperBound) > 0) {
         if (rangeBounds.isEmpty())
           return AcceptStatus.END;
         // peek next sub-range, only seek if the current term is smaller than next lower bound
-        if (termComp.compare(term, rangeBounds.getFirst()) < 0)
+        if (term.compareTo(rangeBounds.getFirst()) < 0)
           return AcceptStatus.NO_AND_SEEK;
         // step forward to next range without seeking, as next lower range bound is less or equal current term
         nextRange();

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/PrefixTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/PrefixTermsEnum.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/PrefixTermsEnum.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/PrefixTermsEnum.java Mon Oct 21 18:58:24 2013
@@ -26,7 +26,7 @@ import org.apache.lucene.util.StringHelp
  * Subclass of FilteredTermEnum for enumerating all terms that match the
  * specified prefix filter term.
  * <p>Term enumerations are always ordered by
- * {@link #getComparator}.  Each term in the enumeration is
+ * {@link BytesRef#compareTo}.  Each term in the enumeration is
  * greater than all that precede it.</p>
  */
 public class PrefixTermsEnum extends FilteredTermsEnum {

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java Mon Oct 21 18:58:24 2013
@@ -87,9 +87,6 @@ public abstract class ScoringRewrite<Q e
     @Override
     public Query rewrite(IndexReader reader, MultiTermQuery query) throws IOException {
       final BooleanQuery bq = SCORING_BOOLEAN_QUERY_REWRITE.rewrite(reader, query);
-      // TODO: if empty boolean query return NullQuery?
-      if (bq.clauses().isEmpty())
-        return bq;
       // strip the scores off
       final Query result = new ConstantScoreQuery(bq);
       result.setBoost(query.getBoost());
@@ -109,7 +106,7 @@ public abstract class ScoringRewrite<Q e
     
     final int size = col.terms.size();
     if (size > 0) {
-      final int sort[] = col.terms.sort(col.termsEnum.getComparator());
+      final int sort[] = col.terms.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
       final float[] boost = col.array.boost;
       final TermContext[] termStates = col.array.termState;
       for (int i = 0; i < size; i++) {

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/Sort.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/Sort.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/Sort.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/Sort.java Mon Oct 21 18:58:24 2013
@@ -205,7 +205,7 @@ public class Sort {
   /** Whether the relevance score is needed to sort documents. */
   boolean needsScores() {
     for (SortField sortField : fields) {
-      if (sortField.getType() == SortField.Type.SCORE) {
+      if (sortField.needsScores()) {
         return true;
       }
     }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/SortField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/SortField.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/SortField.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/SortField.java Mon Oct 21 18:58:24 2013
@@ -402,4 +402,9 @@ public class SortField {
   public SortField rewrite(IndexSearcher searcher) throws IOException {
     return this;
   }
+  
+  /** Whether the relevance score is needed to sort documents. */
+  public boolean needsScores() {
+    return type == Type.SCORE;
+  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/TermCollectingRewrite.java Mon Oct 21 18:58:24 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.util.Comparator;
 
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.Fields;
@@ -47,7 +46,6 @@ abstract class TermCollectingRewrite<Q e
   
   final void collectTerms(IndexReader reader, MultiTermQuery query, TermCollector collector) throws IOException {
     IndexReaderContext topReaderContext = reader.getContext();
-    Comparator<BytesRef> lastTermComp = null;
     for (AtomicReaderContext context : topReaderContext.leaves()) {
       final Fields fields = context.reader().fields();
       if (fields == null) {
@@ -67,11 +65,6 @@ abstract class TermCollectingRewrite<Q e
       if (termsEnum == TermsEnum.EMPTY)
         continue;
       
-      // Check comparator compatibility:
-      final Comparator<BytesRef> newTermComp = termsEnum.getComparator();
-      if (lastTermComp != null && newTermComp != null && newTermComp != lastTermComp)
-        throw new RuntimeException("term comparator should not change between segments: "+lastTermComp+" != "+newTermComp);
-      lastTermComp = newTermComp;
       collector.setReaderContext(topReaderContext, context);
       collector.setNextEnum(termsEnum);
       BytesRef bytes;

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/TermRangeTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/TermRangeTermsEnum.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/TermRangeTermsEnum.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/TermRangeTermsEnum.java Mon Oct 21 18:58:24 2013
@@ -17,18 +17,13 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.Comparator;
-
 import org.apache.lucene.index.FilteredTermsEnum;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
 
 /**
  * Subclass of FilteredTermEnum for enumerating all terms that match the
- * specified range parameters.
- * <p>Term enumerations are always ordered by
- * {@link #getComparator}.  Each term in the enumeration is
+ * specified range parameters.  Each term in the enumeration is
  * greater than all that precede it.</p>
  */
 public class TermRangeTermsEnum extends FilteredTermsEnum {
@@ -37,7 +32,6 @@ public class TermRangeTermsEnum extends 
   final private boolean includeUpper;
   final private BytesRef lowerBytesRef;
   final private BytesRef upperBytesRef;
-  private final Comparator<BytesRef> termComp;
 
   /**
    * Enumerates all terms greater/equal than <code>lowerTerm</code>
@@ -82,7 +76,6 @@ public class TermRangeTermsEnum extends 
     }
 
     setInitialSeekTerm(lowerBytesRef);
-    termComp = getComparator();
   }
 
   @Override
@@ -92,7 +85,7 @@ public class TermRangeTermsEnum extends 
     
     // Use this field's default sort ordering
     if (upperBytesRef != null) {
-      final int cmp = termComp.compare(upperBytesRef, term);
+      final int cmp = upperBytesRef.compareTo(term);
       /*
        * if beyond the upper term, or is exclusive and this is equal to
        * the upper term, break out

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/TopTermsRewrite.java Mon Oct 21 18:58:24 2013
@@ -70,20 +70,18 @@ public abstract class TopTermsRewrite<Q 
       private final Map<BytesRef,ScoreTerm> visitedTerms = new HashMap<BytesRef,ScoreTerm>();
       
       private TermsEnum termsEnum;
-      private Comparator<BytesRef> termComp;
       private BoostAttribute boostAtt;        
       private ScoreTerm st;
       
       @Override
       public void setNextEnum(TermsEnum termsEnum) {
         this.termsEnum = termsEnum;
-        this.termComp = termsEnum.getComparator();
         
         assert compareToLastTerm(null);
 
         // lazy init the initial ScoreTerm because comparator is not known on ctor:
         if (st == null)
-          st = new ScoreTerm(this.termComp, new TermContext(topReaderContext));
+          st = new ScoreTerm(new TermContext(topReaderContext));
         boostAtt = termsEnum.attributes().addAttribute(BoostAttribute.class);
       }
     
@@ -95,7 +93,7 @@ public abstract class TopTermsRewrite<Q 
         } else if (t == null) {
           lastTerm = null;
         } else {
-          assert termsEnum.getComparator().compare(lastTerm, t) < 0: "lastTerm=" + lastTerm + " t=" + t;
+          assert lastTerm.compareTo(t) < 0: "lastTerm=" + lastTerm + " t=" + t;
           lastTerm.copyBytes(t);
         }
         return true;
@@ -115,7 +113,7 @@ public abstract class TopTermsRewrite<Q 
           final ScoreTerm t = stQueue.peek();
           if (boost < t.boost)
             return true;
-          if (boost == t.boost && termComp.compare(bytes, t.bytes) > 0)
+          if (boost == t.boost && bytes.compareTo(t.bytes) > 0)
             return true;
         }
         ScoreTerm t = visitedTerms.get(bytes);
@@ -139,7 +137,7 @@ public abstract class TopTermsRewrite<Q 
             visitedTerms.remove(st.bytes);
             st.termState.clear(); // reset the termstate! 
           } else {
-            st = new ScoreTerm(termComp, new TermContext(topReaderContext));
+            st = new ScoreTerm(new TermContext(topReaderContext));
           }
           assert stQueue.size() <= maxSize : "the PQ size must be limited to maxSize";
           // set maxBoostAtt with values to help FuzzyTermsEnum to optimize
@@ -185,26 +183,22 @@ public abstract class TopTermsRewrite<Q 
     new Comparator<ScoreTerm>() {
       @Override
       public int compare(ScoreTerm st1, ScoreTerm st2) {
-        assert st1.termComp == st2.termComp :
-          "term comparator should not change between segments";
-        return st1.termComp.compare(st1.bytes, st2.bytes);
+        return st1.bytes.compareTo(st2.bytes);
       }
     };
 
   static final class ScoreTerm implements Comparable<ScoreTerm> {
-    public final Comparator<BytesRef> termComp;
     public final BytesRef bytes = new BytesRef();
     public float boost;
     public final TermContext termState;
-    public ScoreTerm(Comparator<BytesRef> termComp, TermContext termState) {
-      this.termComp = termComp;
+    public ScoreTerm(TermContext termState) {
       this.termState = termState;
     }
     
     @Override
     public int compareTo(ScoreTerm other) {
       if (this.boost == other.boost)
-        return termComp.compare(other.bytes, this.bytes);
+        return other.bytes.compareTo(this.bytes);
       else
         return Float.compare(this.boost, other.boost);
     }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/package.html?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/package.html (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/package.html Mon Oct 21 18:58:24 2013
@@ -173,7 +173,7 @@ section for more notes on the process.
     {@link org.apache.lucene.index.Term Term}
     and an upper
     {@link org.apache.lucene.index.Term Term}
-    according to {@link org.apache.lucene.index.TermsEnum#getComparator TermsEnum.getComparator()}. It is not intended
+    according to {@link org.apache.lucene.util.BytesRef#compareTo BytesRef.compareTo()}. It is not intended
     for numerical ranges; use {@link org.apache.lucene.search.NumericRangeQuery NumericRangeQuery} instead.
 
     For example, one could find all documents

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java Mon Oct 21 18:58:24 2013
@@ -233,7 +233,7 @@ public abstract class SimilarityBase ext
     if (discountOverlaps)
       numTerms = state.getLength() - state.getNumOverlap();
     else
-      numTerms = state.getLength() / state.getBoost();
+      numTerms = state.getLength();
     return encodeNormValue(state.getBoost(), numTerms);
   }
   

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/store/CompoundFileDirectory.java Mon Oct 21 18:58:24 2013
@@ -71,7 +71,7 @@ import java.io.IOException;
  * 
  * @lucene.experimental
  */
-public final class CompoundFileDirectory extends Directory {
+public final class CompoundFileDirectory extends BaseDirectory {
   
   /** Offset/Length for a slice inside of a compound file */
   public static final class FileEntry {