You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2012/09/23 03:14:07 UTC

svn commit: r1388936 - in /lucene/dev/branches/lucene3842/lucene: core/src/java/org/apache/lucene/analysis/ core/src/java/org/apache/lucene/util/fst/ core/src/test/org/apache/lucene/analysis/ core/src/test/org/apache/lucene/index/ suggest/src/java/org/...

Author: mikemccand
Date: Sun Sep 23 01:14:06 2012
New Revision: 1388936

URL: http://svn.apache.org/viewvc?rev=1388936&view=rev
Log:
LUCENE-3842: get escaping working; fix exactFirst to mean exact surface form

Added:
    lucene/dev/branches/lucene3842/lucene/test-framework/src/java/org/apache/lucene/analysis/CannedBinaryTokenStream.java   (with props)
Modified:
    lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
    lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
    lucene/dev/branches/lucene3842/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
    lucene/dev/branches/lucene3842/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java
    lucene/dev/branches/lucene3842/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionLookup.java
    lucene/dev/branches/lucene3842/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
    lucene/dev/branches/lucene3842/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionTest.java

Modified: lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java?rev=1388936&r1=1388935&r2=1388936&view=diff
==============================================================================
--- lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java (original)
+++ lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/analysis/TokenStreamToAutomaton.java Sun Sep 23 01:14:06 2012
@@ -62,6 +62,13 @@ public class TokenStreamToAutomaton {
     }
   }
 
+  /** Subclass & implement this if you need to change the
+   *  token (such as escaping certain bytes) before it's
+   *  turned into a graph. */ 
+  protected BytesRef changeToken(BytesRef in) {
+    return in;
+  }
+
   /** We create transition between two adjacent tokens. */
   public static final int POS_SEP = 256;
 
@@ -72,7 +79,7 @@ public class TokenStreamToAutomaton {
    *  PositionLengthAttribute}) from the provided {@link
    *  TokenStream}, and creates the corresponding
    *  automaton where arcs are bytes from each term. */
