Index: lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java =================================================================== --- lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (revision 1385125) +++ lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (working copy) @@ -2293,4 +2293,33 @@ } } } + + public void testMultipleOutputs() throws Exception { + PositiveIntOutputs _outputs = PositiveIntOutputs.getSingleton(); + OneOrMoreOutputs outputs = new OneOrMoreOutputs(_outputs); + final Builder builder = new Builder(FST.INPUT_TYPE.BYTE1, outputs); + + final IntsRef scratch = new IntsRef(); + // Add the same input more than once and the outputs + // are merged: + builder.add(Util.toIntsRef(new BytesRef("a"), scratch), 1L); + builder.add(Util.toIntsRef(new BytesRef("a"), scratch), 3L); + builder.add(Util.toIntsRef(new BytesRef("a"), scratch), 0L); + builder.add(Util.toIntsRef(new BytesRef("b"), scratch), 17L); + final FST fst = builder.finish(); + + Object output = Util.get(fst, new BytesRef("a")); + assertNotNull(output); + List outputList = outputs.asList(output); + assertEquals(3, outputList.size()); + assertEquals(1L, outputList.get(0).longValue()); + assertEquals(3L, outputList.get(1).longValue()); + assertEquals(0L, outputList.get(2).longValue()); + + output = Util.get(fst, new BytesRef("b")); + assertNotNull(output); + outputList = outputs.asList(output); + assertEquals(1, outputList.size()); + assertEquals(17L, outputList.get(0).longValue()); + } } Index: lucene/core/src/java/org/apache/lucene/util/fst/OneOrMoreOutputs.java =================================================================== --- lucene/core/src/java/org/apache/lucene/util/fst/OneOrMoreOutputs.java (revision 0) +++ lucene/core/src/java/org/apache/lucene/util/fst/OneOrMoreOutputs.java (working copy) @@ -0,0 +1,162 @@ +package org.apache.lucene.util.fst; + +/* + * 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 java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.DataOutput; + +/** + * Wraps another Outputs implementation and encodes one or + * more of its outputs. You can use this when a single + * input may need to map to more than one output. + * + * @lucene.experimental + */ + +// nocommit separate unit test for this + +// nocommit should we nuke UpToTwoPositiveInts...? this +// subsumes that? + +@SuppressWarnings("unchecked") +public final class OneOrMoreOutputs extends Outputs { + + private final Outputs outputs; + + public OneOrMoreOutputs(Outputs outputs) { + this.outputs = outputs; + } + + @Override + public Object common(Object output1, Object output2) { + // These will never be a list: + return outputs.common((T) output1, (T) output2); + } + + @Override + public Object subtract(Object object, Object inc) { + // These will never be a list: + return outputs.subtract((T) object, (T) inc); + } + + @Override + public Object add(Object prefix, Object output) { + assert !(prefix instanceof List); + if (!(output instanceof List)) { + return outputs.add((T) prefix, (T) output); + } else { + List outputList = (List) output; + List addedList = new ArrayList(outputList.size()); + for(T _output : outputList) { + addedList.add(outputs.add((T) prefix, _output)); + } + return addedList; + } + } + + @Override + public void write(Object output, DataOutput out) throws IOException { + if (!(output instanceof List)) { + out.writeVInt(1); + outputs.write((T) output, out); + } else { + List outputList = (List) output; + out.writeVInt(outputList.size()); + for(T eachOutput : outputList) { + outputs.write(eachOutput, out); + } + } + } + + @Override + public Object read(DataInput in) throws IOException { + int count = in.readVInt(); + if (count == 1) { + return outputs.read(in); + } else { + List outputList = new ArrayList(count); + for(int i=0;i outputList = (List) output; + + StringBuilder b = new StringBuilder(); + b.append('['); + + for(int i=0;i 0) { + b.append(", "); + } + b.append(outputs.outputToString(outputList.get(i))); + } + b.append(']'); + return b.toString(); + } + } + + @Override + public Object merge(Object first, Object second) { + List outputList = new ArrayList(); + if (!(first instanceof List)) { + outputList.add((T) first); + } else { + outputList.addAll((List) first); + } + if (!(second instanceof List)) { + outputList.add((T) second); + } else { + outputList.addAll((List) second); + } + //System.out.println("MERGE: now " + outputList.size() + " first=" + outputToString(first) + " second=" + outputToString(second)); + //System.out.println(" return " + outputToString(outputList)); + return outputList; + } + + @Override + public String toString() { + return "OneOrMoreOutputs(" + outputs + ")"; + } + + public List asList(Object output) { + if (!(output instanceof List)) { + List result = new ArrayList(1); + result.add((T) output); + return result; + } else { + return (List) output; + } + } +} Property changes on: lucene/core/src/java/org/apache/lucene/util/fst/OneOrMoreOutputs.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/core/src/java/org/apache/lucene/util/fst/Builder.java =================================================================== --- lucene/core/src/java/org/apache/lucene/util/fst/Builder.java (revision 1385124) +++ lucene/core/src/java/org/apache/lucene/util/fst/Builder.java (working copy) @@ -399,8 +399,10 @@ } final UnCompiledNode lastNode = frontier[input.length]; - lastNode.isFinal = true; - lastNode.output = NO_OUTPUT; + if (lastInput.length != input.length || prefixLenPlus1 != input.length + 1) { + lastNode.isFinal = true; + lastNode.output = NO_OUTPUT; + } // push conflicting outputs forward, only as far as // needed Index: lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionLookup.java =================================================================== --- lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionLookup.java (revision 1385159) +++ lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionLookup.java (working copy) @@ -55,6 +55,7 @@ import org.apache.lucene.util.fst.FST.Arc; import org.apache.lucene.util.fst.FST.BytesReader; import org.apache.lucene.util.fst.FST; +import org.apache.lucene.util.fst.OneOrMoreOutputs; import org.apache.lucene.util.fst.PairOutputs.Pair; import org.apache.lucene.util.fst.PairOutputs; import org.apache.lucene.util.fst.PositiveIntOutputs; @@ -95,7 +96,7 @@ * weights are encoded as costs: (Integer.MAX_VALUE-weight) * surface is the original, unanalyzed form. */ - private FST> fst = null; + private FST fst = null; /** * Analyzer that will be used for analyzing suggestions @@ -119,22 +120,28 @@ this(analyzer, EXACT_FIRST | PRESERVE_SEP); } - /** 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)} to always + * return exact matches first regardless of score. This + * has no performance impact but could result in + * low-quality suggestions. Note that "exact" means the + * key provided to {@link #lookup} analyzed to same + * output as the inputs provided to {@link #build}. */ 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)} 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 final PairOutputs _outputs; + private final OneOrMoreOutputs> outputs; + + /** * Creates a new suggester. * @@ -148,6 +155,8 @@ } this.exactFirst = (options & EXACT_FIRST) != 0; this.preserveSep = (options & PRESERVE_SEP) != 0; + _outputs = new PairOutputs(PositiveIntOutputs.getSingleton(true), ByteSequenceOutputs.getSingleton()); + outputs = new OneOrMoreOutputs>(_outputs); } // Replaces SEP with epsilon or remaps them if @@ -254,16 +263,13 @@ new Sort().sort(tempInput, tempSorted); reader = new Sort.ByteSequencesReader(tempSorted); - PairOutputs outputs = new PairOutputs(PositiveIntOutputs.getSingleton(true), ByteSequenceOutputs.getSingleton()); - Builder> builder = new Builder>(FST.INPUT_TYPE.BYTE1, outputs); + Builder builder = new Builder(FST.INPUT_TYPE.BYTE1, outputs); // Build FST: - BytesRef previous = null; BytesRef analyzed = new BytesRef(); 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); input.setPosition(input.length()-2); @@ -280,32 +286,8 @@ surface.offset = input.getPosition(); surface.length = input.length() - input.getPosition() - 2; - if (previous == null) { - previous = new BytesRef(); - } 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: - continue; - } - } else { - dedup = 0; - } - Util.toIntsRef(analyzed, scratchInts); - // nocommit (why?) - builder.add(scratchInts, outputs.newPair(cost, BytesRef.deepCopyOf(surface))); - previous.copyBytes(analyzed); + builder.add(scratchInts, _outputs.newPair(cost, BytesRef.deepCopyOf(surface))); } fst = builder.finish(); @@ -337,7 +319,7 @@ @Override public boolean load(InputStream input) throws IOException { try { - this.fst = new FST>(new InputStreamDataInput(input), new PairOutputs(PositiveIntOutputs.getSingleton(true), ByteSequenceOutputs.getSingleton())); + this.fst = new FST(new InputStreamDataInput(input), outputs); } finally { IOUtils.close(input); } @@ -349,7 +331,7 @@ assert num > 0; Arc> arc = new Arc>(); - //System.out.println("lookup"); + // System.out.println("lookup num=" + num); // TODO: is there a Reader from a CharSequence? // Turn tokenstream into automaton: @@ -376,38 +358,42 @@ // Intersect automaton w/ suggest wFST and get all // prefix starting nodes & their outputs: - final List>> prefixPaths; + final List> prefixPaths; try { 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 + //System.out.println(" prefixPaths: " + prefixPaths.size()); + + List exactResults; + if (exactFirst) { - for (FSTUtil.Path> path : prefixPaths) { + for (FSTUtil.Path 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 + for(Pair pair : outputs.asList(outputs.add(path.output, path.fstNode.nextFinalOutput))) { + UnicodeUtil.UTF8toUTF16(pair.output2, spare); + results.add(new LookupResult(spare.toString(), decodeWeight(pair.output1))); } } } + exactResults = trim(results, num); + if (exactResults.size() == num) { + // That was quick: we found enough "exact" matches + return exactResults; + } + } else { + exactResults = null; } - Util.TopNSearcher> searcher = new Util.TopNSearcher>(fst, num, weightComparator); - for (FSTUtil.Path> path : prefixPaths) { + // nocommit must at this point first sort exactFirst + // matches ... + + //System.out.println(" after prefix: " + results.size()); + + Util.TopNSearcher searcher = new Util.TopNSearcher(fst, num - results.size(), weightComparator); + for (FSTUtil.Path path : prefixPaths) { try { searcher.addStartPaths(path.fstNode, path.output, !exactFirst, path.input); } catch (IOException bogus) { @@ -415,19 +401,57 @@ } } - MinResult> completions[] = null; + MinResult completions[] = null; try { completions = searcher.search(); } catch (IOException bogus) { throw new RuntimeException(bogus); } - for (MinResult> completion : completions) { - spare.grow(completion.output.output2.length); - UnicodeUtil.UTF8toUTF16(completion.output.output2, spare); - results.add(new LookupResult(spare.toString(), decodeWeight(completion.output.output1))); + List normalResults = new ArrayList(); + + for(MinResult completion : completions) { + for(Pair pair : outputs.asList(completion.output)) { + spare.grow(pair.output2.length); + UnicodeUtil.UTF8toUTF16(pair.output2, spare); + LookupResult result = new LookupResult(spare.toString(), decodeWeight(pair.output1)); + //System.out.println(" result=" + result); + normalResults.add(result); + } } + if (exactResults == null) { + return trim(normalResults, num); + } else { + trim(normalResults, num - exactResults.size()); + exactResults.addAll(normalResults); + return exactResults; + } + } + + private List trim(List results, int num) { + // Sort: + Collections.sort(results, new Comparator() { + @Override + public int compare(LookupResult a, LookupResult b) { + if (a.value < b.value) { + return 1; + } else if (a.value > b.value) { + return -1; + } else { + int cmp = CHARSEQUENCE_COMPARATOR.compare(a.key, b.key); + // There should be no ties: + assert cmp != 0; + return cmp; + } + } + }); + + if (results.size() > num) { + // Trim: + results.subList(num, results.size()).clear(); + } + return results; } @@ -452,9 +476,36 @@ return Integer.MAX_VALUE - (int)value; } - static final Comparator> weightComparator = new Comparator> () { - public int compare(Pair left, Pair right) { - return left.output1.compareTo(right.output1); + final Comparator weightComparator = new Comparator () { + @Override + @SuppressWarnings("unchecked") + public int compare(Object _left, Object _right) { + long leastLeftCost; + if (!(_left instanceof List)) { + leastLeftCost = ((Pair) _left).output1; + } else { + leastLeftCost = Long.MAX_VALUE; + for(Pair output : outputs.asList(_left)) { + leastLeftCost = Math.min(leastLeftCost, output.output1); + } + } + long leastRightCost; + if (!(_right instanceof List)) { + leastRightCost = ((Pair) _right).output1; + } else { + leastRightCost = Long.MAX_VALUE; + for(Pair output : outputs.asList(_right)) { + leastRightCost = Math.min(leastRightCost, output.output1); + } + } + + if (leastLeftCost < leastRightCost) { + return -1; + } else if (leastLeftCost > leastRightCost) { + return 1; + } else { + return 0; + } } }; } Index: lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionTest.java =================================================================== --- lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionTest.java (revision 1385159) +++ lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingCompletionTest.java (working copy) @@ -167,38 +167,34 @@ // 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) - }), + 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) + }) + }; - 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 + 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 { + } }; } }; @@ -216,7 +212,6 @@ return t; } - 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 +228,111 @@ 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); + 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 results = suggester.lookup("p", false, topN); + + assertEquals(Math.min(topN, 4), results.size()); + + assertEquals("x y z", results.get(0).key); + assertEquals(3, results.get(0).value); + + if (topN > 1) { + assertEquals("x", results.get(1).key); + assertEquals(2, results.get(1).value); + + if (topN > 2) { + assertEquals("x y", results.get(2).key); + assertEquals(1, results.get(2).value); + + if (topN > 3) { + assertEquals("z z z", results.get(3).key); + assertEquals(20, results.get(3).value); + } + } + } + } + } + + public void testNonExactFirst() throws Exception { + + AnalyzingCompletionLookup suggester = new AnalyzingCompletionLookup(getUnusualAnalyzer(), AnalyzingCompletionLookup.PRESERVE_SEP); + + 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 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);