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

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

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java Tue Feb 22 01:00:39 2011
@@ -27,6 +27,7 @@ import org.apache.lucene.util.automaton.
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.SpecialOperations;
 import org.apache.lucene.util.automaton.Transition;
+import org.apache.lucene.util.automaton.UTF32ToUTF8;
 
 /**
  * A FilteredTermsEnum that enumerates terms based upon what is accepted by a
@@ -46,8 +47,6 @@ import org.apache.lucene.util.automaton.
  * @lucene.experimental
  */
 public class AutomatonTermsEnum extends FilteredTermsEnum {
-  // the object-oriented form of the DFA
-  private final Automaton automaton;
   // a tableized array-based form of the DFA
   private final ByteRunAutomaton runAutomaton;
   // common suffix of the automaton
@@ -71,37 +70,20 @@ public class AutomatonTermsEnum extends 
   private final Comparator<BytesRef> termComp;
 
   /**
-   * Expert ctor:
    * Construct an enumerator based upon an automaton, enumerating the specified
    * field, working on a supplied TermsEnum
    * <p>
    * @lucene.experimental 
    * <p>
-   * @param runAutomaton pre-compiled ByteRunAutomaton
-   * @param finite true if the automaton accepts a finite language
+   * @param compiled CompiledAutomaton
    */
-  public AutomatonTermsEnum(ByteRunAutomaton runAutomaton,
-                     TermsEnum tenum,
-                     boolean finite, BytesRef commonSuffixRef)
-      throws IOException {
+  public AutomatonTermsEnum(TermsEnum tenum, CompiledAutomaton compiled) throws IOException {
     super(tenum);
-    this.automaton = runAutomaton.getAutomaton();
-    this.finite = finite;
+    this.finite = compiled.finite;
+    this.runAutomaton = compiled.runAutomaton;
+    this.commonSuffixRef = compiled.commonSuffixRef;
+    this.allTransitions = compiled.sortedTransitions;
 
-    this.runAutomaton = runAutomaton;
-    if (finite) {
-      // don't use suffix w/ finite DFAs
-      this.commonSuffixRef = null;
-    } else if (commonSuffixRef == null) {
-      // compute now
-      this.commonSuffixRef = SpecialOperations.getCommonSuffixBytesRef(automaton);
-    } else {
-      // precomputed
-      this.commonSuffixRef = commonSuffixRef;
-    }
-
-    // build a cache of sorted transitions for every state
-    allTransitions = this.automaton.getSortedTransitions();
     // used for path tracking, where each bit is a numbered state.
     visited = new long[runAutomaton.getSize()];
 
@@ -109,17 +91,6 @@ public class AutomatonTermsEnum extends 
   }
   
   /**
-   * Construct an enumerator based upon an automaton, enumerating the specified
-   * field, working on a supplied TermsEnum
-   * <p>
-   * It will automatically calculate whether or not the automaton is finite
-   */
-  public AutomatonTermsEnum(Automaton automaton, TermsEnum tenum)
-    throws IOException {
-    this(new ByteRunAutomaton(automaton), tenum, SpecialOperations.isFinite(automaton), null);
-  }
- 
-  /**
    * Returns true if the term matches the automaton. Also stashes away the term
    * to assist with smart enumeration.
    */
@@ -140,9 +111,9 @@ public class AutomatonTermsEnum extends 
   @Override
   protected BytesRef nextSeekTerm(final BytesRef term) throws IOException {
     if (term == null) {
-      seekBytesRef.copy("");
+      assert seekBytesRef.length == 0;
       // return the empty term, as its valid
-      if (runAutomaton.run(seekBytesRef.bytes, seekBytesRef.offset, seekBytesRef.length)) {   
+      if (runAutomaton.isAccept(runAutomaton.getInitialState())) {   
         return seekBytesRef;
       }
     } else {
@@ -151,25 +122,20 @@ public class AutomatonTermsEnum extends 
 
     // seek to the next possible string;
     if (nextString()) {
-      // reposition
-           
-      if (linear)
-        setLinear(infinitePosition);
-      return seekBytesRef;
+      return seekBytesRef;  // reposition
+    } else {
+      return null;          // no more possible strings can match
     }
-    // no more possible strings can match
-    return null;
   }
 
-  // this instance prevents unicode conversion during backtracking,
-  // we can just call setLinear once at the end.
-  int infinitePosition;
-
   /**
    * Sets the enum to operate in linear fashion, as we have found
-   * a looping transition at position
+   * a looping transition at position: we set an upper bound and 
+   * act like a TermRangeQuery for this portion of the term space.
    */
   private void setLinear(int position) {
+    assert linear == false;
+    
     int state = runAutomaton.getInitialState();
     int maxInterval = 0xff;
     for (int i = 0; i < position; i++) {
@@ -193,6 +159,8 @@ public class AutomatonTermsEnum extends 
     System.arraycopy(seekBytesRef.bytes, 0, linearUpperBound.bytes, 0, position);
     linearUpperBound.bytes[position] = (byte) maxInterval;
     linearUpperBound.length = length;
+    
+    linear = true;
   }
 
   private final IntsRef savedStates = new IntsRef(10);
@@ -226,8 +194,7 @@ public class AutomatonTermsEnum extends 
         states[pos+1] = nextState;
         // we found a loop, record it for faster enumeration
         if (!finite && !linear && visited[nextState] == curGen) {
-          linear = true;
-          infinitePosition = pos;
+          setLinear(pos);
         }
         state = nextState;
       }
@@ -313,15 +280,16 @@ public class AutomatonTermsEnum extends 
            */
           transition = allTransitions[state][0];
           state = transition.getDest().getNumber();
-          // we found a loop, record it for faster enumeration
-          if (!finite && !linear && visited[state] == curGen) {
-            linear = true;
-            infinitePosition = seekBytesRef.length;
-          }
+          
           // append the minimum transition
           seekBytesRef.grow(seekBytesRef.length + 1);
           seekBytesRef.length++;
           seekBytesRef.bytes[seekBytesRef.length - 1] = (byte) transition.getMin();
+          
+          // we found a loop, record it for faster enumeration
+          if (!finite && !linear && visited[state] == curGen) {
+            setLinear(seekBytesRef.length-1);
+          }
         }
         return true;
       }
@@ -350,4 +318,26 @@ public class AutomatonTermsEnum extends 
     }
     return -1; /* all solutions exhausted */
   }
+  
+  /**
+   * immutable class with everything this enum needs.
+   */
+  public static class CompiledAutomaton {
+    public final ByteRunAutomaton runAutomaton;
+    public final Transition[][] sortedTransitions;
+    public final BytesRef commonSuffixRef;
+    public final boolean finite;
+    
+    public CompiledAutomaton(Automaton automaton, boolean finite) {
+      Automaton utf8 = new UTF32ToUTF8().convert(automaton);
+      runAutomaton = new ByteRunAutomaton(utf8, true);
+      sortedTransitions = utf8.getSortedTransitions();
+      this.finite = finite;
+      if (finite) {
+        commonSuffixRef = null;
+      } else {
+        commonSuffixRef = SpecialOperations.getCommonSuffixBytesRef(utf8);
+      }
+    }
+  }
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BooleanClause.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BooleanClause.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BooleanClause.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BooleanClause.java Tue Feb 22 01:00:39 2011
@@ -18,7 +18,7 @@ package org.apache.lucene.search;
  */
 
 /** A clause in a BooleanQuery. */