-  public static Automaton toAutomaton(TokenStream in) throws IOException {
+  public Automaton toAutomaton(TokenStream in) throws IOException {
     final Automaton a = new Automaton();
 
     final TermToBytesRefAttribute termBytesAtt = in.addAttribute(TermToBytesRefAttribute.class);
@@ -131,15 +138,16 @@ public class TokenStreamToAutomaton {
       final int endPos = pos + posLengthAtt.getPositionLength();
 
       termBytesAtt.fillBytesRef();
+      final BytesRef term2 = changeToken(term);
       final Position endPosData = positions.get(endPos);
       if (endPosData.arriving == null) {
         endPosData.arriving = new State();
       }
 
       State state = posData.leaving;
-      for(int byteIDX=0;byteIDX<term.length;byteIDX++) {
-        final State nextState = byteIDX == term.length-1 ? endPosData.arriving : new State();
-        state.addTransition(new Transition(term.bytes[term.offset + byteIDX] & 0xff, nextState));
+      for(int byteIDX=0;byteIDX<term2.length;byteIDX++) {
+        final State nextState = byteIDX == term2.length-1 ? endPosData.arriving : new State();
+        state.addTransition(new Transition(term2.bytes[term2.offset + byteIDX] & 0xff, nextState));
         state = nextState;
       }
     }

Modified: lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/util/fst/Util.java?rev=1388936&r1=1388935&r2=1388936&view=diff
==============================================================================
--- lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/branches/lucene3842/lucene/core/src/java/org/apache/lucene/util/fst/Util.java Sun Sep 23 01:14:06 2012
@@ -472,7 +472,10 @@ public final class Util {
           if (path.arc.label == FST.END_LABEL) {
             // Add final output:
             //System.out.println("    done!: " + path);
-            results.add(new MinResult<T>(path.input, fst.outputs.add(path.cost, path.arc.output), comparator));
+            T finalOutput = fst.outputs.add(path.cost, path.arc.output);
+            if (acceptResult(path.input, finalOutput)) {
+              results.add(new MinResult<T>(path.input, finalOutput, comparator));
+            }
             break;
           } else {
             path.input.grow(1+path.input.length);
@@ -487,6 +490,10 @@ public final class Util {
         (MinResult<T>[]) new MinResult[results.size()];
       return results.toArray(arr);
     }
+
+    protected boolean acceptResult(IntsRef input, T output) {
+      return true;
+    }
   }
 
   /** Holds a single input (IntsRef) + output, returned by
@@ -830,7 +837,8 @@ public final class Util {
     scratch.grow(input.length);
     for(int i=0;i<input.length;i++) {
       int value = input.ints[i+input.offset];
-      assert value >= Byte.MIN_VALUE && value <= Byte.MAX_VALUE: "value " + value + " doesn't fit into byte";
+      // NOTE: we allow -128 to 255
+      assert value >= Byte.MIN_VALUE && value <= 255: "value " + value + " doesn't fit into byte";
       scratch.bytes[i] = (byte) value;
     }
     scratch.length = input.length;

Modified: lucene/dev/branches/lucene3842/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3842/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java?rev=1388936&r1=1388935&r2=1388936&view=diff
==============================================================================
--- lucene/dev/branches/lucene3842/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java (original)
+++ lucene/dev/branches/lucene3842/lucene/core/src/test/org/apache/lucene/analysis/TestGraphTokenizers.java Sun Sep 23 01:14:06 2012
@@ -416,7 +416,7 @@ public class TestGraphTokenizers extends
       new Token[] {
         token("abc", 1, 1),
       });
-    final Automaton actual = TokenStreamToAutomaton.toAutomaton(ts);
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
     final Automaton expected = BasicAutomata.makeString("abc");
     assertTrue(BasicOperations.sameLanguage(expected, actual));
   }
@@ -459,7 +459,7 @@ public class TestGraphTokenizers extends
         token("abc", 1, 1),
         token("def", 1, 1),
       });
-    final Automaton actual = TokenStreamToAutomaton.toAutomaton(ts);
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
     final Automaton expected =  join("abc", "def");
 
     //toDot(actual);
@@ -473,7 +473,7 @@ public class TestGraphTokenizers extends
         token("abc", 1, 1),
         token("def", 2, 1),
       });
-    final Automaton actual = TokenStreamToAutomaton.toAutomaton(ts);
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
 
     final Automaton expected = join(s2a("abc"), SEP_A, HOLE_A, SEP_A, s2a("def"));
 
@@ -489,7 +489,7 @@ public class TestGraphTokenizers extends
         token("abc", 1, 1),
         token("xyz", 0, 1)
       });
-    final Automaton actual = TokenStreamToAutomaton.toAutomaton(ts);
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
     final Automaton a1 = BasicAutomata.makeString("abc");
     final Automaton a2 = BasicAutomata.makeString("xyz");
     final Automaton expected = BasicOperations.union(a1, a2);
@@ -504,7 +504,7 @@ public class TestGraphTokenizers extends
         token("xyz", 0, 2),
         token("def", 1, 1),
       });
-    final Automaton actual = TokenStreamToAutomaton.toAutomaton(ts);
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
     final Automaton a1 = BasicAutomata.makeString("xyz");
     final Automaton a2 = join("abc", "def");
                                                                    
@@ -521,7 +521,7 @@ public class TestGraphTokenizers extends
         token("X", 0, 2),
         token("b", 2, 1),
       });
-    final Automaton actual = TokenStreamToAutomaton.toAutomaton(ts);
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
     final Automaton a1 = BasicOperations.union(
                                                join(s2a("a"), SEP_A, HOLE_A),
                                                BasicAutomata.makeString("X"));
@@ -539,7 +539,7 @@ public class TestGraphTokenizers extends
         token("abc", 0, 3),
         token("def", 2, 1),
       });
-    final Automaton actual = TokenStreamToAutomaton.toAutomaton(ts);
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
     final Automaton expected = BasicOperations.union(
                                                      join(s2a("xyz"), SEP_A, HOLE_A, SEP_A, s2a("def")),
                                                      BasicAutomata.makeString("abc"));
@@ -555,7 +555,7 @@ public class TestGraphTokenizers extends
         token("def", 1, 1),
         token("ghi", 1, 1),
       });
-    final Automaton actual = TokenStreamToAutomaton.toAutomaton(ts);
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
     final Automaton a1 = BasicAutomata.makeString("xyz");
     final Automaton a2 = join("abc", "def", "ghi");
     final Automaton expected = BasicOperations.union(a1, a2);
@@ -575,7 +575,7 @@ public class TestGraphTokenizers extends
       new Token[] {
         token("abc", 2, 1),
       });
-    final Automaton actual = TokenStreamToAutomaton.toAutomaton(ts);
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
     final Automaton expected = join(HOLE_A, SEP_A, s2a("abc"));
     //toDot(actual);
     assertTrue(BasicOperations.sameLanguage(expected, actual));
@@ -589,7 +589,7 @@ public class TestGraphTokenizers extends
         token("a", 1, 1),
         token("X", 0, 10),
       });
-    final Automaton actual = TokenStreamToAutomaton.toAutomaton(ts);
+    final Automaton actual = (new TokenStreamToAutomaton()).toAutomaton(ts);
     final Automaton expected = BasicOperations.union(BasicAutomata.makeString("a"),
                                                      BasicAutomata.makeString("X"));
     assertTrue(BasicOperations.sameLanguage(expected, actual));

Modified: lucene/dev/branches/lucene3842/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3842/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java?rev=1388936&r1=1388935&r2=1388936&view=diff
==============================================================================
--- lucene/dev/branches/lucene3842/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java (original)
+++ lucene/dev/branches/lucene3842/lucene/core/src/test/org/apache/lucene/index/BinaryTokenStream.java Sun Sep 23 01:14:06 2012
@@ -25,6 +25,7 @@ import org.apache.lucene.util.BytesRef;
 /**
  * a binary tokenstream that lets you index a BytesRef
  */
+// nocommit absorb into CannedBinaryTS?
 public final class BinaryTokenStream extends TokenStream {
   private final ByteTermAttribute bytesAtt = addAttribute(ByteTermAttribute.class);
   private boolean available = true;
@@ -63,6 +64,8 @@ public final class BinaryTokenStream ext
     }
     
     public void setBytesRef(BytesRef bytes) {
+      // nocommit isn't this dangerous?  the getBytesRef is
+      // pre-shared?  so you can't set this per token?
       this.bytes = bytes;
     }
     

Modified: lucene/dev/branches/lucene3842/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3842/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionLookup.java?rev=1388936&r1=1388935&r2=1388936&view=diff
==============================================================================
--- lucene/dev/branches/lucene3842/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionLookup.java (original)
+++ lucene/dev/branches/lucene3842/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionLookup.java Sun Sep 23 01:14:06 2012
@@ -29,6 +29,7 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.TokenStreamToAutomaton;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
@@ -113,41 +114,78 @@ public class AnalyzingCompletionLookup e
   private final boolean preserveSep;
 
   /**
-   * Calls {@link #AnalyzingCompletionLookup(Analyzer,int) AnalyzingCompletionLookup(analyzer, EXACT_FIRST | PRESERVE_SEP)}
+   * Calls {@link #AnalyzingCompletionLookup(Analyzer,int,int,int)
+   * AnalyzingCompletionLookup(analyzer, EXACT_FIRST |
+   * PRESERVE_SEP, 256, -1)}
    */
   public AnalyzingCompletionLookup(Analyzer analyzer) {
-    this(analyzer, EXACT_FIRST | PRESERVE_SEP);
+    this(analyzer, EXACT_FIRST | PRESERVE_SEP, 256, -1);
   }
 
-  /** Include this flag in the options parameter {@link
-   * #AnalyzingCompletionLookup(Analyzer,int)} to always
-   * return exact matches first regardless of score.  This
-   * has no performance impact but could result in
-   * low-quality suggestions. */
+  /** Include this flag in the options parameter to {@link
+   *  #AnalyzingCompletionLookup(Analyzer,int,int,int)} to always
+   *  return the exact match first, regardless of score.  This
+   *  has no performance impact but could result in
+   *  low-quality suggestions. */
   public static final int EXACT_FIRST = 1;
 
-  /** Include this flag in the options parameter {@link
-   * #AnalyzingCompletionLookup(Analyzer,int)} to preserve
-   * token separators when matching. */
+  /** Include this flag in the options parameter to {@link
+   *  #AnalyzingCompletionLookup(Analyzer,int,int,int)} to preserve
+   *  token separators when matching. */
   public static final int PRESERVE_SEP = 2;
 
   /** Represents the separation between tokens, if
    *  PRESERVE_SEP was specified */
-  private static final byte SEP_BYTE = 0;
+  private static final int SEP_LABEL = 0xff;
+
+  /** Marks end of the analyzed input and start of dedup
+   *  byte. */
+  private static final int END_BYTE = 0x0;
+
+  /** Maximum number of dup surface forms (different surface
+   *  forms for the same analyzed form). */
+  private final int maxSurfaceFormsPerAnalyzedForm;
+
+  /** Maximum graph paths to index for a single analyzed
+   *  surface form.  This only matters if your analyzer
+   *  makes lots of alternate paths (e.g. contains
+   *  SynonymFilter). */
+  private final int maxGraphExpansions;
 
   /**
    * Creates a new suggester.
    * 
    * @param analyzer Analyzer that will be used for analyzing suggestions.
    * @param options see {@link #EXACT_FIRST}, {@link #PRESERVE_SEP}
+   * @param maxSurfaceFormsPerAnalyzedForm Maximum number of
+   *   surface forms to keep for a single analyzed form.
+   *   When there are too many surface forms we discard the
+   *   lowest weighted ones.
+   * @param maxGraphExpansions Maximum number of graph paths
+   *   to expand from the analyzed form.  Set this to -1 for
+   *   no limit.
    */
-  public AnalyzingCompletionLookup(Analyzer analyzer, int options) {
+  public AnalyzingCompletionLookup(Analyzer analyzer, int options, int maxSurfaceFormsPerAnalyzedForm, int maxGraphExpansions) {
     this.analyzer = analyzer;
     if ((options & ~(EXACT_FIRST | PRESERVE_SEP)) != 0) {
       throw new IllegalArgumentException("options should only contain EXACT_FIRST and PRESERVE_SEP; got " + options);
     }
     this.exactFirst = (options & EXACT_FIRST) != 0;
     this.preserveSep = (options & PRESERVE_SEP) != 0;
+
+    // NOTE: this is just an implementation limitation; if
+    // somehow this is a problem we could fix it by using
+    // more than one byte to disambiguate ... but 256 seems
+    // like it should be way more then enough.
+    if (maxSurfaceFormsPerAnalyzedForm <= 0 || maxSurfaceFormsPerAnalyzedForm > 256) {
+      throw new IllegalArgumentException("maxSurfaceFormsPerAnalyzedForm must be > 0 and < 256 (got: " + maxSurfaceFormsPerAnalyzedForm + ")");
+    }
+    this.maxSurfaceFormsPerAnalyzedForm = maxSurfaceFormsPerAnalyzedForm;
+
+    if (maxGraphExpansions < 1 && maxGraphExpansions != -1) {
+      throw new IllegalArgumentException("maxGraphExpansions must -1 (no limit) or > 0 (got: " + maxGraphExpansions + ")");
+    }
+    this.maxGraphExpansions = maxGraphExpansions;
   }
 
   // Replaces SEP with epsilon or remaps them if
@@ -165,8 +203,8 @@ public class AnalyzingCompletionLookup e
         assert t.getMin() == t.getMax();
         if (t.getMin() == TokenStreamToAutomaton.POS_SEP) {
           if (preserveSep) {
-            // Remap to SEP_BYTE:
-            t = new Transition(SEP_BYTE, t.getDest());
+            // Remap to SEP_LABEL:
+            t = new Transition(SEP_LABEL, t.getDest());
           } else {
             // NOTE: sort of weird because this will grow
             // the transition array we are iterating over,
@@ -175,6 +213,22 @@ public class AnalyzingCompletionLookup e
             state.addEpsilon(t.getDest());
             t = null;
           }
+        } else if (t.getMin() == TokenStreamToAutomaton.HOLE) {
+
+          // Just remove the hole: there will then be two
+          // SEP tokens next to each other, which will only
+          // match another hole at search time.  Note that
+          // it will also match an empty-string token ... if
+          // that's somehow a problem we can always map HOLE
+          // to a dedicated byte (and escape it in the
+          // input).
+
+          // NOTE: sort of weird because this will grow
+          // the transition array we are iterating over,
+          // but because we are going in reverse topo sort
+          // it will not add any SEP/HOLE transitions:
+          state.addEpsilon(t.getDest());
+          t = null;
         }
         if (t != null) {
           newTransitions.add(t);
@@ -184,6 +238,34 @@ public class AnalyzingCompletionLookup e
       state.setTransitions(newTransitions.toArray(new Transition[newTransitions.size()]));
     }
   }
+
+  /** Just escapes the bytes we steal (0xff, 0x0). */
+  private static final class  EscapingTokenStreamToAutomaton extends TokenStreamToAutomaton {
+
+    final BytesRef spare = new BytesRef();
+
+    @Override
+    protected BytesRef changeToken(BytesRef in) {
+      int upto = 0;
+      for(int i=0;i<in.length;i++) {
+        byte b = in.bytes[in.offset+i];
+        if (b == (byte) 0xff) {
+          if (spare.bytes.length == upto) {
+            spare.grow(upto+2);
+          }
+          spare.bytes[upto++] = (byte) 0xff;
+          spare.bytes[upto++] = b;
+        } else {
+          if (spare.bytes.length == upto) {
+            spare.grow(upto+1);
+          }
+          spare.bytes[upto++] = b;
+        }
+      }
+      spare.length = upto;
+      return spare;
+    }
+  }
   
   @Override
   public void build(TermFreqIterator iterator) throws IOException {
@@ -197,7 +279,9 @@ public class AnalyzingCompletionLookup e
     BytesRef scratch = new BytesRef();
 
     BytesRef separator = new BytesRef(new byte[] { (byte)TokenStreamToAutomaton.POS_SEP });
-    
+
+    TokenStreamToAutomaton ts2a = new EscapingTokenStreamToAutomaton();
+
     // analyzed sequence + 0(byte) + weight(int) + surface + analyzedLength(short) 
     boolean success = false;
     byte buffer[] = new byte[8];
@@ -212,7 +296,7 @@ public class AnalyzingCompletionLookup e
         // Create corresponding automaton: labels are bytes
         // from each analyzed token, with byte 0 used as
         // separator between tokens:
-        Automaton automaton = TokenStreamToAutomaton.toAutomaton(ts);
+        Automaton automaton = ts2a.toAutomaton(ts);
         ts.end();
         ts.close();
 
@@ -224,8 +308,7 @@ public class AnalyzingCompletionLookup e
         // more than one path, eg if the analyzer created a
         // graph using SynFilter or WDF):
 
-        // nocommit: we should probably not wire this param to -1 but have a reasonable limit?!
-        Set<IntsRef> paths = SpecialOperations.getFiniteStrings(automaton, -1);
+        Set<IntsRef> paths = SpecialOperations.getFiniteStrings(automaton, maxGraphExpansions);
         for (IntsRef path : paths) {
 
           Util.toBytesRef(path, scratch);
@@ -242,7 +325,15 @@ public class AnalyzingCompletionLookup e
           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);
           writer.write(buffer, 0, output.getPosition());
@@ -253,7 +344,7 @@ public class AnalyzingCompletionLookup e
       // Sort all input/output pairs (required by FST.Builder):
       new Sort().sort(tempInput, tempSorted);
       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);
 
@@ -263,6 +354,7 @@ public class AnalyzingCompletionLookup e
       BytesRef surface = new BytesRef();
       IntsRef scratchInts = new IntsRef();
       ByteArrayDataInput input = new ByteArrayDataInput();
+
       int dedup = 0;
       while (reader.read(scratch)) {
         input.reset(scratch.bytes, scratch.offset, scratch.length);
@@ -279,33 +371,32 @@ public class AnalyzingCompletionLookup e
         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)) {
-          // nocommit: "extend" duplicates with useless
-          // increasing bytes (it wont matter) ... or we
-          // could use multiple outputs for a single input?
-          // this would be more efficient?
-          if (dedup < 256) {
-            analyzed.grow(analyzed.length+1);
-            analyzed.bytes[analyzed.length] = (byte) dedup;
-            dedup++;
-            analyzed.length++;
-          } else {
-            // nocommit add param to limit "dups"???
-
-            // More than 256 dups: skip the rest:
+          dedup++;
+          if (dedup >= maxSurfaceFormsPerAnalyzedForm) {
+            // More than maxSurfaceFormsPerAnalyzedForm
+            // dups: skip the rest:
             continue;
           }
         } else {
           dedup = 0;
+          previous.copyBytes(analyzed);
         }
+        
+        analyzed.grow(analyzed.length+2);
+        // 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.length += 2;
 
         Util.toIntsRef(analyzed, scratchInts);
-        // nocommit (why?)
+        //System.out.println("ADD: " + analyzed);
         builder.add(scratchInts, outputs.newPair(cost, BytesRef.deepCopyOf(surface)));
-        previous.copyBytes(analyzed);
       }
       fst = builder.finish();
 
@@ -345,92 +436,146 @@ public class AnalyzingCompletionLookup e
   }
 
   @Override
-  public List<LookupResult> lookup(CharSequence key, boolean onlyMorePopular, int num) {
+  public List<LookupResult> lookup(final CharSequence key, boolean onlyMorePopular, int num) {
     assert num > 0;
     Arc<Pair<Long,BytesRef>> arc = new Arc<Pair<Long,BytesRef>>();
 
-    //System.out.println("lookup");
+    // System.out.println("lookup num=" + num);
 
-    // TODO: is there a Reader from a CharSequence?
-    // Turn tokenstream into automaton:
-    Automaton automaton;
     try {
+
+      // TODO: is there a Reader from a CharSequence?
+      // Turn tokenstream into automaton:
       TokenStream ts = analyzer.tokenStream("", new StringReader(key.toString()));
-      automaton = TokenStreamToAutomaton.toAutomaton(ts);
+      Automaton automaton = (new EscapingTokenStreamToAutomaton()).toAutomaton(ts);
       ts.end();
       ts.close();
-    } catch (IOException bogus) {
-      throw new RuntimeException(bogus);
-    }
 
-    replaceSep(automaton);
+      replaceSep(automaton);
 
-    // TODO: we can optimize this somewhat by determinizing
-    // while we convert
-    automaton = Automaton.minimize(automaton);
+      // TODO: we can optimize this somewhat by determinizing
+      // while we convert
+      automaton = Automaton.minimize(automaton);
 
-    List<LookupResult> results = new ArrayList<LookupResult>(num);
-    CharsRef spare = new CharsRef();
+      final CharsRef spare = new CharsRef();
 
-    //System.out.println("  now intersect exactFirst=" + exactFirst);
+      //System.out.println("  now intersect exactFirst=" + exactFirst);
     
-    // Intersect automaton w/ suggest wFST and get all
-    // prefix starting nodes & their outputs:
-    final List<FSTUtil.Path<Pair<Long,BytesRef>>> prefixPaths;
-    try {
+      // 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);
-    } catch (IOException bogus) {
-      throw new RuntimeException(bogus);
-    }
 
-    // nocommit maybe nuke exactFirst...? but... it's
-    // useful?
-    // nocommit: exactFirst is not well defined here ... the
-    // "exactness" refers to the analyzed form not the
-    // surface form
-    if (exactFirst) {
-      for (FSTUtil.Path<Pair<Long,BytesRef>> path : prefixPaths) {
-        if (path.fstNode.isFinal()) {
-          BytesRef prefix = BytesRef.deepCopyOf(path.output.output2);
-          prefix.append(path.fstNode.nextFinalOutput.output2);
-          spare.grow(prefix.length);
-          UnicodeUtil.UTF8toUTF16(prefix, spare);
-          results.add(new LookupResult(spare.toString(), decodeWeight(path.output.output1 + path.fstNode.nextFinalOutput.output1)));
-          if (--num == 0) {
-            // nocommit hmm should we order all "exact"
-            // matches by their .output1s, then return those
-            // top n...?
-            return results; // that was quick
+      //System.out.println("  prefixPaths: " +
+      //prefixPaths.size());
+
+      BytesReader bytesReader = fst.getBytesReader(0);
+
+      FST.Arc<Pair<Long,BytesRef>> scratchArc = new FST.Arc<Pair<Long,BytesRef>>();
+
+      List<LookupResult> results = new ArrayList<LookupResult>();
+
+      if (exactFirst) {
+
+        Util.TopNSearcher<Pair<Long,BytesRef>> searcher;
+        searcher = new Util.TopNSearcher<Pair<Long,BytesRef>>(fst, num, weightComparator);
+
+        int count = 0;
+        for (FSTUtil.Path<Pair<Long,BytesRef>> path : prefixPaths) {
+          if (fst.findTargetArc(END_BYTE, path.fstNode, scratchArc, bytesReader) != null) {
+            // This node has END_BYTE arc leaving, meaning it's an
+            // "exact" match:
+            count++;
+          }
+        }
+
+        searcher = new Util.TopNSearcher<Pair<Long,BytesRef>>(fst, count * maxSurfaceFormsPerAnalyzedForm, weightComparator);
+
+        // NOTE: we could almost get away with only usine
+        // the first start node.  The only catch is if
+        // maxSurfaceFormsPerAnalyzedForm had kicked in and
+        // pruned our exact match from one of these nodes
+        // ...:
+        for (FSTUtil.Path<Pair<Long,BytesRef>> path : prefixPaths) {
+          if (fst.findTargetArc(END_BYTE, path.fstNode, scratchArc, bytesReader) != null) {
+            // This node has END_BYTE arc leaving, meaning it's an
+            // "exact" match:
+            searcher.addStartPaths(scratchArc, fst.outputs.add(path.output, scratchArc.output), true, path.input);
           }
         }
+
+        MinResult<Pair<Long,BytesRef>> completions[] = searcher.search();
+
+        // NOTE: this is rather inefficient: we enumerate
+        // every matching "exactly the same analyzed form"
+        // path, and then do linear scan to see if one of
+        // these exactly matches the input.  It should be
+        // possible (though hairy) to do something similar
+        // to getByOutput, since the surface form is encoded
+        // into the FST output, so we more efficiently hone
+        // in on the exact surface-form match.  Still, I
+        // suspect very little time is spent in this linear
+        // seach: it's bounded by how many prefix start
+        // 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) {
+            results.add(new LookupResult(spare.toString(), decodeWeight(completion.output.output1)));
+            break;
+          }
+        }
+
+        if (results.size() == num) {
+          // That was quick:
+          return results;
+        }
       }
-    }
 
-    Util.TopNSearcher<Pair<Long,BytesRef>> searcher = new Util.TopNSearcher<Pair<Long,BytesRef>>(fst, num, weightComparator);
-    for (FSTUtil.Path<Pair<Long,BytesRef>> path : prefixPaths) {
-      try {
-        searcher.addStartPaths(path.fstNode, path.output, !exactFirst, path.input);
-      } catch (IOException bogus) {
-        throw new RuntimeException(bogus);
+      // nocommit hmm w/ a graph ... aren't we going to
+      // produce dup surface form suggestions ...?  do we
+      // need to dedup by surface form?
+
+      Util.TopNSearcher<Pair<Long,BytesRef>> searcher;
+      searcher = new Util.TopNSearcher<Pair<Long,BytesRef>>(fst,
+                                                            num - results.size(),
+                                                            weightComparator) {
+        @Override
+        protected boolean acceptResult(IntsRef input, Pair<Long,BytesRef> output) {
+          if (!exactFirst) {
+            return true;
+          } else {
+            // 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);
+            return CHARSEQUENCE_COMPARATOR.compare(spare, key) != 0;
+          }
+        }
+      };
+
+      for (FSTUtil.Path<Pair<Long,BytesRef>> path : prefixPaths) {
+        searcher.addStartPaths(path.fstNode, path.output, true, path.input);
       }
-    }
 
-    MinResult<Pair<Long,BytesRef>> completions[] = null;
-    try {
-      completions = searcher.search();
+      MinResult<Pair<Long,BytesRef>> completions[] = searcher.search();
+
+      for(MinResult<Pair<Long,BytesRef>> completion : completions) {
+        spare.grow(completion.output.output2.length);
+        UnicodeUtil.UTF8toUTF16(completion.output.output2, spare);
+        LookupResult result = new LookupResult(spare.toString(), decodeWeight(completion.output.output1));
+        //System.out.println("    result=" + result);
+        results.add(result);
+      }
+
+      return results;
     } catch (IOException bogus) {
       throw new RuntimeException(bogus);
     }
-
-    for (MinResult<Pair<Long,BytesRef>> completion : completions) {
-      spare.grow(completion.output.output2.length);
-      UnicodeUtil.UTF8toUTF16(completion.output.output2, spare);
-      results.add(new LookupResult(spare.toString(), decodeWeight(completion.output.output1)));
-    }
-
-    return results;
   }
-  
+
   /**
    * Returns the weight associated with an input string,
    * or null if it does not exist.
@@ -455,6 +600,6 @@ public class AnalyzingCompletionLookup e
   static final Comparator<Pair<Long,BytesRef>> weightComparator = new Comparator<Pair<Long,BytesRef>> () {
     public int compare(Pair<Long,BytesRef> left, Pair<Long,BytesRef> right) {
       return left.output1.compareTo(right.output1);
-    }  
+    }
   };
 }

Modified: lucene/dev/branches/lucene3842/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3842/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java?rev=1388936&r1=1388935&r2=1388936&view=diff
==============================================================================
--- lucene/dev/branches/lucene3842/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java (original)
+++ lucene/dev/branches/lucene3842/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java Sun Sep 23 01:14:06 2012
@@ -57,7 +57,6 @@ import org.apache.lucene.util.fst.Util.M
  * Input weights will be cast to a java integer, and any
  * negative, infinite, or NaN values will be rejected.
  * 
- * @see Util#shortestPaths(FST, FST.Arc, Object, Comparator, int)
  * @lucene.experimental
  */
 public class WFSTCompletionLookup extends Lookup {

Modified: lucene/dev/branches/lucene3842/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3842/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionTest.java?rev=1388936&r1=1388935&r2=1388936&view=diff
==============================================================================
--- lucene/dev/branches/lucene3842/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionTest.java (original)
+++ lucene/dev/branches/lucene3842/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionTest.java Sun Sep 23 01:14:06 2012
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.Reader;
 import java.io.StringReader;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
@@ -29,6 +30,8 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.CannedBinaryTokenStream.BinaryToken;
+import org.apache.lucene.analysis.CannedBinaryTokenStream;
 import org.apache.lucene.analysis.CannedTokenStream;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenFilter;
@@ -42,6 +45,7 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.search.suggest.Lookup.LookupResult;
 import org.apache.lucene.search.suggest.TermFreq;
 import org.apache.lucene.search.suggest.TermFreqArrayIterator;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
 
@@ -129,7 +133,7 @@ public class AnalyzingCompletionTest ext
 
     int options = 0;
 
-    AnalyzingCompletionLookup suggester = new AnalyzingCompletionLookup(new MockAnalyzer(random()), options);
+    AnalyzingCompletionLookup suggester = new AnalyzingCompletionLookup(new MockAnalyzer(random()), options, 256, -1);
     suggester.build(new TermFreqArrayIterator(keys));
     // nocommit if i change this to "ab " ... the test fails
     // now but really it should pass???  problem is
@@ -146,10 +150,6 @@ public class AnalyzingCompletionTest ext
   }
 
   public void testInputPathRequired() throws Exception {
-    TermFreq keys[] = new TermFreq[] {
-        new TermFreq("ab xc", 50),
-        new TermFreq("ba xd", 50),
-    };
 
     //  SynonymMap.Builder b = new SynonymMap.Builder(false);
     //  b.add(new CharsRef("ab"), new CharsRef("ba"), true);
@@ -167,42 +167,42 @@ public class AnalyzingCompletionTest ext
         // TokenStream stream = new SynonymFilter(tokenizer, map, true);
         // return new TokenStreamComponents(tokenizer, new RemoveDuplicatesTokenFilter(stream));
         return new TokenStreamComponents(tokenizer) {
-         int tokenStreamCounter = 0;
-         final TokenStream[] tokenStreams = new TokenStream[]{ new CannedTokenStream(
-             new Token[] {
-               token("ab",1,1),
-               token("ba",0,1),
-               token("xc",1,1)
-             }),
-
-         new CannedTokenStream(
-             new Token[] {
-               token("ba",1,1),          
-               token("xd",1,1)
-             }),
-
-         new CannedTokenStream(
-             new Token[] {
-               token("ab",1,1),
-               token("ba",0,1),
-               token("x",1,1)
-             })
-         };
-
-         @Override
-         public TokenStream getTokenStream() {
-           TokenStream result = tokenStreams[tokenStreamCounter];
-           tokenStreamCounter++;
-           return result;
-         }
+          int tokenStreamCounter = 0;
+          final TokenStream[] tokenStreams = new TokenStream[] {
+            new CannedTokenStream(new Token[] {
+                token("ab",1,1),
+                token("ba",0,1),
+                token("xc",1,1)
+              }),
+            new CannedTokenStream(new Token[] {
+                token("ba",1,1),          
+                token("xd",1,1)
+              }),
+            new CannedTokenStream(new Token[] {
+                token("ab",1,1),
+                token("ba",0,1),
+                token("x",1,1)
+              })
+          };
+
+          @Override
+          public TokenStream getTokenStream() {
+            TokenStream result = tokenStreams[tokenStreamCounter];
+            tokenStreamCounter++;
+            return result;
+          }
          
-         @Override
-         protected void setReader(final Reader reader) throws IOException {
-         }
+          @Override
+          protected void setReader(final Reader reader) throws IOException {
+          }
         };
       }
     };
 
+    TermFreq keys[] = new TermFreq[] {
+        new TermFreq("ab xc", 50),
+        new TermFreq("ba xd", 50),
+    };
     AnalyzingCompletionLookup suggester = new AnalyzingCompletionLookup(analyzer);
     suggester.build(new TermFreqArrayIterator(keys));
     List<LookupResult> results = suggester.lookup("ab x", false, 1);
@@ -216,7 +216,10 @@ public class AnalyzingCompletionTest ext
     return t;
   }
 
-  
+  private static BinaryToken token(BytesRef term) {
+    return new BinaryToken(term);
+  }
+
   private void printTokens(final Analyzer analyzer, String input) throws IOException {
     System.out.println("Tokens for " + input);
     TokenStream ts = analyzer.tokenStream("", new StringReader(input));
@@ -233,6 +236,114 @@ public class AnalyzingCompletionTest ext
     ts.close();
   }  
 
+  private final Analyzer getUnusualAnalyzer() {
+    return new Analyzer() {
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.SIMPLE, true);
+        
+        return new TokenStreamComponents(tokenizer) {
+
+          int count;
+
+          @Override
+          public TokenStream getTokenStream() {
+            // 4th time we are called, return tokens a b,
+            // else just a:
+            if (count++ != 3) {
+              return new CannedTokenStream(new Token[] {
+                  token("a", 1, 1),
+                });
+            } else {
+              // After that "a b":
+              return new CannedTokenStream(new Token[] {
+                  token("a", 1, 1),
+                  token("b", 1, 1),
+                });
+            }
+          }
+         
+          @Override
+          protected void setReader(final Reader reader) throws IOException {
+          }
+        };
+      }
+    };
+  }
+
+  public void testExactFirst() throws Exception {
+
+    AnalyzingCompletionLookup suggester = new AnalyzingCompletionLookup(getUnusualAnalyzer(), AnalyzingCompletionLookup.EXACT_FIRST | AnalyzingCompletionLookup.PRESERVE_SEP, 256, -1);
+    suggester.build(new TermFreqArrayIterator(new TermFreq[] {
+          new TermFreq("x y", 1),
+          new TermFreq("x y z", 3),
+          new TermFreq("x", 2),
+          new TermFreq("z z z", 20),
+        }));
+
+    //System.out.println("ALL: " + suggester.lookup("x y", false, 6));
+
+    for(int topN=1;topN<6;topN++) {
+      List<LookupResult> results = suggester.lookup("x y", false, topN);
+      //System.out.println("topN=" + topN + " " + results);
+
+      assertEquals(Math.min(topN, 4), results.size());
+
+      assertEquals("x y", results.get(0).key);
+      assertEquals(1, results.get(0).value);
+
+      if (topN > 1) {
+        assertEquals("z z z", results.get(1).key);
+        assertEquals(20, results.get(1).value);
+
+        if (topN > 2) {
+          assertEquals("x y z", results.get(2).key);
+          assertEquals(3, results.get(2).value);
+
+          if (topN > 3) {
+            assertEquals("x", results.get(3).key);
+            assertEquals(2, results.get(3).value);
+          }
+        }
+      }
+    }
+  }
+
+  public void testNonExactFirst() throws Exception {
+
+    AnalyzingCompletionLookup suggester = new AnalyzingCompletionLookup(getUnusualAnalyzer(), AnalyzingCompletionLookup.PRESERVE_SEP, 256, -1);
+
+    suggester.build(new TermFreqArrayIterator(new TermFreq[] {
+          new TermFreq("x y", 1),
+          new TermFreq("x y z", 3),
+          new TermFreq("x", 2),
+          new TermFreq("z z z", 20),
+        }));
+
+    for(int topN=1;topN<6;topN++) {
+      List<LookupResult> results = suggester.lookup("p", false, topN);
+
+      assertEquals(Math.min(topN, 4), results.size());
+
+      assertEquals("z z z", results.get(0).key);
+      assertEquals(20, results.get(0).value);
+
+      if (topN > 1) {
+        assertEquals("x y z", results.get(1).key);
+        assertEquals(3, results.get(1).value);
+
+        if (topN > 2) {
+          assertEquals("x", results.get(2).key);
+          assertEquals(2, results.get(2).value);
+          
+          if (topN > 3) {
+            assertEquals("x y", results.get(3).key);
+            assertEquals(1, results.get(3).value);
+          }
+        }
+      }
+    }
+  }
   
   public void testRandom() throws Exception {
     int numWords = atLeast(1000);
@@ -265,7 +376,7 @@ public class AnalyzingCompletionTest ext
     // nocommit also test NOT preserving seps/holes
     // nocommit why no failure if we DON'T preserve seps/holes...?
     AnalyzingCompletionLookup suggester = new AnalyzingCompletionLookup(new MockAnalyzer(random(), MockTokenizer.KEYWORD, false),
-                                                                        AnalyzingCompletionLookup.PRESERVE_SEP);
+                                                                        AnalyzingCompletionLookup.PRESERVE_SEP, 256, -1);
     suggester.build(new TermFreqArrayIterator(keys));
     
     for (String prefix : allPrefixes) {
@@ -307,4 +418,88 @@ public class AnalyzingCompletionTest ext
       }
     }
   }
