lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From romseyg...@apache.org
Subject svn commit: r1412849 [8/13] - in /lucene/dev/branches/LUCENE-2878: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/idea/solr/contrib/dataimporthandler/ dev-tools/maven/ dev-tools/maven/solr/contrib/dataimporthandler/ dev-tool...
Date Fri, 23 Nov 2012 12:01:26 GMT
Modified: lucene/dev/branches/LUCENE-2878/lucene/queries/src/java/org/apache/lucene/queries/TermsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/queries/src/java/org/apache/lucene/queries/TermsFilter.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/queries/src/java/org/apache/lucene/queries/TermsFilter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/queries/src/java/org/apache/lucene/queries/TermsFilter.java Fri Nov 23 12:00:32 2012
@@ -21,13 +21,17 @@ import org.apache.lucene.index.*;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Filter;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 
 import java.io.IOException;
-import java.util.Set;
-import java.util.TreeSet;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
 
 /**
  * Constructs a filter for docs matching any of the terms added to this class.
@@ -36,53 +40,171 @@ import java.util.TreeSet;
  * a choice of "category" labels picked by the end user. As a filter, this is much faster than the
  * equivalent query (a BooleanQuery with many "should" TermQueries)
  */
-public class TermsFilter extends Filter {
-
-  private final Set<Term> terms = new TreeSet<Term>();
+public final class TermsFilter extends Filter {
 
+  /*
+   * this class is often used for large number of terms in a single field.
+   * to optimize for this case and to be filter-cache friendly we 
+   * serialize all terms into a single byte array and store offsets
+   * in a parallel array to keep the # of object constant and speed up
+   * equals / hashcode.
+   * 
+   * This adds quite a bit of complexity but allows large term filters to
+   * be efficient for GC and cache-lookups
+   */
+  private final int[] offsets;
+  private final byte[] termsBytes;
+  private final TermsAndField[] termsAndFields;
+  private final int hashCode; // cached hashcode for fast cache lookups
+  private static final int PRIME = 31;
+  
   /**
-   * Adds a term to the list of acceptable terms
+   * Creates a new {@link TermsFilter} from the given list. The list
+   * can contain duplicate terms and multiple fields.
    */
-  public void addTerm(Term term) {
-    terms.add(term);
+  public TermsFilter(final List<Term> terms) {
+    this(new FieldAndTermEnum() {
+      // we need to sort for deduplication and to have a common cache key
+      final Iterator<Term> iter = sort(terms).iterator();
+      @Override
+      public BytesRef next() {
+        if (iter.hasNext()) {
+          Term next = iter.next();
+          field = next.field();
+          return next.bytes();
+        }
+        return null;
+      }}, terms.size());
   }
-
-/* (non-Javadoc)
-   * @see org.apache.lucene.search.Filter#getDocIdSet(org.apache.lucene.index.IndexReader)
+  
+  /**
+   * Creates a new {@link TermsFilter} from the given {@link BytesRef} list for
+   * a single field.
+   */
+  public TermsFilter(final String field, final List<BytesRef> terms) {
+    this(new FieldAndTermEnum(field) {
+      // we need to sort for deduplication and to have a common cache key
+      final Iterator<BytesRef> iter = sort(terms).iterator();
+      @Override
+      public BytesRef next() {
+        if (iter.hasNext()) {
+          return iter.next();
+        }
+        return null;
+      }
+    }, terms.size());
+  }
+  
+  /**
+   * Creates a new {@link TermsFilter} from the given {@link BytesRef} array for
+   * a single field.
+   */
+  public TermsFilter(final String field, final BytesRef...terms) {
+    // this ctor prevents unnecessary Term creations
+   this(field, Arrays.asList(terms));
+  }
+  
+  /**
+   * Creates a new {@link TermsFilter} from the given array. The array can
+   * contain duplicate terms and multiple fields.
    */
+  public TermsFilter(final Term... terms) {
+    this(Arrays.asList(terms));
+  }
+  
+  
+  private TermsFilter(FieldAndTermEnum iter, int length) {
+    // TODO: maybe use oal.index.PrefixCodedTerms instead?
+    // If number of terms is more than a few hundred it
+    // should be a win
+
+    // TODO: we also pack terms in FieldCache/DocValues
+    // ... maybe we can refactor to share that code
+
+    // TODO: yet another option is to build the union of the terms in
+    // an automaton an call intersect on the termsenum if the density is high
 
+    int hash = 9;
+    byte[] serializedTerms = new byte[0];
+    this.offsets = new int[length+1];
+    int lastEndOffset = 0;
+    int index = 0;
+    ArrayList<TermsAndField> termsAndFields = new ArrayList<TermsAndField>();
+    TermsAndField lastTermsAndField = null;
+    BytesRef previousTerm = null;
+    String previousField = null;
+    BytesRef currentTerm;
+    String currentField;
+    while((currentTerm = iter.next()) != null) {
+      currentField = iter.field();
+      if (currentField == null) {
+        throw new IllegalArgumentException("Field must not be null");
+      }
+      if (previousField != null) {
+        // deduplicate
+        if (previousField.equals(currentField)) {
+          if (previousTerm.bytesEquals(currentTerm)){
+            continue;            
+          }
+        } else {
+          final int start = lastTermsAndField == null ? 0 : lastTermsAndField.end;
+          lastTermsAndField = new TermsAndField(start, index, previousField);
+          termsAndFields.add(lastTermsAndField);
+        }
+      }
+      hash = PRIME *  hash + currentField.hashCode();
+      hash = PRIME *  hash + currentTerm.hashCode();
+      if (serializedTerms.length < lastEndOffset+currentTerm.length) {
+        serializedTerms = ArrayUtil.grow(serializedTerms, lastEndOffset+currentTerm.length);
+      }
+      System.arraycopy(currentTerm.bytes, currentTerm.offset, serializedTerms, lastEndOffset, currentTerm.length);
+      offsets[index] = lastEndOffset; 
+      lastEndOffset += currentTerm.length;
+      index++;
+      previousTerm = currentTerm;
+      previousField = currentField;
+    }
+    offsets[index] = lastEndOffset;
+    final int start = lastTermsAndField == null ? 0 : lastTermsAndField.end;
+    lastTermsAndField = new TermsAndField(start, index, previousField);
+    termsAndFields.add(lastTermsAndField);
+    this.termsBytes = ArrayUtil.shrink(serializedTerms, lastEndOffset);
+    this.termsAndFields = termsAndFields.toArray(new TermsAndField[termsAndFields.size()]);
+    this.hashCode = hash;
+    
+  }
+  
+  
   @Override
   public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
-    AtomicReader reader = context.reader();
-    FixedBitSet result = new FixedBitSet(reader.maxDoc());
-    Fields fields = reader.fields();
-
+    final AtomicReader reader = context.reader();
+    FixedBitSet result = null;  // lazy init if needed - no need to create a big bitset ahead of time
+    final Fields fields = reader.fields();
+    final BytesRef spare = new BytesRef(this.termsBytes);
     if (fields == null) {
       return result;
     }
-
-    BytesRef br = new BytesRef();
-    String lastField = null;
-    Terms termsC;
+    Terms terms = null;
     TermsEnum termsEnum = null;
     DocsEnum docs = null;
-    for (Term term : terms) {
-      if (!term.field().equals(lastField)) {
-        termsC = fields.terms(term.field());
-        if (termsC == null) {
-          return result;
-        }
-        termsEnum = termsC.iterator(null);
-        lastField = term.field();
-      }
-
-      if (terms != null) { // TODO this check doesn't make sense, decide which variable its supposed to be for
-        br.copyBytes(term.bytes());
-        assert termsEnum != null;
-        if (termsEnum.seekExact(br,true)) {
-          docs = termsEnum.docs(acceptDocs, docs, 0);
-          while (docs.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
-            result.set(docs.docID());
+    for (TermsAndField termsAndField : this.termsAndFields) {
+      if ((terms = fields.terms(termsAndField.field)) != null) {
+        termsEnum = terms.iterator(termsEnum); // this won't return null
+        for (int i = termsAndField.start; i < termsAndField.end; i++) {
+          spare.offset = offsets[i];
+          spare.length = offsets[i+1] - offsets[i];
+          if (termsEnum.seekExact(spare, false)) { // don't use cache since we could pollute the cache here easily
+            docs = termsEnum.docs(acceptDocs, docs, 0); // no freq since we don't need them
+            if (result == null) {
+              if (docs.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
+                result = new FixedBitSet(reader.maxDoc());
+                // lazy init but don't do it in the hot loop since we could read many docs
+                result.set(docs.docID());
+              }
+            }
+            while (docs.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
+              result.set(docs.docID());
+            }
           }
         }
       }
@@ -98,19 +220,114 @@ public class TermsFilter extends Filter 
     if ((obj == null) || (obj.getClass() != this.getClass())) {
       return false;
     }
-
+    
     TermsFilter test = (TermsFilter) obj;
-    return (terms == test.terms ||
-        (terms != null && terms.equals(test.terms)));
+    if (test.hashCode == hashCode && this.termsAndFields.length == test.termsAndFields.length) {
+      // first check the fields before even comparing the bytes
+      for (int i = 0; i < termsAndFields.length; i++) {
+        TermsAndField current = termsAndFields[i];
+        if (!current.equals(test.termsAndFields[i])) {
+          return false;
+        }
+      }
+      // straight byte comparison since we sort they must be identical
+      int end = offsets[termsAndFields.length];
+      byte[] left = this.termsBytes;
+      byte[] right = test.termsBytes;
+      for(int i=0;i < end;i++) {
+        if (left[i] != right[i]) {
+          return false;
+        }
+      }
+      return true;
+    }
+    return false;
   }
 
   @Override
   public int hashCode() {
-    int hash = 9;
-    for (Term term : terms) {
-      hash = 31 * hash + term.hashCode();
+    return hashCode;
+  }
+  
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    BytesRef spare = new BytesRef(termsBytes);
+    for (int i = 0; i < termsAndFields.length; i++) {
+      TermsAndField current = termsAndFields[i];
+      for (int j = current.start; j < current.end; j++) {
+        spare.offset = offsets[j];
+        spare.length = offsets[j+1] - offsets[j];
+        builder.append(current.field).append(':');
+        builder.append(spare.utf8ToString());
+        builder.append(' ');
+      }
+
     }
-    return hash;
+    return builder.toString();
   }
+  
+  private static final class TermsAndField {
+    final int start;
+    final int end;
+    final String field;
+    
+    
+    TermsAndField(int start, int end, String field) {
+      super();
+      this.start = start;
+      this.end = end;
+      this.field = field;
+    }
 
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((field == null) ? 0 : field.hashCode());
+      result = prime * result + end;
+      result = prime * result + start;
+      return result;
+    }
+    
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) return true;
+      if (obj == null) return false;
+      if (getClass() != obj.getClass()) return false;
+      TermsAndField other = (TermsAndField) obj;
+      if (field == null) {
+        if (other.field != null) return false;
+      } else if (!field.equals(other.field)) return false;
+      if (end != other.end) return false;
+      if (start != other.start) return false;
+      return true;
+    }
+    
+  }
+  
+  private static abstract class FieldAndTermEnum {
+    protected String field;
+    
+    public abstract BytesRef next();
+    
+    public FieldAndTermEnum() {}
+    
+    public FieldAndTermEnum(String field) { this.field = field; }
+    
+    public String field() {
+      return field;
+    }
+  }
+  
+  /*
+   * simple utility that returns the in-place sorted list
+   */
+  private static <T extends Comparable<? super T>> List<T> sort(List<T> toSort) {
+    if (toSort.isEmpty()) {
+      throw new IllegalArgumentException("no terms provided");
+    }
+    Collections.sort(toSort);
+    return toSort;
+  }
 }

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

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

Modified: lucene/dev/branches/LUCENE-2878/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSource.java Fri Nov 23 12:00:32 2012
@@ -89,9 +89,8 @@ public abstract class ValueSource {
    *
    * @param reverse true if this is a reverse sort.
    * @return The {@link org.apache.lucene.search.SortField} for the ValueSource
-   * @throws IOException if there was a problem reading the values.
    */
-  public SortField getSortField(boolean reverse) throws IOException {
+  public SortField getSortField(boolean reverse) {
     return new ValueSourceSortField(reverse);
   }
 

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

Modified: lucene/dev/branches/LUCENE-2878/lucene/queries/src/test/org/apache/lucene/queries/BooleanFilterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/queries/src/test/org/apache/lucene/queries/BooleanFilterTest.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/queries/src/test/org/apache/lucene/queries/BooleanFilterTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/queries/src/test/org/apache/lucene/queries/BooleanFilterTest.java Fri Nov 23 12:00:32 2012
@@ -82,10 +82,7 @@ public class BooleanFilterTest extends L
   }
 
   private Filter getTermsFilter(String field, String text) {
-    TermsFilter tf = new TermsFilter();
-    tf.addTerm(new Term(field, text));
-
-    return tf;
+    return new TermsFilter(new Term(field, text));
   }
   
   private Filter getWrappedTermQuery(String field, String text) {

Modified: lucene/dev/branches/LUCENE-2878/lucene/queries/src/test/org/apache/lucene/queries/TermsFilterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/queries/src/test/org/apache/lucene/queries/TermsFilterTest.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/queries/src/test/org/apache/lucene/queries/TermsFilterTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/queries/src/test/org/apache/lucene/queries/TermsFilterTest.java Fri Nov 23 12:00:32 2012
@@ -17,7 +17,14 @@ package org.apache.lucene.queries;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -27,28 +34,34 @@ import org.apache.lucene.index.MultiRead
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.SlowCompositeReaderWrapper;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 
 public class TermsFilterTest extends LuceneTestCase {
 
   public void testCachability() throws Exception {
-    TermsFilter a = new TermsFilter();
-    a.addTerm(new Term("field1", "a"));
-    a.addTerm(new Term("field1", "b"));
+    TermsFilter a = termsFilter(random().nextBoolean(), new Term("field1", "a"), new Term("field1", "b"));
     HashSet<Filter> cachedFilters = new HashSet<Filter>();
     cachedFilters.add(a);
-    TermsFilter b = new TermsFilter();
-    b.addTerm(new Term("field1", "a"));
-    b.addTerm(new Term("field1", "b"));
-
-    assertTrue("Must be cached", cachedFilters.contains(b));
-    b.addTerm(new Term("field1", "a")); //duplicate term
+    TermsFilter b = termsFilter(random().nextBoolean(), new Term("field1", "b"), new Term("field1", "a"));
     assertTrue("Must be cached", cachedFilters.contains(b));
-    b.addTerm(new Term("field1", "c"));
-    assertFalse("Must not be cached", cachedFilters.contains(b));
+    //duplicate term
+    assertTrue("Must be cached", cachedFilters.contains(termsFilter(true, new Term("field1", "a"), new Term("field1", "a"), new Term("field1", "b"))));
+    assertFalse("Must not be cached", cachedFilters.contains(termsFilter(random().nextBoolean(), new Term("field1", "a"), new Term("field1", "a"), new Term("field1", "b"),  new Term("field1", "v"))));
   }
 
   public void testMissingTerms() throws Exception {
@@ -66,21 +79,21 @@ public class TermsFilterTest extends Luc
     AtomicReaderContext context = (AtomicReaderContext) reader.getContext();
     w.close();
 
-    TermsFilter tf = new TermsFilter();
-    tf.addTerm(new Term(fieldName, "19"));
-    FixedBitSet bits = (FixedBitSet) tf.getDocIdSet(context, context.reader().getLiveDocs());
-    assertEquals("Must match nothing", 0, bits.cardinality());
+    List<Term> terms = new ArrayList<Term>();
+    terms.add(new Term(fieldName, "19"));
+    FixedBitSet bits = (FixedBitSet) termsFilter(random().nextBoolean(), terms).getDocIdSet(context, context.reader().getLiveDocs());
+    assertNull("Must match nothing", bits);
 
-    tf.addTerm(new Term(fieldName, "20"));
-    bits = (FixedBitSet) tf.getDocIdSet(context, context.reader().getLiveDocs());
+    terms.add(new Term(fieldName, "20"));
+    bits = (FixedBitSet) termsFilter(random().nextBoolean(), terms).getDocIdSet(context, context.reader().getLiveDocs());
     assertEquals("Must match 1", 1, bits.cardinality());
 
-    tf.addTerm(new Term(fieldName, "10"));
-    bits = (FixedBitSet) tf.getDocIdSet(context, context.reader().getLiveDocs());
+    terms.add(new Term(fieldName, "10"));
+    bits = (FixedBitSet) termsFilter(random().nextBoolean(), terms).getDocIdSet(context, context.reader().getLiveDocs());
     assertEquals("Must match 2", 2, bits.cardinality());
 
-    tf.addTerm(new Term(fieldName, "00"));
-    bits = (FixedBitSet) tf.getDocIdSet(context, context.reader().getLiveDocs());
+    terms.add(new Term(fieldName, "00"));
+    bits = (FixedBitSet) termsFilter(random().nextBoolean(), terms).getDocIdSet(context, context.reader().getLiveDocs());
     assertEquals("Must match 2", 2, bits.cardinality());
 
     reader.close();
@@ -106,13 +119,16 @@ public class TermsFilterTest extends Luc
     IndexReader reader2 = w2.getReader();
     w2.close();
     
-    TermsFilter tf = new TermsFilter();
-    tf.addTerm(new Term(fieldName, "content1"));
-    
+    TermsFilter tf = new TermsFilter(new Term(fieldName, "content1"));
     MultiReader multi = new MultiReader(reader1, reader2);
     for (AtomicReaderContext context : multi.leaves()) {
-      FixedBitSet bits = (FixedBitSet) tf.getDocIdSet(context, context.reader().getLiveDocs());
-      assertTrue("Must be >= 0", bits.cardinality() >= 0);      
+      DocIdSet docIdSet = tf.getDocIdSet(context, context.reader().getLiveDocs());
+      if (context.reader().docFreq(new Term(fieldName, "content1")) == 0) {
+        assertNull(docIdSet);
+      } else {
+        FixedBitSet bits = (FixedBitSet) docIdSet;
+        assertTrue("Must be >= 0", bits.cardinality() >= 0);      
+      }
     }
     multi.close();
     reader1.close();
@@ -120,5 +136,188 @@ public class TermsFilterTest extends Luc
     rd1.close();
     rd2.close();
   }
+  
+  public void testFieldNotPresent() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    int num = atLeast(3);
+    int skip = random().nextInt(num);
+    List<Term> terms = new ArrayList<Term>();
+    for (int i = 0; i < num; i++) {
+      terms.add(new Term("field" + i, "content1"));
+      Document doc = new Document();
+      if (skip == i) {
+        continue;
+      }
+      doc.add(newStringField("field" + i, "content1", Field.Store.YES));
+      w.addDocument(doc);  
+    }
+    
+    w.forceMerge(1);
+    IndexReader reader = w.getReader();
+    w.close();
+    assertEquals(1, reader.leaves().size());
+    
+    
+    
+    AtomicReaderContext context = reader.leaves().get(0);
+    TermsFilter tf = new TermsFilter(terms);
+
+    FixedBitSet bits = (FixedBitSet) tf.getDocIdSet(context, context.reader().getLiveDocs());
+    assertEquals("Must be num fields - 1 since we skip only one field", num-1, bits.cardinality());  
+    reader.close();
+    dir.close();
+  }
+  
+  public void testSkipField() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    int num = atLeast(10);
+    Set<Term> terms = new HashSet<Term>();
+    for (int i = 0; i < num; i++) {
+      String field = "field" + random().nextInt(100);
+      terms.add(new Term(field, "content1"));
+      Document doc = new Document();
+      doc.add(newStringField(field, "content1", Field.Store.YES));
+      w.addDocument(doc);
+    }
+    int randomFields = random().nextInt(10);
+    for (int i = 0; i < randomFields; i++) {
+      while (true) {
+        String field = "field" + random().nextInt(100);
+        Term t = new Term(field, "content1");
+        if (!terms.contains(t)) {
+          terms.add(t);
+          break;
+        }
+      }
+    }
+    w.forceMerge(1);
+    IndexReader reader = w.getReader();
+    w.close();
+    assertEquals(1, reader.leaves().size());
+    AtomicReaderContext context = reader.leaves().get(0);
+    TermsFilter tf = new TermsFilter(new ArrayList<Term>(terms));
+
+    FixedBitSet bits = (FixedBitSet) tf.getDocIdSet(context, context.reader().getLiveDocs());
+    assertEquals(context.reader().numDocs(), bits.cardinality());  
+    reader.close();
+    dir.close();
+  }
+  
+  public void testRandom() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    int num = atLeast(100);
+    final boolean singleField = random().nextBoolean();
+    List<Term> terms = new ArrayList<Term>();
+    for (int i = 0; i < num; i++) {
+      String field = "field" + (singleField ? "1" : random().nextInt(100));
+      String string = _TestUtil.randomRealisticUnicodeString(random());
+      terms.add(new Term(field, string));
+      Document doc = new Document();
+      doc.add(newStringField(field, string, Field.Store.YES));
+      w.addDocument(doc);
+    }
+    IndexReader reader = w.getReader();
+    w.close();
+    
+    IndexSearcher searcher = new IndexSearcher(reader);
+    
+    int numQueries = atLeast(10);
+    for (int i = 0; i < numQueries; i++) {
+      Collections.shuffle(terms, random());
+      int numTerms = 1 + random().nextInt(
+          Math.min(BooleanQuery.getMaxClauseCount(), terms.size()));
+      BooleanQuery bq = new BooleanQuery();
+      for (int j = 0; j < numTerms; j++) {
+        bq.add(new BooleanClause(new TermQuery(terms.get(j)), Occur.SHOULD));
+      }
+      TopDocs queryResult = searcher.search(new ConstantScoreQuery(bq), reader.maxDoc());
+      
+      MatchAllDocsQuery matchAll = new MatchAllDocsQuery();
+      final TermsFilter filter = termsFilter(singleField, terms.subList(0, numTerms));;
+      TopDocs filterResult = searcher.search(matchAll, filter, reader.maxDoc());
+      assertEquals(filterResult.totalHits, queryResult.totalHits);
+      ScoreDoc[] scoreDocs = filterResult.scoreDocs;
+      for (int j = 0; j < scoreDocs.length; j++) {
+        assertEquals(scoreDocs[j].doc, queryResult.scoreDocs[j].doc);
+      }
+    }
+    
+    reader.close();
+    dir.close();
+  }
+  
+  private TermsFilter termsFilter(boolean singleField, Term...terms) {
+    return termsFilter(singleField, Arrays.asList(terms));
+  }
 
+  private TermsFilter termsFilter(boolean singleField, Collection<Term> termList) {
+    if (!singleField) {
+      return new TermsFilter(new ArrayList<Term>(termList));
+    }
+    final TermsFilter filter;
+    List<BytesRef> bytes = new ArrayList<BytesRef>();
+    String field = null;
+    for (Term term : termList) {
+        bytes.add(term.bytes());
+        if (field != null) {
+          assertEquals(term.field(), field);
+        }
+        field = term.field();
+    }
+    assertNotNull(field);
+    filter = new TermsFilter(field, bytes);
+    return filter;
+  }
+  
+  public void testHashCodeAndEquals() {
+    int num = atLeast(100);
+    final boolean singleField = random().nextBoolean();
+    List<Term> terms = new ArrayList<Term>();
+    Set<Term> uniqueTerms = new HashSet<Term>();
+    for (int i = 0; i < num; i++) {
+      String field = "field" + (singleField ? "1" : random().nextInt(100));
+      String string = _TestUtil.randomRealisticUnicodeString(random());
+      terms.add(new Term(field, string));
+      uniqueTerms.add(new Term(field, string));
+      TermsFilter left = termsFilter(singleField ? random().nextBoolean() : false, uniqueTerms);
+      Collections.shuffle(terms, random());
+      TermsFilter right = termsFilter(singleField ? random().nextBoolean() : false, terms);
+      assertEquals(right, left);
+      assertEquals(right.hashCode(), left.hashCode());
+      if (uniqueTerms.size() > 1) {
+        List<Term> asList = new ArrayList<Term>(uniqueTerms);
+        asList.remove(0);
+        TermsFilter notEqual = termsFilter(singleField ? random().nextBoolean() : false, asList);
+        assertFalse(left.equals(notEqual));
+        assertFalse(right.equals(notEqual));
+      }
+    }
+  }
+  
+  public void testNoTerms() {
+    List<Term> emptyTerms = Collections.emptyList();
+    List<BytesRef> emptyBytesRef = Collections.emptyList();
+    try {
+      new TermsFilter(emptyTerms);
+      fail("must fail - no terms!");
+    } catch (IllegalArgumentException e) {}
+    
+    try {
+      new TermsFilter(emptyTerms.toArray(new Term[0]));
+      fail("must fail - no terms!");
+    } catch (IllegalArgumentException e) {}
+    
+    try {
+      new TermsFilter(null, emptyBytesRef.toArray(new BytesRef[0]));
+      fail("must fail - no terms!");
+    } catch (IllegalArgumentException e) {}
+    
+    try {
+      new TermsFilter(null, emptyBytesRef);
+      fail("must fail - no terms!");
+    } catch (IllegalArgumentException e) {}
+  }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/TermsFilterBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/TermsFilterBuilder.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/TermsFilterBuilder.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/TermsFilterBuilder.java Fri Nov 23 12:00:32 2012
@@ -14,6 +14,8 @@ import org.w3c.dom.Element;
 
 import java.io.IOException;
 import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.List;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -49,7 +51,7 @@ public class TermsFilterBuilder implemen
     * @see org.apache.lucene.xmlparser.FilterBuilder#process(org.w3c.dom.Element)
     */
   public Filter getFilter(Element e) throws ParserException {
-    TermsFilter tf = new TermsFilter();
+    List<BytesRef> terms = new ArrayList<BytesRef>();
     String text = DOMUtils.getNonBlankTextOrFail(e);
     String fieldName = DOMUtils.getAttributeWithInheritanceOrFail(e, "fieldName");
 
@@ -61,8 +63,7 @@ public class TermsFilterBuilder implemen
       ts.reset();
       while (ts.incrementToken()) {
         termAtt.fillBytesRef();
-        term = new Term(fieldName, BytesRef.deepCopyOf(bytes));
-        tf.addTerm(term);
+        terms.add(BytesRef.deepCopyOf(bytes));
       }
       ts.end();
       ts.close();
@@ -70,6 +71,6 @@ public class TermsFilterBuilder implemen
     catch (IOException ioe) {
       throw new RuntimeException("Error constructing terms from index:" + ioe);
     }
-    return tf;
+    return new TermsFilter(fieldName, terms);
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/sandbox/src/test/org/apache/lucene/sandbox/queries/TestSlowCollationMethods.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/sandbox/src/test/org/apache/lucene/sandbox/queries/TestSlowCollationMethods.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/sandbox/src/test/org/apache/lucene/sandbox/queries/TestSlowCollationMethods.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/sandbox/src/test/org/apache/lucene/sandbox/queries/TestSlowCollationMethods.java Fri Nov 23 12:00:32 2012
@@ -147,10 +147,40 @@ public class TestSlowCollationMethods ex
     }
   }
   
-  public void testQuery() {
+  public void testQuery() throws Exception {
+
+    // Copied from beforeClass, but scaled down to few docs:
+    // since otherwise this test can run for a very long
+    // time (1-2 hours or more; see Lucene-Solr-4.x-Linux Build #2204):
+    final Locale locale = LuceneTestCase.randomLocale(random());
+    Collator collator = Collator.getInstance(locale);
+    collator.setStrength(Collator.IDENTICAL);
+    collator.setDecomposition(Collator.NO_DECOMPOSITION);
+
+    int numDocs = 20 * RANDOM_MULTIPLIER;
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      String value = _TestUtil.randomUnicodeString(random());
+      Field field = newStringField("field", value, Field.Store.YES);
+      doc.add(field);
+      iw.addDocument(doc);
+    }
+    IndexReader reader = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(reader);
+
     String startPoint = _TestUtil.randomUnicodeString(random());
     String endPoint = _TestUtil.randomUnicodeString(random());
     Query query = new SlowCollatedTermRangeQuery("field", startPoint, endPoint, true, true, collator);
     QueryUtils.check(random(), query, searcher);
+    reader.close();
+    dir.close();
+    collator = null;
+    searcher = null;
+    reader = null;
+    dir = null;
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java Fri Nov 23 12:00:32 2012
@@ -18,7 +18,6 @@ package org.apache.lucene.spatial.prefix
  */
 
 import com.spatial4j.core.shape.Shape;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.TermsFilter;
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.spatial.prefix.tree.Node;
@@ -26,6 +25,7 @@ import org.apache.lucene.spatial.prefix.
 import org.apache.lucene.spatial.query.SpatialArgs;
 import org.apache.lucene.spatial.query.SpatialOperation;
 import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
+import org.apache.lucene.util.BytesRef;
 
 import java.util.List;
 
@@ -55,11 +55,12 @@ public class TermQueryPrefixTreeStrategy
     Shape shape = args.getShape();
     int detailLevel = grid.getLevelForDistance(args.resolveDistErr(ctx, distErrPct));
     List<Node> cells = grid.getNodes(shape, detailLevel, false);
-    TermsFilter filter = new TermsFilter();
+    BytesRef[] terms = new BytesRef[cells.size()];
+    int i = 0;
     for (Node cell : cells) {
-      filter.addTerm(new Term(getFieldName(), cell.getTokenString()));
+      terms[i++] = new BytesRef(cell.getTokenString());
     }
-    return filter;
+    return new TermsFilter(getFieldName(), terms);
   }
 
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefList.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefList.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefList.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefList.java Fri Nov 23 12:00:32 2012
@@ -64,7 +64,7 @@ public final class BytesRefList {
     lastElement = 0;
     currentOffset = 0;
     Arrays.fill(offsets, 0);
-    pool.reset();
+    pool.reset(false, true); // no need to 0 fill the buffers we control the allocator
   }
   
   /**
@@ -101,10 +101,10 @@ public final class BytesRefList {
    */
   public BytesRef get(BytesRef spare, int ord) {
     if (lastElement > ord) {
-      spare.offset = offsets[ord];
-      spare.length = ord == lastElement - 1 ? currentOffset - spare.offset
-          : offsets[ord + 1] - spare.offset;
-      pool.copyFrom(spare);
+      int offset = offsets[ord];
+      int length = ord == lastElement - 1 ? currentOffset - offset
+          : offsets[ord + 1] - offset;
+      pool.copyFrom(spare, offset, length);
       return spare;
     }
     throw new IndexOutOfBoundsException("index " + ord

Modified: lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedTermFreqIteratorWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedTermFreqIteratorWrapper.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedTermFreqIteratorWrapper.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedTermFreqIteratorWrapper.java Fri Nov 23 12:00:32 2012
@@ -41,28 +41,33 @@ public class SortedTermFreqIteratorWrapp
   private File tempInput;
   private File tempSorted;
   private final ByteSequencesReader reader;
+  private final Comparator<BytesRef> comparator;
   private boolean done = false;
   
   private long weight;
   private final BytesRef scratch = new BytesRef();
-  private final Comparator<BytesRef> comparator;
   
-  /** 
-   * Calls {@link #SortedTermFreqIteratorWrapper(TermFreqIterator, Comparator, boolean) 
-   * SortedTermFreqIteratorWrapper(source, comparator, false)}
-   */
-  public SortedTermFreqIteratorWrapper(TermFreqIterator source, Comparator<BytesRef> comparator) throws IOException {
-    this(source, comparator, false);
+  /**
+   * Creates a new sorted wrapper, using {@link
+   * BytesRef#getUTF8SortedAsUnicodeComparator} for
+   * sorting. */
+  public SortedTermFreqIteratorWrapper(TermFreqIterator source) throws IOException {
+    this(source, BytesRef.getUTF8SortedAsUnicodeComparator());
   }
-  
+
   /**
-   * Creates a new sorted wrapper. if <code>compareRawBytes</code> is true, then
-   * only the bytes (not the weight) will be used for comparison.
+   * Creates a new sorted wrapper, sorting by BytesRef
+   * (ascending) then cost (ascending).
    */
-  public SortedTermFreqIteratorWrapper(TermFreqIterator source, Comparator<BytesRef> comparator, boolean compareRawBytes) throws IOException {
+  public SortedTermFreqIteratorWrapper(TermFreqIterator source, Comparator<BytesRef> comparator) throws IOException {
     this.source = source;
     this.comparator = comparator;
-    this.reader = sort(compareRawBytes ? comparator : new BytesOnlyComparator(this.comparator));
+    this.reader = sort();
+  }
+  
+  @Override
+  public Comparator<BytesRef> getComparator() {
+    return comparator;
   }
   
   @Override
@@ -90,16 +95,43 @@ public class SortedTermFreqIteratorWrapp
   }
   
   @Override
-  public Comparator<BytesRef> getComparator() {
-    return comparator;
-  }
-  
-  @Override
   public long weight() {
     return weight;
   }
+
+  /** Sortes by BytesRef (ascending) then cost (ascending). */
+  private final Comparator<BytesRef> tieBreakByCostComparator = new Comparator<BytesRef>() {
+
+    private final BytesRef leftScratch = new BytesRef();
+    private final BytesRef rightScratch = new BytesRef();
+    private final ByteArrayDataInput input = new ByteArrayDataInput();
+    
+    @Override
+    public int compare(BytesRef left, BytesRef right) {
+      // Make shallow copy in case decode changes the BytesRef:
+      leftScratch.bytes = left.bytes;
+      leftScratch.offset = left.offset;
+      leftScratch.length = left.length;
+      rightScratch.bytes = right.bytes;
+      rightScratch.offset = right.offset;
+      rightScratch.length = right.length;
+      long leftCost = decode(leftScratch, input);
+      long rightCost = decode(rightScratch, input);
+      int cmp = comparator.compare(leftScratch, rightScratch);
+      if (cmp != 0) {
+        return cmp;
+      }
+      if (leftCost < rightCost) {
+        return -1;
+      } else if (rightCost < leftCost) {
+        return 1;
+      } else {
+        return 0;
+      }
+    }
+  };
   
-  private Sort.ByteSequencesReader sort(Comparator<BytesRef> comparator) throws IOException {
+  private Sort.ByteSequencesReader sort() throws IOException {
     String prefix = getClass().getSimpleName();
     File directory = Sort.defaultTempDir();
     tempInput = File.createTempFile(prefix, ".input", directory);
@@ -116,7 +148,7 @@ public class SortedTermFreqIteratorWrapp
         encode(writer, output, buffer, spare, source.weight());
       }
       writer.close();
-      new Sort(comparator).sort(tempInput, tempSorted);
+      new Sort(tieBreakByCostComparator).sort(tempInput, tempSorted);
       ByteSequencesReader reader = new Sort.ByteSequencesReader(tempSorted);
       success = true;
       return reader;
@@ -131,7 +163,6 @@ public class SortedTermFreqIteratorWrapp
           close();
         }
       }
-      
     }
   }
   
@@ -145,31 +176,6 @@ public class SortedTermFreqIteratorWrapp
     }
   }
   