-public class BooleanClause implements java.io.Serializable {
+public class BooleanClause {
   
   /** Specifies how clauses are to occur in matching documents. */
   public static enum Occur {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BoostAttribute.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BoostAttribute.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BoostAttribute.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/BoostAttribute.java Tue Feb 22 01:00:39 2011
@@ -27,7 +27,7 @@ import org.apache.lucene.index.TermsEnum
  * {@link TopTermsRewrite} mode.
  * {@link FuzzyQuery} is using this to take the edit distance into account.
  * <p><b>Please note:</b> This attribute is intended to be added only by the TermsEnum
- * to itsself in its constructor and consumed by the {@link MultiTermQuery.RewriteMethod}.
+ * to itself in its constructor and consumed by the {@link MultiTermQuery.RewriteMethod}.
  * @lucene.internal
  */
 public interface BoostAttribute extends Attribute {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/CachingWrapperFilter.java Tue Feb 22 01:00:39 2011
@@ -17,7 +17,6 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import java.io.Serializable;
 import java.io.IOException;
 import java.util.Map;
 import java.util.WeakHashMap;
@@ -68,7 +67,7 @@ public class CachingWrapperFilter extend
 
   protected final FilterCache<DocIdSet> cache;
 
-  static abstract class FilterCache<T> implements Serializable {
+  static abstract class FilterCache<T> {
 
     /**
      * A transient Filter cache (package private because of test)

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java Tue Feb 22 01:00:39 2011
@@ -31,7 +31,7 @@ public class DefaultSimilarity extends S
    *
    *  @lucene.experimental */
   @Override
-  public float computeNorm(String field, FieldInvertState state) {
+  public float computeNorm(FieldInvertState state) {
     final int numTerms;
     if (discountOverlaps)
       numTerms = state.getLength() - state.getNumOverlap();

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DocIdSetIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DocIdSetIterator.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DocIdSetIterator.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/DocIdSetIterator.java Tue Feb 22 01:00:39 2011
@@ -78,10 +78,10 @@ public abstract class DocIdSetIterator {
    * 
    * Some implementations are considerably more efficient than that.
    * <p>
-   * <b>NOTE:</b> certain implementations may return a different value (each
-   * time) if called several times in a row with the same target.
+   * <b>NOTE:</b> when <code> target &le; current</code> implementations may opt 
+   * not to advance beyond their current {@link #docID()}.
    * <p>
-   * <b>NOTE:</b> this method may be called with {@value #NO_MORE_DOCS} for
+   * <b>NOTE:</b> this method may be called with {@link #NO_MORE_DOCS} for
    * efficiency by some Scorers. If your implementation cannot efficiently
    * determine that it should exhaust, it is recommended that you check for that
    * value in each call to this method.

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Explanation.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Explanation.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Explanation.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Explanation.java Tue Feb 22 01:00:39 2011
@@ -17,11 +17,10 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import java.io.Serializable;
 import java.util.ArrayList;
 
 /** Expert: Describes the score computation for document and query. */
-public class Explanation implements java.io.Serializable {
+public class Explanation {
   private float value;                            // the value of this node
   private String description;                     // what it represents
   private ArrayList<Explanation> details;                      // sub-explanations
@@ -135,7 +134,7 @@ public class Explanation implements java
    * before storing any large or un-serializable fields.
    *
    */
-  public static abstract class IDFExplanation implements Serializable {
+  public static abstract class IDFExplanation {
     /**
      * @return the idf factor
      */

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldCache.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldCache.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldCache.java Tue Feb 22 01:00:39 2011
@@ -29,7 +29,6 @@ import org.apache.lucene.analysis.Numeri
 import org.apache.lucene.util.packed.PackedInts;
 
 import java.io.IOException;
-import java.io.Serializable;
 import java.io.PrintStream;
 
 import java.text.DecimalFormat;
@@ -61,7 +60,7 @@ public interface FieldCache {
    * is used to specify a custom parser to {@link
    * SortField#SortField(String, FieldCache.Parser)}.
    */
-  public interface Parser extends Serializable {
+  public interface Parser {
   }
 
   /** Interface to parse bytes from document fields.

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldCacheImpl.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldCacheImpl.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldCacheImpl.java Tue Feb 22 01:00:39 2011
@@ -137,6 +137,13 @@ public class FieldCacheImpl implements F
     public Object getValue() { return value; }
   }
 
+  final static IndexReader.ReaderFinishedListener purgeReader = new IndexReader.ReaderFinishedListener() {
+    // @Override -- not until Java 1.6
+    public void finished(IndexReader reader) {
+      FieldCache.DEFAULT.purge(reader);
+    }
+  };
+
   /** Expert: Internal cache. */
   final static class Cache<T> {
     Cache() {
@@ -171,8 +178,10 @@ public class FieldCacheImpl implements F
       synchronized (readerCache) {
         innerCache = readerCache.get(readerKey);
         if (innerCache == null) {
+          // First time this reader is using FieldCache
           innerCache = new HashMap<Entry<T>,Object>();
           readerCache.put(readerKey, innerCache);
+          reader.addReaderFinishedListener(purgeReader);
           value = null;
         } else {
           value = innerCache.get(key);

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldComparatorSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldComparatorSource.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldComparatorSource.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldComparatorSource.java Tue Feb 22 01:00:39 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.io.Serializable;
 
 /**
  * Provides a {@link FieldComparator} for custom field sorting.
@@ -26,7 +25,7 @@ import java.io.Serializable;
  * @lucene.experimental
  *
  */
-public abstract class FieldComparatorSource implements Serializable {
+public abstract class FieldComparatorSource {
 
   /**
    * Creates a comparator for the field in the given index.

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Filter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Filter.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Filter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Filter.java Tue Feb 22 01:00:39 2011
@@ -27,7 +27,7 @@ import org.apache.lucene.util.DocIdBitSe
  *  Abstract base class for restricting which documents may
  *  be returned during searching.
  */
-public abstract class Filter implements java.io.Serializable {
+public abstract class Filter {
   
   /**
    * Creates a {@link DocIdSet} enumerating the documents that should be

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Tue Feb 22 01:00:39 2011
@@ -22,6 +22,7 @@ import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.AutomatonTermsEnum.CompiledAutomaton;
 import org.apache.lucene.util.Attribute;
 import org.apache.lucene.util.AttributeImpl;
 import org.apache.lucene.util.AttributeSource;
@@ -140,18 +141,18 @@ public final class FuzzyTermsEnum extend
    */
   private TermsEnum getAutomatonEnum(int editDistance, BytesRef lastTerm)
       throws IOException {
-    final List<ByteRunAutomaton> runAutomata = initAutomata(editDistance);
+    final List<CompiledAutomaton> runAutomata = initAutomata(editDistance);
     if (editDistance < runAutomata.size()) {
       return new AutomatonFuzzyTermsEnum(runAutomata.subList(0, editDistance + 1)
-          .toArray(new ByteRunAutomaton[editDistance + 1]), lastTerm);
+          .toArray(new CompiledAutomaton[editDistance + 1]), lastTerm);
     } else {
       return null;
     }
   }
 
   /** initialize levenshtein DFAs up to maxDistance, if possible */
-  private List<ByteRunAutomaton> initAutomata(int maxDistance) {
-    final List<ByteRunAutomaton> runAutomata = dfaAtt.automata();
+  private List<CompiledAutomaton> initAutomata(int maxDistance) {
+    final List<CompiledAutomaton> runAutomata = dfaAtt.automata();
     if (runAutomata.size() <= maxDistance && 
         maxDistance <= LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE) {
       LevenshteinAutomata builder = 
@@ -165,7 +166,7 @@ public final class FuzzyTermsEnum extend
             UnicodeUtil.newString(termText, 0, realPrefixLength));
           a = BasicOperations.concatenate(prefix, a);
         }
-        runAutomata.add(new ByteRunAutomaton(a));
+        runAutomata.add(new CompiledAutomaton(a, true));
       }
     }
     return runAutomata;
@@ -261,6 +262,7 @@ public final class FuzzyTermsEnum extend
     return actualEnum.docsAndPositions(skipDocs, reuse);
   }
   
+  @Override
   public void seek(BytesRef term, TermState state) throws IOException {
     actualEnum.seek(term, state);
   }
@@ -311,32 +313,38 @@ public final class FuzzyTermsEnum extend
     private final BoostAttribute boostAtt =
       attributes().addAttribute(BoostAttribute.class);
     
-    public AutomatonFuzzyTermsEnum(ByteRunAutomaton matchers[], 
+    public AutomatonFuzzyTermsEnum(CompiledAutomaton compiled[], 
         BytesRef lastTerm) throws IOException {
-      super(matchers[matchers.length - 1], tenum, true, null);
-      this.matchers = matchers;
+      super(tenum, compiled[compiled.length - 1]);
+      this.matchers = new ByteRunAutomaton[compiled.length];
+      for (int i = 0; i < compiled.length; i++)
+        this.matchers[i] = compiled[i].runAutomaton;
       this.lastTerm = lastTerm;
       termRef = new BytesRef(term.text());
     }
     
     /** finds the smallest Lev(n) DFA that accepts the term. */
     @Override
-    protected AcceptStatus accept(BytesRef term) {
-      if (term.equals(termRef)) { // ed = 0
-        boostAtt.setBoost(1.0F);
-        return AcceptStatus.YES_AND_SEEK;
-      }
-      
-      int codePointCount = -1;
+    protected AcceptStatus accept(BytesRef term) {    
+      int ed = matchers.length - 1;
       
-      // TODO: benchmark doing this backwards
-      for (int i = 1; i < matchers.length; i++)
-        if (matchers[i].run(term.bytes, term.offset, term.length)) {
-          // this sucks, we convert just to score based on length.
-          if (codePointCount == -1) {
-            codePointCount = UnicodeUtil.codePointCount(term);
+      if (matches(term, ed)) { // we match the outer dfa
+        // now compute exact edit distance
+        while (ed > 0) {
+          if (matches(term, ed - 1)) {
+            ed--;
+          } else {
+            break;
           }
-          final float similarity = 1.0f - ((float) i / (float) 
+        }
+        
+        // scale to a boost and return (if similarity > minSimilarity)
+        if (ed == 0) { // exact match
+          boostAtt.setBoost(1.0F);
+          return AcceptStatus.YES_AND_SEEK;
+        } else {
+          final int codePointCount = UnicodeUtil.codePointCount(term);
+          final float similarity = 1.0f - ((float) ed / (float) 
               (Math.min(codePointCount, termLength)));
           if (similarity > minSimilarity) {
             boostAtt.setBoost((similarity - minSimilarity) * scale_factor);
@@ -345,8 +353,14 @@ public final class FuzzyTermsEnum extend
             return AcceptStatus.NO_AND_SEEK;
           }
         }
-      
-      return AcceptStatus.NO_AND_SEEK;
+      } else {
+        return AcceptStatus.NO_AND_SEEK;
+      }
+    }
+    
+    /** returns true if term is within k edits of the query term */
+    final boolean matches(BytesRef term, int k) {
+      return k == 0 ? term.equals(termRef) : matchers[k].run(term.bytes, term.offset, term.length);
     }
     
     /** defers to superclass, except can start at an arbitrary location */
@@ -562,14 +576,14 @@ public final class FuzzyTermsEnum extend
   
   /** @lucene.internal */
   public static interface LevenshteinAutomataAttribute extends Attribute {
-    public List<ByteRunAutomaton> automata();
+    public List<CompiledAutomaton> automata();
   }
     
   /** @lucene.internal */
   public static final class LevenshteinAutomataAttributeImpl extends AttributeImpl implements LevenshteinAutomataAttribute {
-    private final List<ByteRunAutomaton> automata = new ArrayList<ByteRunAutomaton>();
+    private final List<CompiledAutomaton> automata = new ArrayList<CompiledAutomaton>();
       
-    public List<ByteRunAutomaton> automata() {
+    public List<CompiledAutomaton> automata() {
       return automata;
     }
 
@@ -594,7 +608,7 @@ public final class FuzzyTermsEnum extend
 
     @Override
     public void copyTo(AttributeImpl target) {
-      final List<ByteRunAutomaton> targetAutomata =
+      final List<CompiledAutomaton> targetAutomata =
         ((LevenshteinAutomataAttribute) target).automata();
       targetAutomata.clear();
       targetAutomata.addAll(automata);

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Tue Feb 22 01:00:39 2011
@@ -65,10 +65,11 @@ public class IndexSearcher {
   // in the next release
   protected final ReaderContext readerContext;
   protected final AtomicReaderContext[] leafContexts;
+  // used with executor - each slice holds a set of leafs executed within one thread
+  protected final LeafSlice[] leafSlices;
 
   // These are only used for multi-threaded search
   private final ExecutorService executor;
-  protected final IndexSearcher[] subSearchers;
 
   // the default SimilarityProvider
   private static final SimilarityProvider defaultProvider = new DefaultSimilarity();
@@ -175,47 +176,22 @@ public class IndexSearcher {
     this.closeReader = closeReader;
     this.readerContext = context;
     leafContexts = ReaderUtil.leaves(context);
-    
-    if (executor == null) {
-      subSearchers = null;
-    } else {
-      subSearchers = new IndexSearcher[this.leafContexts.length];
-      for (int i = 0; i < subSearchers.length; i++) {
-        if (leafContexts[i].reader == context.reader) {
-          subSearchers[i] = this;
-        } else {
-          subSearchers[i] = new IndexSearcher(context, leafContexts[i]);
-        }
-      }
-    }
-  }
-
-  /**
-   * Expert: Creates a searcher from a top-level {@link ReaderContext} with and
-   * executes searches on the given leave slice exclusively instead of searching
-   * over all leaves. This constructor should be used to run one or more leaves
-   * within a single thread. Hence, for scorer and filter this looks like an
-   * ordinary search in the hierarchy such that there is no difference between
-   * single and multi-threaded.
-   * 
-   * @lucene.experimental
-   * */
-  public IndexSearcher(ReaderContext topLevel, AtomicReaderContext... leaves) {
-    assert assertLeaves(topLevel, leaves);
-    readerContext = topLevel;
-    reader = topLevel.reader;
-    leafContexts = leaves;
-    executor = null;
-    subSearchers = null;
-    closeReader = false;
+    this.leafSlices = executor == null ? null : slices(leafContexts);
   }
   
-  private boolean assertLeaves(ReaderContext topLevel, AtomicReaderContext... leaves) {
-    for (AtomicReaderContext leaf : leaves) {
-      assert ReaderUtil.getTopLevelContext(leaf) == topLevel : "leaf context is not a leaf of the given top-level context";
+  /**
+   * Expert: Creates an array of leaf slices each holding a subset of the given leaves.
+   * Each {@link LeafSlice} is executed in a single thread. By default there
+   * will be one {@link LeafSlice} per leaf ({@link AtomicReaderContext}).
+   */
+  protected LeafSlice[] slices(AtomicReaderContext...leaves) {
+    LeafSlice[] slices = new LeafSlice[leaves.length];
+    for (int i = 0; i < slices.length; i++) {
+      slices[i] = new LeafSlice(leaves[i]);
     }
-    return true;
+    return slices;
   }
+
   
   /** Return the {@link IndexReader} this searches. */
   public IndexReader getIndexReader() {
@@ -236,11 +212,11 @@ public class IndexSearcher {
       return reader.docFreq(term);
     } else {
       final ExecutionHelper<Integer> runner = new ExecutionHelper<Integer>(executor);
-      for(int i = 0; i < subSearchers.length; i++) {
-        final IndexSearcher searchable = subSearchers[i];
+      for(int i = 0; i < leafContexts.length; i++) {
+        final IndexReader leaf = leafContexts[i].reader;
         runner.submit(new Callable<Integer>() {
             public Integer call() throws IOException {
-              return Integer.valueOf(searchable.docFreq(term));
+              return Integer.valueOf(leaf.docFreq(term));
             }
           });
       }
@@ -324,7 +300,7 @@ public class IndexSearcher {
    */
   public void search(Query query, Filter filter, Collector results)
     throws IOException {
-    search(createWeight(query), filter, results);
+    search(leafContexts, createWeight(query), filter, results);
   }
 
   /** Lower-level search API.
@@ -342,7 +318,7 @@ public class IndexSearcher {
   */
   public void search(Query query, Collector results)
     throws IOException {
-    search(createWeight(query), null, results);
+    search(leafContexts, createWeight(query), null, results);
   }
   
   /** Search implementation with arbitrary sorting.  Finds
@@ -382,32 +358,25 @@ public class IndexSearcher {
    * @throws BooleanQuery.TooManyClauses
    */
   protected TopDocs search(Weight weight, Filter filter, int nDocs) throws IOException {
-
     if (executor == null) {
-      // single thread
-      int limit = reader.maxDoc();
-      if (limit == 0) {
-        limit = 1;
-      }
-      nDocs = Math.min(nDocs, limit);
-      TopScoreDocCollector collector = TopScoreDocCollector.create(nDocs, !weight.scoresDocsOutOfOrder());
-      search(weight, filter, collector);
-      return collector.topDocs();
+      return search(leafContexts, weight, filter, nDocs);
     } else {
       final HitQueue hq = new HitQueue(nDocs, false);
       final Lock lock = new ReentrantLock();
       final ExecutionHelper<TopDocs> runner = new ExecutionHelper<TopDocs>(executor);
     
-      for (int i = 0; i < subSearchers.length; i++) { // search each sub
+      for (int i = 0; i < leafSlices.length; i++) { // search each sub
         runner.submit(
-                      new SearcherCallableNoSort(lock, subSearchers[i], weight, filter, nDocs, hq));
+                      new SearcherCallableNoSort(lock, this, leafSlices[i], weight, filter, nDocs, hq));
       }
 
       int totalHits = 0;
       float maxScore = Float.NEGATIVE_INFINITY;
       for (final TopDocs topDocs : runner) {
-        totalHits += topDocs.totalHits;
-        maxScore = Math.max(maxScore, topDocs.getMaxScore());
+        if(topDocs.totalHits != 0) {
+          totalHits += topDocs.totalHits;
+          maxScore = Math.max(maxScore, topDocs.getMaxScore());
+        }
       }
 
       final ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
@@ -418,6 +387,25 @@ public class IndexSearcher {
     }
   }
 
+  /** Expert: Low-level search implementation.  Finds the top <code>n</code>
+   * hits for <code>query</code>, using the given leaf readers applying <code>filter</code> if non-null.
+   *
+   * <p>Applications should usually call {@link IndexSearcher#search(Query,int)} or
+   * {@link IndexSearcher#search(Query,Filter,int)} instead.
+   * @throws BooleanQuery.TooManyClauses
+   */
+  protected TopDocs search(AtomicReaderContext[] leaves, Weight weight, Filter filter, int nDocs) throws IOException {
+    // single thread
+    int limit = reader.maxDoc();
+    if (limit == 0) {
+      limit = 1;
+    }
+    nDocs = Math.min(nDocs, limit);
+    TopScoreDocCollector collector = TopScoreDocCollector.create(nDocs, !weight.scoresDocsOutOfOrder());
+    search(leaves, weight, filter, collector);
+    return collector.topDocs();
+  }
+
   /** Expert: Low-level search implementation with arbitrary sorting.  Finds
    * the top <code>n</code> hits for <code>query</code>, applying
    * <code>filter</code> if non-null, and sorting the hits by the criteria in
@@ -449,33 +437,26 @@ public class IndexSearcher {
       throws IOException {
 
     if (sort == null) throw new NullPointerException();
-
+    
     if (executor == null) {
-      // single thread
-      int limit = reader.maxDoc();
-      if (limit == 0) {
-        limit = 1;
-      }
-      nDocs = Math.min(nDocs, limit);
-
-      TopFieldCollector collector = TopFieldCollector.create(sort, nDocs,
-                                                             fillFields, fieldSortDoTrackScores, fieldSortDoMaxScore, !weight.scoresDocsOutOfOrder());
-      search(weight, filter, collector);
-      return (TopFieldDocs) collector.topDocs();
+      // use all leaves here!
+      return search (leafContexts, weight, filter, nDocs, sort, fillFields);
     } else {
       // TODO: make this respect fillFields
       final FieldDocSortedHitQueue hq = new FieldDocSortedHitQueue(nDocs);
       final Lock lock = new ReentrantLock();
       final ExecutionHelper<TopFieldDocs> runner = new ExecutionHelper<TopFieldDocs>(executor);
-      for (int i = 0; i < subSearchers.length; i++) { // search each sub
+      for (int i = 0; i < leafSlices.length; i++) { // search each leaf slice
         runner.submit(
-                      new SearcherCallableWithSort(lock, subSearchers[i], weight, filter, nDocs, hq, sort));
+                      new SearcherCallableWithSort(lock, this, leafSlices[i], weight, filter, nDocs, hq, sort));
       }
       int totalHits = 0;
       float maxScore = Float.NEGATIVE_INFINITY;
       for (final TopFieldDocs topFieldDocs : runner) {
-        totalHits += topFieldDocs.totalHits;
-        maxScore = Math.max(maxScore, topFieldDocs.getMaxScore());
+        if (topFieldDocs.totalHits != 0) {
+          totalHits += topFieldDocs.totalHits;
+          maxScore = Math.max(maxScore, topFieldDocs.getMaxScore());
+        }
       }
       final ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
       for (int i = hq.size() - 1; i >= 0; i--) // put docs in array
@@ -484,6 +465,33 @@ public class IndexSearcher {
       return new TopFieldDocs(totalHits, scoreDocs, hq.getFields(), maxScore);
     }
   }
+  
+  
+  /**
+   * Just like {@link #search(Weight, Filter, int, Sort)}, 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(Weight, Filter,
+   * Collector)}.</p>
+   */
+  protected TopFieldDocs search(AtomicReaderContext[] leaves, Weight weight, Filter filter, int nDocs,
+      Sort sort, boolean fillFields) throws IOException {
+    // single thread
+    int limit = reader.maxDoc();
+    if (limit == 0) {
+      limit = 1;
+    }
+    nDocs = Math.min(nDocs, limit);
+
+    TopFieldCollector collector = TopFieldCollector.create(sort, nDocs,
+                                                           fillFields, fieldSortDoTrackScores, fieldSortDoMaxScore, !weight.scoresDocsOutOfOrder());
+    search(leaves, weight, filter, collector);
+    return (TopFieldDocs) collector.topDocs();
+  }
 
   /**
    * Lower-level search API.
@@ -497,6 +505,12 @@ public class IndexSearcher {
    * documents. The high-level search API ({@link IndexSearcher#search(Query,int)}) is
    * usually more efficient, as it skips non-high-scoring hits.
    * 
+   * <p>
+   * NOTE: this method executes the searches on all given leaves exclusively.
+   * To search across all the searchers leaves use {@link #leafContexts}.
+   * 
+   * @param leaves 
+   *          the searchers leaves to execute the searches on
    * @param weight
    *          to match documents
    * @param filter
@@ -505,7 +519,7 @@ public class IndexSearcher {
    *          to receive hits
    * @throws BooleanQuery.TooManyClauses
    */
-  protected void search(Weight weight, Filter filter, Collector collector)
+  protected void search(AtomicReaderContext[] leaves, Weight weight, Filter filter, Collector collector)
       throws IOException {
 
     // TODO: should we make this
@@ -513,18 +527,18 @@ public class IndexSearcher {
     ScorerContext scorerContext =  ScorerContext.def().scoreDocsInOrder(true).topScorer(true);
     // always use single thread:
     if (filter == null) {
-      for (int i = 0; i < leafContexts.length; i++) { // search each subreader
-        collector.setNextReader(leafContexts[i]);
+      for (int i = 0; i < leaves.length; i++) { // search each subreader
+        collector.setNextReader(leaves[i]);
         scorerContext = scorerContext.scoreDocsInOrder(!collector.acceptsDocsOutOfOrder());
-        Scorer scorer = weight.scorer(leafContexts[i], scorerContext);
+        Scorer scorer = weight.scorer(leaves[i], scorerContext);
         if (scorer != null) {
           scorer.score(collector);
         }
       }
     } else {
-      for (int i = 0; i < leafContexts.length; i++) { // search each subreader
-        collector.setNextReader(leafContexts[i]);
-        searchWithFilter(leafContexts[i], weight, filter, collector);
+      for (int i = 0; i < leaves.length; i++) { // search each subreader
+        collector.setNextReader(leaves[i]);
+        searchWithFilter(leaves[i], weight, filter, collector);
       }
     }
   }
@@ -649,7 +663,7 @@ public class IndexSearcher {
    * Returns this searchers the top-level {@link ReaderContext}.
    * @see IndexReader#getTopReaderContext()
    */
-  /* Sugar for .getIndexReader().getTopReaderContext() */
+  /* sugar for #getReader().getTopReaderContext() */
   public ReaderContext getTopReaderContext() {
     return readerContext;
   }
@@ -660,24 +674,26 @@ public class IndexSearcher {
   private static final class SearcherCallableNoSort implements Callable<TopDocs> {
 
     private final Lock lock;
-    private final IndexSearcher searchable;
+    private final IndexSearcher searcher;
     private final Weight weight;
     private final Filter filter;
     private final int nDocs;
     private final HitQueue hq;
+    private final LeafSlice slice;
 
-    public SearcherCallableNoSort(Lock lock, IndexSearcher searchable, Weight weight,
+    public SearcherCallableNoSort(Lock lock, IndexSearcher searcher, LeafSlice slice,  Weight weight,
         Filter filter, int nDocs, HitQueue hq) {
       this.lock = lock;
-      this.searchable = searchable;
+      this.searcher = searcher;
       this.weight = weight;
       this.filter = filter;
       this.nDocs = nDocs;
       this.hq = hq;
+      this.slice = slice;
     }
 
     public TopDocs call() throws IOException {
-      final TopDocs docs = searchable.search (weight, filter, nDocs);
+      final TopDocs docs = searcher.search (slice.leaves, weight, filter, nDocs);
       final ScoreDoc[] scoreDocs = docs.scoreDocs;
       for (int j = 0; j < scoreDocs.length; j++) { // merge scoreDocs into hq
         final ScoreDoc scoreDoc = scoreDocs[j];
@@ -701,26 +717,28 @@ public class IndexSearcher {
   private static final class SearcherCallableWithSort implements Callable<TopFieldDocs> {
 
     private final Lock lock;
-    private final IndexSearcher searchable;
+    private final IndexSearcher searcher;
     private final Weight weight;
     private final Filter filter;
     private final int nDocs;
     private final FieldDocSortedHitQueue hq;
     private final Sort sort;
+    private final LeafSlice slice;
 
-    public SearcherCallableWithSort(Lock lock, IndexSearcher searchable, Weight weight,
+    public SearcherCallableWithSort(Lock lock, IndexSearcher searcher, LeafSlice slice, Weight weight,
         Filter filter, int nDocs, FieldDocSortedHitQueue hq, Sort sort) {
       this.lock = lock;
-      this.searchable = searchable;
+      this.searcher = searcher;
       this.weight = weight;
       this.filter = filter;
       this.nDocs = nDocs;
       this.hq = hq;
       this.sort = sort;
+      this.slice = slice;
     }
 
     public TopFieldDocs call() throws IOException {
-      final TopFieldDocs docs = searchable.search (weight, filter, nDocs, sort);
+      final TopFieldDocs docs = searcher.search (slice.leaves, weight, filter, nDocs, sort, true);
       lock.lock();
       try {
         hq.setFields(docs.fields);
@@ -787,8 +805,22 @@ public class IndexSearcher {
     }
 
     public Iterator<T> iterator() {
-      // use the shortcut here - this is only used in a privat context
+      // use the shortcut here - this is only used in a private context
       return this;
     }
   }
+
+  /**
+   * A class holding a subset of the {@link IndexSearcher}s leaf contexts to be
+   * executed within a single thread.
+   * 
+   * @lucene.experimental
+   */
+  public static class LeafSlice {
+    final AtomicReaderContext[] leaves;
+    
+    public LeafSlice(AtomicReaderContext...leaves) {
+      this.leaves = leaves;
+    }
+  }
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MultiCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MultiCollector.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MultiCollector.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MultiCollector.java Tue Feb 22 01:00:39 2011
@@ -26,7 +26,7 @@ import org.apache.lucene.search.Scorer;
 /**
  * A {@link Collector} which allows running a search with several
  * {@link Collector}s. It offers a static {@link #wrap} method which accepts a
- * list of collectots and wraps them with {@link MultiCollector}, while
+ * list of collectors and wraps them with {@link MultiCollector}, while
  * filtering out the <code>null</code> null ones.
  */
 public class MultiCollector extends Collector {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java Tue Feb 22 01:00:39 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.io.Serializable;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
@@ -67,7 +66,7 @@ public abstract class MultiTermQuery ext
   transient int numberOfTerms = 0;
 
   /** Abstract class that defines how the query is rewritten. */
-  public static abstract class RewriteMethod implements Serializable {
+  public static abstract class RewriteMethod {
     public abstract Query rewrite(IndexReader reader, MultiTermQuery query) throws IOException;
   }
 
@@ -90,11 +89,6 @@ public abstract class MultiTermQuery ext
       result.setBoost(query.getBoost());
       return result;
     }
-
-    // Make sure we are still a singleton even after deserializing
-    protected Object readResolve() {
-      return CONSTANT_SCORE_FILTER_REWRITE;
-    }
   };
 
   /** A rewrite method that first translates each term into
@@ -240,11 +234,6 @@ public abstract class MultiTermQuery ext
     public void setDocCountPercent(double percent) {
       throw new UnsupportedOperationException("Please create a private instance");
     }
-
-    // Make sure we are still a singleton even after deserializing
-    protected Object readResolve() {
-      return CONSTANT_SCORE_AUTO_REWRITE_DEFAULT;
-    }
   };
 
   /**

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/NumericRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/NumericRangeFilter.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/NumericRangeFilter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/NumericRangeFilter.java Tue Feb 22 01:00:39 2011
@@ -39,8 +39,6 @@ import org.apache.lucene.util.NumericUti
  * See {@link NumericRangeQuery} for details on how Lucene
  * indexes and searches numeric valued fields.
  *
- * @lucene.experimental
- *
  * @since 2.9
  **/
 public final class NumericRangeFilter<T extends Number> extends MultiTermQueryWrapperFilter<NumericRangeQuery<T>> {
@@ -181,4 +179,7 @@ public final class NumericRangeFilter<T 
   /** Returns the upper value of this range filter */
   public T getMax() { return query.getMax(); }
   
+  /** Returns the precision step. */
+  public int getPrecisionStep() { return query.getPrecisionStep(); }
+  
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java Tue Feb 22 01:00:39 2011
@@ -76,8 +76,6 @@ import org.apache.lucene.index.TermsEnum
  * BooleanQuery rewrite methods without changing
  * BooleanQuery's default max clause count.
  *
- * @lucene.experimental
- *
  * <br><h3>How it works</h3>
  *
  * <p>See the publication about <a target="_blank" href="http://www.panfmp.org">panFMP</a>,
@@ -321,6 +319,9 @@ public final class NumericRangeQuery<T e
   /** Returns the upper value of this range query */
   public T getMax() { return max; }
   
+  /** Returns the precision step. */
+  public int getPrecisionStep() { return precisionStep; }
+  
   @Override
   public String toString(final String field) {
     final StringBuilder sb = new StringBuilder();

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Query.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Query.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Query.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Query.java Tue Feb 22 01:00:39 2011
@@ -44,7 +44,7 @@ import org.apache.lucene.index.Term;
     <li>{@link org.apache.lucene.queryParser.QueryParser QueryParser}
     </ul>
 */
-public abstract class Query implements java.io.Serializable, Cloneable {
+public abstract class Query implements Cloneable {
   private float boost = 1.0f;                     // query boost factor
 
   /** Sets the boost for this query clause to <code>b</code>.  Documents

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ScoreDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ScoreDoc.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ScoreDoc.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/ScoreDoc.java Tue Feb 22 01:00:39 2011
@@ -19,7 +19,7 @@ package org.apache.lucene.search;
 
 /** Expert: Returned by low-level search implementations.
  * @see TopDocs */
-public class ScoreDoc implements java.io.Serializable {
+public class ScoreDoc {
   /** Expert: The score of this document for the query. */
   public float score;
 

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Similarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Similarity.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Similarity.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Similarity.java Tue Feb 22 01:00:39 2011
@@ -19,7 +19,6 @@ package org.apache.lucene.search;
 
 
 import java.io.IOException;
-import java.io.Serializable;
 import java.util.Collection;
 
 import org.apache.lucene.index.FieldInvertState;
@@ -525,7 +524,7 @@ import org.apache.lucene.util.SmallFloat
  * @see org.apache.lucene.index.IndexWriterConfig#setSimilarityProvider(SimilarityProvider)
  * @see IndexSearcher#setSimilarityProvider(SimilarityProvider)
  */
-public abstract class Similarity implements Serializable {
+public abstract class Similarity {
   
   public static final int NO_DOC_ID_PROVIDED = -1;
 
@@ -562,49 +561,13 @@ public abstract class Similarity impleme
    * Thus they have limited precision, and documents
    * must be re-indexed if this method is altered.
    *
-   * <p>For backward compatibility this method by default calls
-   * {@link #lengthNorm(String, int)} passing
-   * {@link FieldInvertState#getLength()} as the second argument, and
-   * then multiplies this value by {@link FieldInvertState#getBoost()}.</p>
-   * 
    * @lucene.experimental
    * 
-   * @param field field name
    * @param state current processing state for this field
    * @return the calculated float norm
    */
-  public abstract float computeNorm(String field, FieldInvertState state);
+  public abstract float computeNorm(FieldInvertState state);
   
-  /** Computes the normalization value for a field given the total number of
-   * terms contained in a field.  These values, together with field boosts, are
-   * stored in an index and multipled into scores for hits on each field by the
-   * search code.
-   *
-   * <p>Matches in longer fields are less precise, so implementations of this
-   * method usually return smaller values when <code>numTokens</code> is large,
-   * and larger values when <code>numTokens</code> is small.
-   * 
-   * <p>Note that the return values are computed under 
-   * {@link org.apache.lucene.index.IndexWriter#addDocument(org.apache.lucene.document.Document)} 
-   * and then stored using
-   * {@link #encodeNormValue(float)}.  
-   * Thus they have limited precision, and documents
-   * must be re-indexed if this method is altered.
-   *
-   * @param fieldName the name of the field
-   * @param numTokens the total number of tokens contained in fields named
-   * <i>fieldName</i> of <i>doc</i>.
-   * @return a normalization factor for hits on this field of this document
-   *
-   * @see org.apache.lucene.document.Field#setBoost(float)
-   *
-   * @deprecated Please override computeNorm instead
-   */
-  @Deprecated
-  public final float lengthNorm(String fieldName, int numTokens) {
-    throw new UnsupportedOperationException("please use computeNorm instead");
-  }
-
   /** Encodes a normalization factor for storage in an index.
    *
    * <p>The encoding uses a three-bit mantissa, a five-bit exponent, and
@@ -782,7 +745,6 @@ public abstract class Similarity impleme
    * The default implementation returns 1.
    *
    * @param docId The docId currently being scored.  If this value is {@link #NO_DOC_ID_PROVIDED}, then it should be assumed that the PayloadQuery implementation does not provide document information
-   * @param fieldName The fieldName of the term this payload belongs to
    * @param start The start position of the payload
    * @param end The end position of the payload
    * @param payload The payload byte array to be scored
@@ -792,7 +754,7 @@ public abstract class Similarity impleme
    *
    */
   // TODO: maybe switch this API to BytesRef?
-  public float scorePayload(int docId, String fieldName, int start, int end, byte [] payload, int offset, int length)
+  public float scorePayload(int docId, int start, int end, byte [] payload, int offset, int length)
   {
     return 1;
   }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Sort.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Sort.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Sort.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Sort.java Tue Feb 22 01:00:39 2011
@@ -17,7 +17,6 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import java.io.Serializable;
 import java.util.Arrays;
 
 
@@ -97,8 +96,7 @@ import java.util.Arrays;
  *
  * @since   lucene 1.4
  */
-public class Sort
-implements Serializable {
+public class Sort {
 
   /**
    * Represents sorting by computed relevance. Using this sort criteria returns

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SortField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SortField.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SortField.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SortField.java Tue Feb 22 01:00:39 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.io.Serializable;
 import java.util.Locale;
 
 import org.apache.lucene.search.cache.*;
@@ -33,10 +32,9 @@ import org.apache.lucene.util.StringHelp
  * @since   lucene 1.4
  * @see Sort
  */
-public class SortField
-implements Serializable {
+public class SortField {
 
-  /** Sort by document score (relevancy).  Sort values are Float and higher
+  /** Sort by document score (relevance).  Sort values are Float and higher
    * values are at the front. */
   public static final int SCORE = 0;
 
@@ -84,7 +82,7 @@ implements Serializable {
    * uses ordinals to do the sorting. */
   public static final int STRING_VAL = 11;
   
-  /** Represents sorting by document score (relevancy). */
+  /** Represents sorting by document score (relevance). */
   public static final SortField FIELD_SCORE = new SortField (null, SCORE);
 
   /** Represents sorting by document number (index order). */
@@ -427,13 +425,6 @@ implements Serializable {
     return hash;
   }
 
-  // field must be interned after reading from stream
-  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
-    in.defaultReadObject();
-    if (field != null)
-      field = StringHelper.intern(field);
-  }
-
   /** Returns the {@link FieldComparator} to use for
    * sorting.
    *

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TopDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TopDocs.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TopDocs.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TopDocs.java Tue Feb 22 01:00:39 2011
@@ -20,7 +20,7 @@ package org.apache.lucene.search;
 /** Represents hits returned by {@link
  * IndexSearcher#search(Query,Filter,int)} and {@link
  * IndexSearcher#search(Query,int)}. */
-public class TopDocs implements java.io.Serializable {
+public class TopDocs {
   /** The total number of hits for the query.
   */
   public int totalHits;

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java Tue Feb 22 01:00:39 2011
@@ -87,7 +87,7 @@ public abstract class TopTermsRewrite<Q 
       @Override
       public boolean collect(BytesRef bytes) throws IOException {
         final float boost = boostAtt.getBoost();
-        // ignore uncompetetive hits
+        // ignore uncompetitive hits
         if (stQueue.size() == maxSize) {
           final ScoreTerm t = stQueue.peek();
           if (boost < t.boost)

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TotalHitCountCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TotalHitCountCollector.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TotalHitCountCollector.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TotalHitCountCollector.java Tue Feb 22 01:00:39 2011
@@ -31,16 +31,20 @@ public class TotalHitCountCollector exte
     return totalHits;
   }
 
+  @Override
   public void setScorer(Scorer scorer) {
   }
 
+  @Override
   public void collect(int doc) {
     totalHits++;
   }
 
+  @Override
   public void setNextReader(AtomicReaderContext context) {
   }
 
+  @Override
   public boolean acceptsDocsOutOfOrder() {
     return true;
   }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Weight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Weight.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Weight.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/Weight.java Tue Feb 22 01:00:39 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.io.Serializable;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
@@ -54,7 +53,7 @@ import org.apache.lucene.index.IndexRead
  * 
  * @since 2.9
  */
-public abstract class Weight implements Serializable {
+public abstract class Weight {
 
   /**
    * An explanation of the score computation for the named document.

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/cache/EntryCreator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/cache/EntryCreator.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/cache/EntryCreator.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/cache/EntryCreator.java Tue Feb 22 01:00:39 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.search.cache;
  */
 
 import java.io.IOException;
-import java.io.Serializable;
 
 import org.apache.lucene.index.IndexReader;
 
@@ -27,7 +26,7 @@ import org.apache.lucene.index.IndexRead
  * 
  * @lucene.experimental
  */
-public abstract class EntryCreator<T> implements Serializable
+public abstract class EntryCreator<T>
 {
   public abstract T create( IndexReader reader ) throws IOException;
   public abstract T validate( T entry, IndexReader reader ) throws IOException;
@@ -58,6 +57,7 @@ public abstract class EntryCreator<T> im
   // This can be removed
   //------------------------------------------------------------------------
 
+  @Override
   public boolean equals(Object obj) {
     if( obj instanceof EntryCreator ) {
       return getCacheKey().equals( ((EntryCreator)obj).getCacheKey() );

Added: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/cache/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/cache/package.html?rev=1073192&view=auto
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/cache/package.html (added)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/cache/package.html Tue Feb 22 01:00:39 2011
@@ -0,0 +1,25 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<html>
+<head>
+   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+</head>
+<body>
+Fieldcache
+</body>
+</html>

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/CustomScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/CustomScoreQuery.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/CustomScoreQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/CustomScoreQuery.java Tue Feb 22 01:00:39 2011
@@ -52,7 +52,7 @@ public class CustomScoreQuery extends Qu
   
   /**
    * Create a CustomScoreQuery over input subQuery.
-   * @param subQuery the sub query whose scored is being customed. Must not be null. 
+   * @param subQuery the sub query whose scored is being customized. Must not be null. 
    */
   public CustomScoreQuery(Query subQuery) {
     this(subQuery, new ValueSourceQuery[0]);

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/MultiValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/MultiValueSource.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/MultiValueSource.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/MultiValueSource.java Tue Feb 22 01:00:39 2011
@@ -52,6 +52,7 @@ public final class MultiValueSource exte
       return other.getValues(context);
   }
   
+  @Override
   public DocValues getValues(ReaderContext context) throws IOException {
     if (context.isAtomic) {
       return getValues((AtomicReaderContext) context);

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/OrdFieldSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/OrdFieldSource.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/OrdFieldSource.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/OrdFieldSource.java Tue Feb 22 01:00:39 2011
@@ -99,7 +99,9 @@ public class OrdFieldSource extends Valu
   /*(non-Javadoc) @see java.lang.Object#equals(java.lang.Object) */
   @Override
   public boolean equals(Object o) {
-    if (o.getClass() !=  OrdFieldSource.class) return false;
+    if (o == this) return true;
+    if (o == null) return false;
+    if (o.getClass() != OrdFieldSource.class) return false;
     OrdFieldSource other = (OrdFieldSource)o;
     return this.field.equals(other.field);
   }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ReverseOrdFieldSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ReverseOrdFieldSource.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ReverseOrdFieldSource.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ReverseOrdFieldSource.java Tue Feb 22 01:00:39 2011
@@ -107,7 +107,9 @@ public class ReverseOrdFieldSource exten
   /*(non-Javadoc) @see java.lang.Object#equals(java.lang.Object) */
   @Override
   public boolean equals(Object o) {
-    if (o.getClass() !=  ReverseOrdFieldSource.class) return false;
+    if (o == this) return true;
+    if (o == null) return false;
+    if (o.getClass() != ReverseOrdFieldSource.class) return false;
     ReverseOrdFieldSource other = (ReverseOrdFieldSource)o;
     return this.field.equals(other.field); 
   }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSource.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSource.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSource.java Tue Feb 22 01:00:39 2011
@@ -23,7 +23,6 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.search.function.DocValues;
 
 import java.io.IOException;
-import java.io.Serializable;
 
 /**
  * Expert: source of values for basic function queries.
@@ -37,7 +36,7 @@ import java.io.Serializable;
  *
  *
  */
-public abstract class ValueSource implements Serializable {
+public abstract class ValueSource {
 
   /**
    * Return the DocValues used by the function query.

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadFunction.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadFunction.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadFunction.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadFunction.java Tue Feb 22 01:00:39 2011
@@ -16,7 +16,6 @@ package org.apache.lucene.search.payload
  * limitations under the License.
  */
 
-import java.io.Serializable;
 import org.apache.lucene.search.Explanation;
 
 /**
@@ -29,7 +28,7 @@ import org.apache.lucene.search.Explanat
  *               change
  * 
  **/
-public abstract class PayloadFunction implements Serializable {
+public abstract class PayloadFunction {
 
   /**
    * Calculate the score up to this point for this doc and field

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java Tue Feb 22 01:00:39 2011
@@ -192,7 +192,7 @@ public class PayloadNearQuery extends Sp
     protected void processPayloads(Collection<byte[]> payLoads, int start, int end) {
       for (final byte[] thePayload : payLoads) {
         payloadScore = function.currentScore(doc, fieldName, start, end,
-            payloadsSeen, payloadScore, similarity.scorePayload(doc, fieldName,
+            payloadsSeen, payloadScore, similarity.scorePayload(doc,
                 spans.start(), spans.end(), thePayload, 0, thePayload.length));
         ++payloadsSeen;
       }
@@ -219,6 +219,7 @@ public class PayloadNearQuery extends Sp
           return true;    	
     }
 
+    @Override
     public float score() throws IOException {
 
       return super.score()

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java Tue Feb 22 01:00:39 2011
@@ -41,7 +41,7 @@ import java.io.IOException;
  * {@link org.apache.lucene.index.Term} occurs.
  * <p>
  * In order to take advantage of this, you must override
- * {@link org.apache.lucene.search.Similarity#scorePayload(int, String, int, int, byte[],int,int)}
+ * {@link org.apache.lucene.search.Similarity#scorePayload(int, int, int, byte[],int,int)}
  * which returns 1 by default.
  * <p>
  * Payload scores are aggregated using a pluggable {@link PayloadFunction}.
@@ -119,14 +119,14 @@ public class PayloadTermQuery extends Sp
           if (payload != null) {
             payloadScore = function.currentScore(doc, term.field(),
                                                  spans.start(), spans.end(), payloadsSeen, payloadScore,
-                                                 similarity.scorePayload(doc, term.field(), spans.start(),
+                                                 similarity.scorePayload(doc, spans.start(),
                                                                          spans.end(), payload.bytes,
                                                                          payload.offset,
                                                                          payload.length));
           } else {
             payloadScore = function.currentScore(doc, term.field(),
                                                  spans.start(), spans.end(), payloadsSeen, payloadScore,
-                                                 similarity.scorePayload(doc, term.field(), spans.start(),
+                                                 similarity.scorePayload(doc, spans.start(),
                                                                          spans.end(), null,
                                                                          0,
                                                                          0));

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanNearPayloadCheckQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanNearPayloadCheckQuery.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanNearPayloadCheckQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanNearPayloadCheckQuery.java Tue Feb 22 01:00:39 2011
@@ -72,6 +72,7 @@ public class SpanNearPayloadCheckQuery e
     return AcceptStatus.NO;
   }
 
+  @Override
   public String toString(String field) {
     StringBuilder buffer = new StringBuilder();
     buffer.append("spanPayCheck(");

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanPayloadCheckQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanPayloadCheckQuery.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanPayloadCheckQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanPayloadCheckQuery.java Tue Feb 22 01:00:39 2011
@@ -74,6 +74,7 @@ public class SpanPayloadCheckQuery exten
     return AcceptStatus.YES;
   } 
 
+  @Override
   public String toString(String field) {
     StringBuilder buffer = new StringBuilder();
     buffer.append("spanPayCheck(");

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/FSLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/FSLockFactory.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/FSLockFactory.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/FSLockFactory.java Tue Feb 22 01:00:39 2011
@@ -33,7 +33,7 @@ public abstract class FSLockFactory exte
   /**
    * Set the lock directory. This method can be only called
    * once to initialize the lock directory. It is used by {@link FSDirectory}
-   * to set the lock directory to itsself.
+   * to set the lock directory to itself.
    * Subclasses can also use this method to set the directory
    * in the constructor.
    */

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/NativeFSLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/NativeFSLockFactory.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/NativeFSLockFactory.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/NativeFSLockFactory.java Tue Feb 22 01:00:39 2011
@@ -60,7 +60,7 @@ public class NativeFSLockFactory extends
    * Create a NativeFSLockFactory instance, with null (unset)
    * lock directory. When you pass this factory to a {@link FSDirectory}
    * subclass, the lock directory is automatically set to the
-   * directory itsself. Be sure to create one instance for each directory
+   * directory itself. Be sure to create one instance for each directory
    * your create!
    */
   public NativeFSLockFactory() throws IOException {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/RAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/RAMDirectory.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/RAMDirectory.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/RAMDirectory.java Tue Feb 22 01:00:39 2011
@@ -19,7 +19,6 @@ package org.apache.lucene.store;
 
 import java.io.IOException;
 import java.io.FileNotFoundException;
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -35,10 +34,7 @@ import org.apache.lucene.util.ThreadInte
  * implementation is by default the {@link SingleInstanceLockFactory}
  * but can be changed with {@link #setLockFactory}.
  */
-public class RAMDirectory extends Directory implements Serializable {
-
-  private static final long serialVersionUID = 1l;
-
+public class RAMDirectory extends Directory {
   protected final Map<String,RAMFile> fileMap = new ConcurrentHashMap<String,RAMFile>();
   protected final AtomicLong sizeInBytes = new AtomicLong();
   

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/RAMFile.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/RAMFile.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/RAMFile.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/RAMFile.java Tue Feb 22 01:00:39 2011
@@ -18,13 +18,9 @@ package org.apache.lucene.store;
  */
 
 import java.util.ArrayList;
-import java.io.Serializable;
 
 /** @lucene.internal */
-public class RAMFile implements Serializable {
-
-  private static final long serialVersionUID = 1l;
-
+public class RAMFile {
   protected ArrayList<byte[]> buffers = new ArrayList<byte[]>();
   long length;
   RAMDirectory directory;

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/SimpleFSLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/SimpleFSLockFactory.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/SimpleFSLockFactory.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/SimpleFSLockFactory.java Tue Feb 22 01:00:39 2011
@@ -57,7 +57,7 @@ public class SimpleFSLockFactory extends
    * Create a SimpleFSLockFactory instance, with null (unset)
    * lock directory. When you pass this factory to a {@link FSDirectory}
    * subclass, the lock directory is automatically set to the
-   * directory itsself. Be sure to create one instance for each directory
+   * directory itself. Be sure to create one instance for each directory
    * your create!
    */
   public SimpleFSLockFactory() throws IOException {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/ArrayUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/ArrayUtil.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/ArrayUtil.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/ArrayUtil.java Tue Feb 22 01:00:39 2011
@@ -634,7 +634,7 @@ public final class ArrayUtil {
   
   /**
    * Sorts the given array slice using the {@link Comparator}. This method uses the insertion sort
-   * algorithm. It is only recommened to use this algorithm for partially sorted small arrays!
+   * algorithm. It is only recommended to use this algorithm for partially sorted small arrays!
    * @param fromIndex start index (inclusive)
    * @param toIndex end index (exclusive)
    */
@@ -644,7 +644,7 @@ public final class ArrayUtil {
   
   /**
    * Sorts the given array using the {@link Comparator}. This method uses the insertion sort
-   * algorithm. It is only recommened to use this algorithm for partially sorted small arrays!
+   * algorithm. It is only recommended to use this algorithm for partially sorted small arrays!
    */
   public static <T> void insertionSort(T[] a, Comparator<? super T> comp) {
     insertionSort(a, 0, a.length, comp);
@@ -652,7 +652,7 @@ public final class ArrayUtil {
   
   /**
    * Sorts the given array slice in natural order. This method uses the insertion sort
-   * algorithm. It is only recommened to use this algorithm for partially sorted small arrays!
+   * algorithm. It is only recommended to use this algorithm for partially sorted small arrays!
    * @param fromIndex start index (inclusive)
    * @param toIndex end index (exclusive)
    */
@@ -662,7 +662,7 @@ public final class ArrayUtil {
   
   /**
    * Sorts the given array in natural order. This method uses the insertion sort
-   * algorithm. It is only recommened to use this algorithm for partially sorted small arrays!
+   * algorithm. It is only recommended to use this algorithm for partially sorted small arrays!
    */
   public static <T extends Comparable<? super T>> void insertionSort(T[] a) {
     insertionSort(a, 0, a.length);

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/AttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/AttributeImpl.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/AttributeImpl.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/AttributeImpl.java Tue Feb 22 01:00:39 2011
@@ -17,7 +17,6 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
-import java.io.Serializable;
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
 import java.lang.ref.WeakReference;
@@ -30,7 +29,7 @@ import java.util.LinkedList;
  * Attributes are used to add data in a dynamic, yet type-safe way to a source
  * of usually streamed objects, e. g. a {@link org.apache.lucene.analysis.TokenStream}.
  */
-public abstract class AttributeImpl implements Cloneable, Serializable, Attribute {  
+public abstract class AttributeImpl implements Cloneable, Attribute {  
   /**
    * Clears the values in this AttributeImpl and resets it to its 
    * default value. If this implementation implements more than one Attribute interface

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRef.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRef.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRef.java Tue Feb 22 01:00:39 2011
@@ -21,14 +21,13 @@ import java.util.Comparator;
 import java.io.UnsupportedEncodingException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
-import java.io.Externalizable;
 import java.io.IOException;
 
 /** Represents byte[], as a slice (offset + length) into an
  *  existing byte[].
  *
  *  @lucene.experimental */
-public final class BytesRef implements Comparable<BytesRef>, Externalizable {
+public final class BytesRef implements Comparable<BytesRef> {
 
   static final int HASH_PRIME = 31;
   public static final byte[] EMPTY_BYTES = new byte[0]; 
@@ -209,6 +208,7 @@ public final class BytesRef implements C
   }
 
   /** Returns hex encoded bytes, eg [0x6c 0x75 0x63 0x65 0x6e 0x65] */
+  @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();
     sb.append('[');
@@ -364,25 +364,4 @@ public final class BytesRef implements C
       return a.length - b.length;
     }
   }
-
-  public void writeExternal(ObjectOutput out)
-    throws IOException
-  {
-    out.writeInt(length);
-    if (length > 0) {
-      out.write(bytes, offset, length);
-    }
-  }
-
-  public void readExternal( ObjectInput in ) throws
-      IOException, ClassNotFoundException {
-    length = in.readInt();
-    offset = 0;
-    if (length > 0) {
-      bytes = new byte[length];
-      in.read(bytes, 0, length);
-    } else {
-      bytes = EMPTY_BYTES;
-    }
-  }
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRefHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRefHash.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRefHash.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRefHash.java Tue Feb 22 01:00:39 2011
@@ -38,7 +38,7 @@ import org.apache.lucene.util.ByteBlockP
  * <p>
  * Note: The maximum capacity {@link BytesRef} instance passed to
  * {@link #add(BytesRef)} must not be longer than {@link ByteBlockPool#BYTE_BLOCK_SIZE}-2. 
- * The internal storage is limited to 2GB totalbyte storage.
+ * The internal storage is limited to 2GB total byte storage.
  * </p>
  * 
  * @lucene.internal