+
+  // nocommit need random full binary test
+
+  public void testStolenBytes() throws Exception {
+    
+    final Analyzer analyzer = new Analyzer() {
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.SIMPLE, true);
+        
+        // TokenStream stream = new SynonymFilter(tokenizer, map, true);
+        // return new TokenStreamComponents(tokenizer, new RemoveDuplicatesTokenFilter(stream));
+        return new TokenStreamComponents(tokenizer) {
+          int tokenStreamCounter = 0;
+          final TokenStream[] tokenStreams = new TokenStream[] {
+            new CannedBinaryTokenStream(new BinaryToken[] {
+                token(new BytesRef(new byte[] {0x61, (byte) 0xff, 0x61})),
+              }),
+            new CannedTokenStream(new Token[] {
+                token("a",1,1),          
+                token("a",1,1)
+              }),
+            new CannedTokenStream(new Token[] {
+                token("a",1,1),
+                token("a",1,1)
+              }),
+            new CannedBinaryTokenStream(new BinaryToken[] {
+                token(new BytesRef(new byte[] {0x61, (byte) 0xff, 0x61})),
+              })
+          };
+
+          @Override
+          public TokenStream getTokenStream() {
+            TokenStream result = tokenStreams[tokenStreamCounter];
+            tokenStreamCounter++;
+            return result;
+          }
+         
+          @Override
+          protected void setReader(final Reader reader) throws IOException {
+          }
+        };
+      }
+    };
+
+    TermFreq keys[] = new TermFreq[] {
+      new TermFreq("a a", 50),
+      new TermFreq("a b", 50),
+    };
+
+    AnalyzingCompletionLookup suggester = new AnalyzingCompletionLookup(analyzer);
+    suggester.build(new TermFreqArrayIterator(keys));
+    List<LookupResult> results = suggester.lookup("a a", false, 5);
+    assertEquals(1, results.size());
+    assertEquals("a b", results.get(0).key);
+    assertEquals(50, results.get(0).value);
+
+    results = suggester.lookup("a a", false, 5);
+    assertEquals(1, results.size());
+    assertEquals("a a", results.get(0).key);
+    assertEquals(50, results.get(0).value);
+  }
+
+  // nocommit test that dups are preserved by weight:
+  public void testMaxSurfaceFormsPerAnalyzedForm() throws Exception {
+
+    AnalyzingCompletionLookup suggester = new AnalyzingCompletionLookup(new MockAnalyzer(random()), 0, 2, -1);
+
+    List<TermFreq> keys = Arrays.asList(new TermFreq[] {
+        new TermFreq("a", 40),
+        new TermFreq("a ", 50),
+        new TermFreq(" a", 60),
+      });
+
+    Collections.shuffle(keys, random());
+    suggester.build(new TermFreqArrayIterator(keys));
+
+    List<LookupResult> results = suggester.lookup("a", false, 5);
+    assertEquals(2, results.size());
+    assertEquals(" a", results.get(0).key);
+    assertEquals(60, results.get(0).value);
+    assertEquals("a ", results.get(1).key);
+    assertEquals(50, results.get(1).value);
+  }
 }