-  private final static class BytesOnlyComparator implements Comparator<BytesRef> {
-
-    final Comparator<BytesRef> other;
-    private final BytesRef leftScratch = new BytesRef();
-    private final BytesRef rightScratch = new BytesRef();
-    
-    public BytesOnlyComparator(Comparator<BytesRef> other) {
-      this.other = other;
-    }
-
-    @Override
-    public int compare(BytesRef left, BytesRef right) {
-      wrap(leftScratch, left);
-      wrap(rightScratch, right);
-      return other.compare(leftScratch, rightScratch);
-    }
-    
-    private void wrap(BytesRef wrapper, BytesRef source) {
-      wrapper.bytes = source.bytes;
-      wrapper.offset = source.offset;
-      wrapper.length = source.length - 8;
-      
-    }
-  }
-  
   /** encodes an entry (bytes+weight) to the provided writer */
   protected void encode(ByteSequencesWriter writer, ByteArrayDataOutput output, byte[] buffer, BytesRef spare, long weight) throws IOException {
     if (spare.length + 8 >= buffer.length) {
@@ -184,9 +190,8 @@ public class SortedTermFreqIteratorWrapp
   /** decodes the weight at the current position */
   protected long decode(BytesRef scratch, ByteArrayDataInput tmpInput) {
     tmpInput.reset(scratch.bytes);
-    tmpInput.skipBytes(scratch.length - 8); // suggestion + separator
-    scratch.length -= 8; // sep + long
+    tmpInput.skipBytes(scratch.length - 8); // suggestion
+    scratch.length -= 8; // long
     return tmpInput.readLong();
   }
-  
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java Fri Nov 23 12:00:32 2012
@@ -23,6 +23,7 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.StringReader;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
 import java.util.List;
@@ -72,14 +73,24 @@ import org.apache.lucene.util.fst.Util;
  * This can result in powerful suggester functionality.  For
  * example, if you use an analyzer removing stop words, 
  * then the partial text "ghost chr..." could see the
- * suggestion "The Ghost of Christmas Past".  If
- * SynonymFilter is used to map wifi and wireless network to
+ * suggestion "The Ghost of Christmas Past". Note that
+ * your {@code StopFilter} instance must NOT preserve
+ * position increments for this example to work, so you should call
+ * {@code setEnablePositionIncrements(false)} on it.
+ *
+ * <p>
+ * If SynonymFilter is used to map wifi and wireless network to
  * hotspot then the partial text "wirele..." could suggest
  * "wifi router".  Token normalization like stemmers, accent
  * removal, etc., would allow suggestions to ignore such
  * variations.
  *
  * <p>
+ * When two matching suggestions have the same weight, they
+ * are tie-broken by the analyzed form.  If their analyzed
+ * form is the same then the order is undefined.
+ *
+ * <p>
  * There are some limitations:
  * <ul>
  *
@@ -310,7 +321,7 @@ public class AnalyzingSuggester extends 
     }
   }
 