Added: lucene/dev/branches/lucene3842/lucene/test-framework/src/java/org/apache/lucene/analysis/CannedBinaryTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3842/lucene/test-framework/src/java/org/apache/lucene/analysis/CannedBinaryTokenStream.java?rev=1388936&view=auto
==============================================================================
--- lucene/dev/branches/lucene3842/lucene/test-framework/src/java/org/apache/lucene/analysis/CannedBinaryTokenStream.java (added)
+++ lucene/dev/branches/lucene3842/lucene/test-framework/src/java/org/apache/lucene/analysis/CannedBinaryTokenStream.java Sun Sep 23 01:14:06 2012
@@ -0,0 +1,130 @@
+package org.apache.lucene.analysis;
+
+/*
+ * 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.
+ */
+
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
+import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
+import org.apache.lucene.util.AttributeImpl;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * TokenStream from a canned list of binary (BytesRef-based)
+ * tokens.
+ */
+public final class CannedBinaryTokenStream extends TokenStream {
+
+  /** Represents a binary token. */
+  public final static class BinaryToken {
+    BytesRef term;
+    int posInc;
+    int posLen;
+    int startOffset;
+    int endOffset;
+
+    public BinaryToken(BytesRef term) {
+      this.term = term;
+      this.posInc = 1;
+      this.posLen = 1;
+    }
+
+    public BinaryToken(BytesRef term, int posInc, int posLen) {
+      this.term = term;
+      this.posInc = posInc;
+      this.posLen = posLen;
+    }
+  }
+
+  private final BinaryToken[] tokens;
+  private int upto = 0;
+  private final BinaryTermAttribute termAtt = addAttribute(BinaryTermAttribute.class);
+  private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
+  private final PositionLengthAttribute posLengthAtt = addAttribute(PositionLengthAttribute.class);
+  private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+
+  public interface BinaryTermAttribute extends TermToBytesRefAttribute {
+    public void setBytesRef(BytesRef bytes);
+  }
+
+  public final static class BinaryTermAttributeImpl extends AttributeImpl implements BinaryTermAttribute, TermToBytesRefAttribute {
+    private final BytesRef bytes = new BytesRef();
+
+    @Override
+    public int fillBytesRef() {
+      return bytes.hashCode();
+    }
+      
+    @Override
+    public BytesRef getBytesRef() {
+      return bytes;
+    }
+
+    public void setBytesRef(BytesRef bytes) {
+      this.bytes.copyBytes(bytes);
+    }
+    
+    @Override
+    public void clear() {
+    }
+
+    @Override
+    public boolean equals(Object other) {
+      return other == this;
+    }
+
+    @Override
+    public int hashCode() {
+      return System.identityHashCode(this);
+    }
+    
+    @Override
+    public void copyTo(AttributeImpl target) {
+      BinaryTermAttributeImpl other = (BinaryTermAttributeImpl) target;
+      other.bytes.copyBytes(bytes);
+    }
+    
+    @Override
+    public BinaryTermAttributeImpl clone() {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  public CannedBinaryTokenStream(BinaryToken... tokens) {
+    super();
+    this.tokens = tokens;
+  }
+  
+  @Override
+  public boolean incrementToken() {
+    if (upto < tokens.length) {
+      final BinaryToken token = tokens[upto++];     
+      // TODO: can we just capture/restoreState so
+      // we get all attrs...?
+      clearAttributes();      
+      termAtt.setBytesRef(token.term);
+      posIncrAtt.setPositionIncrement(token.posInc);
+      posLengthAtt.setPositionLength(token.posLen);
+      offsetAtt.setOffset(token.startOffset, token.endOffset);
+      return true;
+    } else {
+      return false;
+    }
+  }
+}