-  private TokenStreamToAutomaton getTokenStreamToAutomaton() {
+  TokenStreamToAutomaton getTokenStreamToAutomaton() {
     if (preserveSep) {
       return new EscapingTokenStreamToAutomaton();
     } else {
@@ -319,6 +330,56 @@ public class AnalyzingSuggester extends 
       return new TokenStreamToAutomaton();
     }
   }
+
+  private  Comparator<BytesRef> sortComparator = new Comparator<BytesRef>() {
+    private final ByteArrayDataInput readerA = new ByteArrayDataInput();
+    private final ByteArrayDataInput readerB = new ByteArrayDataInput();
+    private final BytesRef scratchA = new BytesRef();
+    private final BytesRef scratchB = new BytesRef();
+
+    @Override
+    public int compare(BytesRef a, BytesRef b) {
+
+      // First by analyzed form:
+      readerA.reset(a.bytes, a.offset, a.length);
+      scratchA.length = readerA.readShort();
+      scratchA.bytes = a.bytes;
+      scratchA.offset = readerA.getPosition();
+
+      readerB.reset(b.bytes, b.offset, b.length);
+      scratchB.bytes = b.bytes;
+      scratchB.length = readerB.readShort();
+      scratchB.offset = readerB.getPosition();
+
+      int cmp = scratchA.compareTo(scratchB);
+      if (cmp != 0) {
+        return cmp;
+      }
+
+      // Next by cost:
+      long aCost = readerA.readInt();
+      long bCost = readerB.readInt();
+
+      if (aCost < bCost) {
+        return -1;
+      } else if (aCost > bCost) {
+        return 1;
+      }
+
+      // Finally by surface form:
+      scratchA.offset = readerA.getPosition();
+      scratchA.length = a.length - scratchA.offset;
+      scratchB.offset = readerB.getPosition();
+      scratchB.length = b.length - scratchB.offset;
+
+      cmp = scratchA.compareTo(scratchB);
+      if (cmp != 0) {
+        return cmp;
+      }
+
+      return 0;
+    }
+  };
   
   @Override
   public void build(TermFreqIterator iterator) throws IOException {
@@ -332,36 +393,17 @@ public class AnalyzingSuggester extends 
     BytesRef scratch = new BytesRef();
 
     TokenStreamToAutomaton ts2a = getTokenStreamToAutomaton();
+
     // analyzed sequence + 0(byte) + weight(int) + surface + analyzedLength(short) 
     boolean success = false;
     byte buffer[] = new byte[8];
     try {
       ByteArrayDataOutput output = new ByteArrayDataOutput(buffer);
       BytesRef surfaceForm;
-      while ((surfaceForm = iterator.next()) != null) {
-
-        // Analyze surface form:
-        TokenStream ts = indexAnalyzer.tokenStream("", new StringReader(surfaceForm.utf8ToString()));
 
-        // Create corresponding automaton: labels are bytes
-        // from each analyzed token, with byte 0 used as
-        // separator between tokens:
-        Automaton automaton = ts2a.toAutomaton(ts);
-        ts.end();
-        ts.close();
-
-        replaceSep(automaton);
-
-        assert SpecialOperations.isFinite(automaton);
-
-        // Get all paths from the automaton (there can be
-        // more than one path, eg if the analyzer created a
-        // graph using SynFilter or WDF):
-
-        // TODO: we could walk & add simultaneously, so we
-        // don't have to alloc [possibly biggish]
-        // intermediate HashSet in RAM:
-        Set<IntsRef> paths = SpecialOperations.getFiniteStrings(automaton, maxGraphExpansions);
+      while ((surfaceForm = iterator.next()) != null) {
+        Set<IntsRef> paths = toFiniteStrings(surfaceForm, ts2a);
+        
         maxAnalyzedPathsForOneInput = Math.max(maxAnalyzedPathsForOneInput, paths.size());
 
         for (IntsRef path : paths) {
@@ -369,81 +411,94 @@ public class AnalyzingSuggester extends 
           Util.toBytesRef(path, scratch);
           
           // length of the analyzed text (FST input)
+          if (scratch.length > Short.MAX_VALUE-2) {
+            throw new IllegalArgumentException("cannot handle analyzed forms > " + (Short.MAX_VALUE-2) + " in length (got " + scratch.length + ")");
+          }
           short analyzedLength = (short) scratch.length;
+
           // compute the required length:
-          // analyzed sequence + 12 (separator) + weight (4) + surface + analyzedLength (short)
-          int requiredLength = analyzedLength + 2 + 4 + surfaceForm.length + 2;
+          // analyzed sequence + weight (4) + surface + analyzedLength (short)
+          int requiredLength = analyzedLength + 4 + surfaceForm.length + 2;
           
           buffer = ArrayUtil.grow(buffer, requiredLength);
           
           output.reset(buffer);
+
+          output.writeShort(analyzedLength);
+
           output.writeBytes(scratch.bytes, scratch.offset, scratch.length);
-          output.writeByte((byte)0); // separator: not used, just for sort order
-          output.writeByte((byte)0); // separator: not used, just for sort order
 
-          // NOTE: important that writeInt is big-endian,
-          // because this means we sort secondarily by
-          // cost ascending (= weight descending) so that
-          // when we discard too many surface forms for a
-          // single analyzed form we are discarding the
-          // least weight ones:
           output.writeInt(encodeWeight(iterator.weight()));
 
           output.writeBytes(surfaceForm.bytes, surfaceForm.offset, surfaceForm.length);
-          output.writeShort(analyzedLength);
+
+          assert output.getPosition() == requiredLength: output.getPosition() + " vs " + requiredLength;
+
           writer.write(buffer, 0, output.getPosition());
         }
       }
       writer.close();
 
       // Sort all input/output pairs (required by FST.Builder):
-      new Sort().sort(tempInput, tempSorted);
+      new Sort(sortComparator).sort(tempInput, tempSorted);
+
+      // Free disk space:
+      tempInput.delete();
+
       reader = new Sort.ByteSequencesReader(tempSorted);
      
       PairOutputs<Long,BytesRef> outputs = new PairOutputs<Long,BytesRef>(PositiveIntOutputs.getSingleton(true), ByteSequenceOutputs.getSingleton());
       Builder<Pair<Long,BytesRef>> builder = new Builder<Pair<Long,BytesRef>>(FST.INPUT_TYPE.BYTE1, outputs);
 
       // Build FST:
-      BytesRef previous = null;
+      BytesRef previousAnalyzed = null;
       BytesRef analyzed = new BytesRef();
       BytesRef surface = new BytesRef();
       IntsRef scratchInts = new IntsRef();
       ByteArrayDataInput input = new ByteArrayDataInput();
 
+      // Used to remove duplicate surface forms (but we
+      // still index the hightest-weight one).  We clear
+      // this when we see a new analyzed form, so it cannot
+      // grow unbounded (at most 256 entries):
+      Set<BytesRef> seenSurfaceForms = new HashSet<BytesRef>();
+
       int dedup = 0;
       while (reader.read(scratch)) {
         input.reset(scratch.bytes, scratch.offset, scratch.length);
-        input.setPosition(input.length()-2);
         short analyzedLength = input.readShort();
-
-        analyzed.bytes = scratch.bytes;
-        analyzed.offset = scratch.offset;
+        analyzed.grow(analyzedLength+2);
+        input.readBytes(analyzed.bytes, 0, analyzedLength);
         analyzed.length = analyzedLength;
-        
-        input.setPosition(analyzedLength + 2); // analyzed sequence + separator
+
         long cost = input.readInt();
-   
+
         surface.bytes = scratch.bytes;
         surface.offset = input.getPosition();
-        surface.length = input.length() - input.getPosition() - 2;
-
-        if (previous == null) {
-          previous = new BytesRef();
-          previous.copyBytes(analyzed);
-        } else if (analyzed.equals(previous)) {
+        surface.length = scratch.length - surface.offset;
+        
+        if (previousAnalyzed == null) {
+          previousAnalyzed = new BytesRef();
+          previousAnalyzed.copyBytes(analyzed);
+          seenSurfaceForms.add(BytesRef.deepCopyOf(surface));
+        } else if (analyzed.equals(previousAnalyzed)) {
           dedup++;
           if (dedup >= maxSurfaceFormsPerAnalyzedForm) {
             // More than maxSurfaceFormsPerAnalyzedForm
             // dups: skip the rest:
             continue;
           }
+          if (seenSurfaceForms.contains(surface)) {
+            continue;
+          }
+          seenSurfaceForms.add(BytesRef.deepCopyOf(surface));
         } else {
           dedup = 0;
-          previous.copyBytes(analyzed);
+          previousAnalyzed.copyBytes(analyzed);
+          seenSurfaceForms.clear();
+          seenSurfaceForms.add(BytesRef.deepCopyOf(surface));
         }
 
-        analyzed.grow(analyzed.length+2);
-
         // TODO: I think we can avoid the extra 2 bytes when
         // there is no dup (dedup==0), but we'd have to fix
         // the exactFirst logic ... which would be sort of
@@ -452,8 +507,8 @@ public class AnalyzingSuggester extends 
 
         // NOTE: must be byte 0 so we sort before whatever
         // is next
-        analyzed.bytes[analyzed.length] = 0;
-        analyzed.bytes[analyzed.length+1] = (byte) dedup;
+        analyzed.bytes[analyzed.offset+analyzed.length] = 0;
+        analyzed.bytes[analyzed.offset+analyzed.length+1] = (byte) dedup;
         analyzed.length += 2;
 
         Util.toIntsRef(analyzed, scratchInts);
@@ -481,6 +536,10 @@ public class AnalyzingSuggester extends 
   public boolean store(OutputStream output) throws IOException {
     DataOutput dataOut = new OutputStreamDataOutput(output);
     try {
+      if (fst == null) {
+        return false;
+      }
+
       fst.save(dataOut);
       dataOut.writeVInt(maxAnalyzedPathsForOneInput);
     } finally {
@@ -508,29 +567,15 @@ public class AnalyzingSuggester extends 
     if (onlyMorePopular) {
       throw new IllegalArgumentException("this suggester only works with onlyMorePopular=false");
     }
+    if (fst == null) {
+      return Collections.emptyList();
+    }
 
     //System.out.println("lookup key=" + key + " num=" + num);
-
+    final BytesRef utf8Key = new BytesRef(key);
     try {
 
-      // TODO: is there a Reader from a CharSequence?
-      // Turn tokenstream into automaton:
-      TokenStream ts = queryAnalyzer.tokenStream("", new StringReader(key.toString()));
-      Automaton automaton = getTokenStreamToAutomaton().toAutomaton(ts);
-      ts.end();
-      ts.close();
-
-      // TODO: we could use the end offset to "guess"
-      // whether the final token was a partial token; this
-      // would only be a heuristic ... but maybe an OK one.
-      // This way we could eg differentiate "net" from "net ",
-      // which we can't today...
-
-      replaceSep(automaton);
-
-      // TODO: we can optimize this somewhat by determinizing
-      // while we convert
-      BasicOperations.determinize(automaton);
+      Automaton lookupAutomaton = toLookupAutomaton(key);
 
       final CharsRef spare = new CharsRef();
 
@@ -538,8 +583,7 @@ public class AnalyzingSuggester extends 
     
       // Intersect automaton w/ suggest wFST and get all
       // prefix starting nodes & their outputs:
-      final List<FSTUtil.Path<Pair<Long,BytesRef>>> prefixPaths;
-      prefixPaths = FSTUtil.intersectPrefixPaths(automaton, fst);
+      //final PathIntersector intersector = getPathIntersector(lookupAutomaton, fst);
 
       //System.out.println("  prefixPaths: " + prefixPaths.size());
 
@@ -549,6 +593,8 @@ public class AnalyzingSuggester extends 
 
       final List<LookupResult> results = new ArrayList<LookupResult>();
 
+      List<FSTUtil.Path<Pair<Long,BytesRef>>> prefixPaths = FSTUtil.intersectPrefixPaths(lookupAutomaton, fst);
+
       if (exactFirst) {
 
         int count = 0;
@@ -593,9 +639,9 @@ public class AnalyzingSuggester extends 
         // nodes we have and the
         // maxSurfaceFormsPerAnalyzedForm:
         for(MinResult<Pair<Long,BytesRef>> completion : completions) {
-          spare.grow(completion.output.output2.length);
-          UnicodeUtil.UTF8toUTF16(completion.output.output2, spare);
-          if (CHARSEQUENCE_COMPARATOR.compare(spare, key) == 0) {
+          if (utf8Key.bytesEquals(completion.output.output2)) {
+            spare.grow(completion.output.output2.length);
+            UnicodeUtil.UTF8toUTF16(completion.output.output2, spare);
             results.add(new LookupResult(spare.toString(), decodeWeight(completion.output.output1)));
             break;
           }
@@ -630,9 +676,7 @@ public class AnalyzingSuggester extends 
             // In exactFirst mode, don't accept any paths
             // matching the surface form since that will
             // create duplicate results:
-            spare.grow(output.output2.length);
-            UnicodeUtil.UTF8toUTF16(output.output2, spare);
-            if (CHARSEQUENCE_COMPARATOR.compare(spare, key) == 0) {
+            if (utf8Key.bytesEquals(output.output2)) {
               // We found exact match, which means we should
               // have already found it in the first search:
               assert results.size() == 1;
@@ -644,6 +688,8 @@ public class AnalyzingSuggester extends 
         }
       };
 
+      prefixPaths = getFullPrefixPaths(prefixPaths, lookupAutomaton, fst);
+      
       for (FSTUtil.Path<Pair<Long,BytesRef>> path : prefixPaths) {
         searcher.addStartPaths(path.fstNode, path.output, true, path.input);
       }
@@ -654,6 +700,10 @@ public class AnalyzingSuggester extends 
         spare.grow(completion.output.output2.length);
         UnicodeUtil.UTF8toUTF16(completion.output.output2, spare);
         LookupResult result = new LookupResult(spare.toString(), decodeWeight(completion.output.output1));
+
+        // TODO: for fuzzy case would be nice to return
+        // how many edits were required
+
         //System.out.println("    result=" + result);
         results.add(result);
 
@@ -670,6 +720,63 @@ public class AnalyzingSuggester extends 
     }
   }
 
+  /** Returns all prefix paths to initialize the search. */
+  protected List<FSTUtil.Path<Pair<Long,BytesRef>>> getFullPrefixPaths(List<FSTUtil.Path<Pair<Long,BytesRef>>> prefixPaths,
+                                                                       Automaton lookupAutomaton,
+                                                                       FST<Pair<Long,BytesRef>> fst)
+    throws IOException {
+    return prefixPaths;
+  }
+  
+  final Set<IntsRef> toFiniteStrings(final BytesRef surfaceForm, final TokenStreamToAutomaton ts2a) throws IOException {
+ // Analyze surface form:
+    TokenStream ts = indexAnalyzer.tokenStream("", new StringReader(surfaceForm.utf8ToString()));
+
+    // Create corresponding automaton: labels are bytes
+    // from each analyzed token, with byte 0 used as
+    // separator between tokens:
+    Automaton automaton = ts2a.toAutomaton(ts);
+    ts.end();
+    ts.close();
+
+    replaceSep(automaton);
+
+    assert SpecialOperations.isFinite(automaton);
+
+    // Get all paths from the automaton (there can be
+    // more than one path, eg if the analyzer created a
+    // graph using SynFilter or WDF):
+
+    // TODO: we could walk & add simultaneously, so we
+    // don't have to alloc [possibly biggish]
+    // intermediate HashSet in RAM:
+    return SpecialOperations.getFiniteStrings(automaton, maxGraphExpansions);
+  }
+
+  final Automaton toLookupAutomaton(final CharSequence key) throws IOException {
+    // TODO: is there a Reader from a CharSequence?
+    // Turn tokenstream into automaton:
+    TokenStream ts = queryAnalyzer.tokenStream("", new StringReader(key.toString()));
+    Automaton automaton = (getTokenStreamToAutomaton()).toAutomaton(ts);
+    ts.end();
+    ts.close();
+
+    // TODO: we could use the end offset to "guess"
+    // whether the final token was a partial token; this
+    // would only be a heuristic ... but maybe an OK one.
+    // This way we could eg differentiate "net" from "net ",
+    // which we can't today...
+
+    replaceSep(automaton);
+
+    // TODO: we can optimize this somewhat by determinizing
+    // while we convert
+    BasicOperations.determinize(automaton);
+    return automaton;
+  }
+  
+  
+
   /**
    * Returns the weight associated with an input string,
    * or null if it does not exist.

Modified: lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FSTUtil.java Fri Nov 23 12:00:32 2012
@@ -26,6 +26,7 @@ import org.apache.lucene.util.automaton.
 import org.apache.lucene.util.automaton.State;
 import org.apache.lucene.util.automaton.Transition;
 import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.Util;
 
 // TODO: move to core?  nobody else uses it yet though...
 
@@ -62,57 +63,78 @@ public class FSTUtil {
     }
   }
 
-  /** Enumerates all paths in the automaton that also
-   *  intersect the FST, accumulating the FST end node and
-   *  output for each path. */
-  public static<T> List<Path<T>> intersectPrefixPaths(Automaton a, FST<T> fst) throws IOException {
+  /**
+   * Enumerates all minimal prefix paths in the automaton that also intersect the FST,
+   * accumulating the FST end node and output for each path.
+   */
+  public static <T> List<Path<T>> intersectPrefixPaths(Automaton a, FST<T> fst)
+      throws IOException {
+    assert a.isDeterministic();
     final List<Path<T>> queue = new ArrayList<Path<T>>();
     final List<Path<T>> endNodes = new ArrayList<Path<T>>();
-
-    queue.add(new Path<T>(a.getInitialState(),
-                          fst.getFirstArc(new FST.Arc<T>()),       
-                          fst.outputs.getNoOutput(),
-                          new IntsRef()));
-
+    queue.add(new Path<T>(a.getInitialState(), fst
+        .getFirstArc(new FST.Arc<T>()), fst.outputs.getNoOutput(),
+        new IntsRef()));
+    
     final FST.Arc<T> scratchArc = new FST.Arc<T>();
     final FST.BytesReader fstReader = fst.getBytesReader(0);
-
-    //System.out.println("fst/a intersect");
-
+    
     while (queue.size() != 0) {
-      final Path<T> path = queue.remove(queue.size()-1);
-      //System.out.println("  cycle path=" + path);
+      final Path<T> path = queue.remove(queue.size() - 1);
       if (path.state.isAccept()) {
         endNodes.add(path);
+        // we can stop here if we accept this path,
+        // we accept all further paths too
+        continue;
       }
-
+      
       IntsRef currentInput = path.input;
-      for(Transition t : path.state.getTransitions()) {
-        
-        // TODO: we can fix this if necessary:
-        if (t.getMin() != t.getMax()) {
-          throw new IllegalStateException("can only handle Transitions that match one character");
-        }
-
-        //System.out.println("    t=" + (char) t.getMin());
-
-        final FST.Arc<T> nextArc = fst.findTargetArc(t.getMin(), path.fstNode, scratchArc, fstReader);
-        if (nextArc != null) {
-          //System.out.println("      fst matches");
-          // Path continues:
-          IntsRef newInput = new IntsRef(currentInput.length + 1);
-          newInput.copyInts(currentInput);
-          newInput.ints[currentInput.length] = t.getMin();
-          newInput.length = currentInput.length + 1;
-
-          queue.add(new Path<T>(t.getDest(),
-                                new FST.Arc<T>().copyFrom(nextArc),
-                                fst.outputs.add(path.output, nextArc.output),
-                                newInput));
+      for (Transition t : path.state.getTransitions()) {
+        final int min = t.getMin();
+        final int max = t.getMax();
+        if (min == max) {
+          final FST.Arc<T> nextArc = fst.findTargetArc(t.getMin(),
+              path.fstNode, scratchArc, fstReader);
+          if (nextArc != null) {
+            final IntsRef newInput = new IntsRef(currentInput.length + 1);
+            newInput.copyInts(currentInput);
+            newInput.ints[currentInput.length] = t.getMin();
+            newInput.length = currentInput.length + 1;
+            queue.add(new Path<T>(t.getDest(), new FST.Arc<T>()
+                .copyFrom(nextArc), fst.outputs
+                .add(path.output, nextArc.output), newInput));
+          }
+        } else {
+          // TODO: if this transition's TO state is accepting, and
+          // it accepts the entire range possible in the FST (ie. 0 to 255),
+          // we can simply use the prefix as the accepted state instead of
+          // looking up all the ranges and terminate early
+          // here.  This just shifts the work from one queue
+          // (this one) to another (the completion search
+          // done in AnalyzingSuggester).
+          FST.Arc<T> nextArc = Util.readCeilArc(min, fst, path.fstNode,
+              scratchArc, fstReader);
+          while (nextArc != null && nextArc.label <= max) {
+            assert nextArc.label <=  max;
+            assert nextArc.label >= min : nextArc.label + " "
+                + min;
+            final IntsRef newInput = new IntsRef(currentInput.length + 1);
+            newInput.copyInts(currentInput);
+            newInput.ints[currentInput.length] = nextArc.label;
+            newInput.length = currentInput.length + 1;
+            queue.add(new Path<T>(t.getDest(), new FST.Arc<T>()
+                .copyFrom(nextArc), fst.outputs
+                .add(path.output, nextArc.output), newInput));
+            final int label = nextArc.label; // used in assert
+            nextArc = nextArc.isLast() ? null : fst.readNextRealArc(nextArc,
+                fstReader);
+            assert nextArc == null || label < nextArc.label : "last: " + label
+                + " next: " + nextArc.label;
+          }
         }
       }
     }
-
     return endNodes;
   }
+  
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletion.java Fri Nov 23 12:00:32 2012
@@ -109,7 +109,7 @@ public class FSTCompletion {
    *          Find and push an exact match to the first position of the result
    *          list if found.
    */
-  @SuppressWarnings("unchecked")
+  @SuppressWarnings({"unchecked","rawtypes"})
   public FSTCompletion(FST<Object> automaton, boolean higherWeightsFirst, boolean exactFirst) {
     this.automaton = automaton;
     if (automaton != null) {
@@ -133,7 +133,7 @@ public class FSTCompletion {
    * Cache the root node's output arcs starting with completions with the
    * highest weights.
    */
-  @SuppressWarnings({"all"})
+  @SuppressWarnings({"unchecked","rawtypes"})
   private static Arc<Object>[] cacheRootArcs(FST<Object> automaton) {
     try {
       List<Arc<Object>> rootArcs = new ArrayList<Arc<Object>>();

Modified: lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java Fri Nov 23 12:00:32 2012
@@ -94,8 +94,7 @@ public class WFSTCompletionLookup extend
   @Override
   public void build(TermFreqIterator iterator) throws IOException {
     BytesRef scratch = new BytesRef();
-    TermFreqIterator iter = new WFSTTermFreqIteratorWrapper(iterator,
-        BytesRef.getUTF8SortedAsUnicodeComparator());
+    TermFreqIterator iter = new WFSTTermFreqIteratorWrapper(iterator);
     IntsRef scratchInts = new IntsRef();
     BytesRef previous = null;
     PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
@@ -148,6 +147,10 @@ public class WFSTCompletionLookup extend
       throw new IllegalArgumentException("this suggester only works with onlyMorePopular=false");
     }
 
+    if (fst == null) {
+      return Collections.emptyList();
+    }
+
     BytesRef scratch = new BytesRef(key);
     int prefixLength = scratch.length;
     Arc<Long> arc = new Arc<Long>();
@@ -220,6 +223,9 @@ public class WFSTCompletionLookup extend
    * or null if it does not exist.
    */
   public Object get(CharSequence key) {
+    if (fst == null) {
+      return null;
+    }
     Arc<Long> arc = new Arc<Long>();
     Long result = null;
     try {
@@ -247,28 +253,26 @@ public class WFSTCompletionLookup extend
   
   private final class WFSTTermFreqIteratorWrapper extends SortedTermFreqIteratorWrapper {
 
-    WFSTTermFreqIteratorWrapper(TermFreqIterator source,
-        Comparator<BytesRef> comparator) throws IOException {
-      super(source, comparator, true);
+    WFSTTermFreqIteratorWrapper(TermFreqIterator source) throws IOException {
+      super(source);
     }
 
     @Override
     protected void encode(ByteSequencesWriter writer, ByteArrayDataOutput output, byte[] buffer, BytesRef spare, long weight) throws IOException {
-      if (spare.length + 5 >= buffer.length) {
-        buffer = ArrayUtil.grow(buffer, spare.length + 5);
+      if (spare.length + 4 >= buffer.length) {
+        buffer = ArrayUtil.grow(buffer, spare.length + 4);
       }
       output.reset(buffer);
       output.writeBytes(spare.bytes, spare.offset, spare.length);
-      output.writeByte((byte)0); // separator: not used, just for sort order
       output.writeInt(encodeWeight(weight));
       writer.write(buffer, 0, output.getPosition());
     }
     
     @Override
     protected long decode(BytesRef scratch, ByteArrayDataInput tmpInput) {
-      tmpInput.reset(scratch.bytes);
-      tmpInput.skipBytes(scratch.length - 4); // suggestion + separator
-      scratch.length -= 5; // sep + long
+      scratch.length -= 4; // int
+      // skip suggestion:
+      tmpInput.reset(scratch.bytes, scratch.offset+scratch.length, 4);
       return tmpInput.readInt();
     }
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java Fri Nov 23 12:00:32 2012
@@ -36,6 +36,7 @@ import org.apache.lucene.analysis.MockAn
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.search.suggest.Lookup; // javadocs
 import org.apache.lucene.search.suggest.analyzing.AnalyzingSuggester;
+import org.apache.lucene.search.suggest.analyzing.FuzzySuggester;
 import org.apache.lucene.search.suggest.fst.FSTCompletionLookup;
 import org.apache.lucene.search.suggest.fst.WFSTCompletionLookup;
 import org.apache.lucene.search.suggest.jaspell.JaspellLookup;
@@ -51,17 +52,20 @@ import org.junit.Ignore;
 public class LookupBenchmarkTest extends LuceneTestCase {
   @SuppressWarnings("unchecked")
   private final List<Class<? extends Lookup>> benchmarkClasses = Arrays.asList(
+      FuzzySuggester.class,
+      AnalyzingSuggester.class,
       JaspellLookup.class, 
       TSTLookup.class,
       FSTCompletionLookup.class,
-      WFSTCompletionLookup.class,
-      AnalyzingSuggester.class);
+      WFSTCompletionLookup.class
+      
+      );
 
   private final static int rounds = 15;
   private final static int warmup = 5;
 
   private final int num = 7;
-  private final boolean onlyMorePopular = true;
+  private final boolean onlyMorePopular = false;
 
   private final static Random random = new Random(0xdeadbeef);
 
@@ -212,8 +216,9 @@ public class LookupBenchmarkTest extends
       final List<String> input = new ArrayList<String>(benchmarkInput.size());
       for (TermFreq tf : benchmarkInput) {
         String s = tf.term.utf8ToString();
-        input.add(s.substring(0, Math.min(s.length(), 
-              minPrefixLen + random.nextInt(maxPrefixLen - minPrefixLen + 1))));
+        String sub = s.substring(0, Math.min(s.length(), 
+            minPrefixLen + random.nextInt(maxPrefixLen - minPrefixLen + 1)));
+        input.add(sub);
       }
 
       BenchmarkResult result = measure(new Callable<Integer>() {
@@ -250,7 +255,9 @@ public class LookupBenchmarkTest extends
       }
       return new BenchmarkResult(times, warmup, rounds);
     } catch (Exception e) {
+      e.printStackTrace();
       throw new RuntimeException(e);
+      
     }
   }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestTermFreqIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestTermFreqIterator.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestTermFreqIterator.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestTermFreqIterator.java Fri Nov 23 12:00:32 2012
@@ -80,50 +80,6 @@ public class TestTermFreqIterator extend
     assertEquals(sorted, actual);
   }
   
-  
-  public void testRaw() throws Exception {
-    int num = atLeast(10000);
-    
-    Comparator<BytesRef> comparator = BytesRef.getUTF8SortedAsUnicodeComparator();
-    BytesRefHash sorted = new BytesRefHash();
-    TermFreq[] unsorted = new TermFreq[num];
-    byte[] buffer = new byte[0];
-    ByteArrayDataOutput output = new ByteArrayDataOutput(buffer);
-
-    final Random random = new Random(random().nextLong());
-    for (int i = 0; i < num; i++) {
-      BytesRef spare;
-      long weight;
-      do {
-        spare = new BytesRef(_TestUtil.randomUnicodeString(random));
-        if (spare.length + 8 >= buffer.length) {
-          buffer = ArrayUtil.grow(buffer, spare.length + 8);
-        }
-        output.reset(buffer);
-        output.writeBytes(spare.bytes, spare.offset, spare.length);
-        weight = random.nextLong();
-        output.writeLong(weight);
-        
-      } while (sorted.add(new BytesRef(buffer, 0, output.getPosition())) < 0);
-      unsorted[i] = new TermFreq(spare, weight);
-    }
-    
-    // test the sorted iterator wrapper
-    TermFreqIterator wrapper = new SortedTermFreqIteratorWrapper(new TermFreqArrayIterator(unsorted), comparator, true);
-    int[] sort = sorted.sort(comparator);
-    int size = sorted.size();
-    BytesRef spare = new BytesRef();
-    for (int i = 0; i < size; i++) {
-      sorted.get(sort[i], spare);
-      spare.length -= 8; // sub the long value
-      assertEquals(spare, wrapper.next());
-      spare.offset = spare.offset + spare.length;
-      spare.length = 8;
-      assertEquals(asLong(spare), wrapper.weight());
-    }
-    assertNull(wrapper.next());
-  }
-  
   public static long asLong(BytesRef b) {
     return (((long) asIntInternal(b, b.offset) << 32) | asIntInternal(b,
         b.offset + 4) & 0xFFFFFFFFL);



Mime
View raw message