Index: lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java =================================================================== --- lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java (revision 0) +++ lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java (working copy) @@ -0,0 +1,105 @@ +package org.apache.lucene.search.suggest.analyzing; + +/* + * 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.File; +import java.util.List; + +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.analysis.MockAnalyzer; +import org.apache.lucene.analysis.MockTokenizer; +import org.apache.lucene.search.suggest.Lookup.LookupResult; +import org.apache.lucene.search.suggest.TermFreqPayload; +import org.apache.lucene.search.suggest.TermFreqPayloadArrayIterator; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util._TestUtil; + +// Test requires postings offsets: +@SuppressCodecs({"Lucene3x","MockFixedIntBlock","MockVariableIntBlock","MockSep","MockRandom"}) +public class AnalyzingInfixSuggesterTest extends LuceneTestCase { + + public void testBasic() throws Exception { + TermFreqPayload keys[] = new TermFreqPayload[] { + new TermFreqPayload("lend me your ear", 8, new BytesRef("foobar")), + new TermFreqPayload("a penny saved is a penny earned", 10, new BytesRef("foobaz")), + }; + + File tempDir = _TestUtil.getTempDir("AnalyzingInfixSuggesterTest"); + + Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false); + AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, 3); + suggester.build(new TermFreqPayloadArrayIterator(keys)); + + for(int i=0;i<2;i++) { + boolean doHighlight = i == 0; + + List results = suggester.lookup(_TestUtil.stringToCharSequence("ear", random()), 10, true, doHighlight); + assertEquals(2, results.size()); + if (doHighlight) { + assertEquals("a penny saved is a penny earned", results.get(0).key); + } else { + assertEquals("a penny saved is a penny earned", results.get(0).key); + } + assertEquals(10, results.get(0).value); + if (doHighlight) { + assertEquals("lend me your ear", results.get(1).key); + } else { + assertEquals("lend me your ear", results.get(1).key); + } + assertEquals(new BytesRef("foobaz"), results.get(0).payload); + assertEquals(8, results.get(1).value); + assertEquals(new BytesRef("foobar"), results.get(1).payload); + + results = suggester.lookup(_TestUtil.stringToCharSequence("ear ", random()), 10, true, doHighlight); + assertEquals(1, results.size()); + if (doHighlight) { + assertEquals("lend me your ear", results.get(0).key); + } else { + assertEquals("lend me your ear", results.get(0).key); + } + assertEquals(8, results.get(0).value); + assertEquals(new BytesRef("foobar"), results.get(0).payload); + + results = suggester.lookup(_TestUtil.stringToCharSequence("pen", random()), 10, true, doHighlight); + assertEquals(1, results.size()); + if (doHighlight) { + assertEquals("a penny saved is a penny earned", results.get(0).key); + } else { + assertEquals("a penny saved is a penny earned", results.get(0).key); + } + assertEquals(10, results.get(0).value); + assertEquals(new BytesRef("foobaz"), results.get(0).payload); + + results = suggester.lookup(_TestUtil.stringToCharSequence("p", random()), 10, true, doHighlight); + assertEquals(1, results.size()); + if (doHighlight) { + assertEquals("a penny saved is a penny earned", results.get(0).key); + } else { + assertEquals("a penny saved is a penny earned", results.get(0).key); + } + assertEquals(10, results.get(0).value); + assertEquals(new BytesRef("foobaz"), results.get(0).payload); + } + } + + // nocommit test showing analysis effects + + // nocommit test save/load +} Property changes on: lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java ___________________________________________________________________ Added: svn:eol-style ## -0,0 +1 ## +native \ No newline at end of property Index: lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java =================================================================== --- lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java (revision 0) +++ lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java (working copy) @@ -0,0 +1,548 @@ +package org.apache.lucene.search.suggest.analyzing; + +/* + * 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.Closeable; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.StringReader; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.analysis.AnalyzerWrapper; +import org.apache.lucene.analysis.TokenStream; +import org.apache.lucene.analysis.ngram.EdgeNGramTokenFilter; +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.codecs.lucene42.Lucene42Codec; +import org.apache.lucene.document.BinaryDocValuesField; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.FieldType; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.TextField; +import org.apache.lucene.index.AtomicReader; +import org.apache.lucene.index.AtomicReaderContext; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.FieldInfo.IndexOptions; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.MultiDocValues; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.SlowCompositeReaderWrapper; +import org.apache.lucene.index.Term; +import org.apache.lucene.index.sorter.Sorter; +import org.apache.lucene.index.sorter.SortingAtomicReader; +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.Collector; +import org.apache.lucene.search.FieldDoc; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.MultiTermQuery; +import org.apache.lucene.search.PrefixQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreDoc; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SortField; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.TopDocs; +import org.apache.lucene.search.TopFieldDocs; +import org.apache.lucene.search.spell.TermFreqIterator; +import org.apache.lucene.search.spell.TermFreqPayloadIterator; +import org.apache.lucene.search.suggest.Lookup; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FSDirectory; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.Version; + +// TODO: a PostingsFormat that stores super-high-freq terms +// as a bitset...? + +// nocommit: custom codec? FST terms dict? +// nocommit: allow for coord scoring to have impact? if we allow OR + +/** Analyzes the input text and then suggests matches based + * on matches to any matching tokens query text against the + * suggestions. This also highlights the tokens that + * match. + * + *

This just uses an ordinary Lucene index. It + * supports payloads, and records these as a + * {@link BinaryDocValues} field. Matches are sorted only + * by the suggest weight ... would be nice to supported + * blended score + weight sort in the future. This means + * this suggester is really only appropriate in cases + * where there is a strong apriori ranking of all the + * suggestions. */ + +public class AnalyzingInfixSuggester extends Lookup implements Closeable { + + protected final static String TEXT_FIELD_NAME = "text"; + + private final Analyzer queryAnalyzer; + private final Analyzer indexAnalyzer; + private final Directory dir; + private final Version matchVersion; + private final File indexPath; + private final int minPrefixChars; + + protected IndexSearcher searcher; + + /** null if payloads were not indexed: */ + private BinaryDocValues payloadsDV; + private BinaryDocValues textDV; + private NumericDocValues weightsDV; + + /** Default minimum number of leading characters before + * PrefixQuery is used (4). */ + public static final int DEFAULT_MIN_PREFIX_CHARS = 4; + + public AnalyzingInfixSuggester(Version matchVersion, File indexPath, Analyzer analyzer) throws IOException { + this(matchVersion, indexPath, analyzer, analyzer, DEFAULT_MIN_PREFIX_CHARS); + } + + // TODO: this could support NRT additions to the suggester + // ... be sure to remove that forceMerge(1) if we do!!! + /** + @param minPrefixChars Minimum number of leading characters + before PrefixQuery is used (default 4). + Prefixes shorter than this are indexed as character + ngrams (increasing index size but making lookups + faster). + */ + public AnalyzingInfixSuggester(Version matchVersion, File indexPath, Analyzer indexAnalyzer, Analyzer queryAnalyzer, int minPrefixChars) throws IOException { + + this.queryAnalyzer = queryAnalyzer; + this.indexAnalyzer = indexAnalyzer; + this.matchVersion = matchVersion; + this.indexPath = indexPath; + this.minPrefixChars = minPrefixChars; + dir = FSDirectory.open(indexPath); + + if (DirectoryReader.indexExists(dir)) { + // Already built; open it: + searcher = new IndexSearcher(DirectoryReader.open(dir)); + // This will just be null if app didn't pass payloads to build(): + // nocommit just stored fields? they compress... + payloadsDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), "payloads"); + weightsDV = MultiDocValues.getNumericValues(searcher.getIndexReader(), "weight"); + textDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), TEXT_FIELD_NAME); + assert textDV != null; + } + } + + /** Override this to customize index settings, e.g. which + * codec to use. */ + protected IndexWriterConfig getIndexWriterConfig(Version matchVersion, Analyzer indexAnalyzer) { + IndexWriterConfig iwc = new IndexWriterConfig(matchVersion, indexAnalyzer); + iwc.setCodec(new Lucene42Codec()); + iwc.setOpenMode(IndexWriterConfig.OpenMode.CREATE); + return iwc; + } + + @Override + public void build(TermFreqIterator iter) throws IOException { + + TermFreqPayloadIterator payloads; + if (iter instanceof TermFreqPayloadIterator) { + payloads = (TermFreqPayloadIterator) iter; + } else { + payloads = null; + } + Directory dirTmp = FSDirectory.open(new File(indexPath.toString() + ".tmp")); + + Analyzer gramAnalyzer = new AnalyzerWrapper() { + @Override + protected Analyzer getWrappedAnalyzer(String fieldName) { + return indexAnalyzer; + } + + @Override + protected TokenStreamComponents wrapComponents(String fieldName, TokenStreamComponents components) { + if (fieldName.equals("textgrams")) { + return new TokenStreamComponents(components.getTokenizer(), + new EdgeNGramTokenFilter(matchVersion, + components.getTokenStream(), + 1, minPrefixChars)); + } else { + return components; + } + } + }; + + IndexWriter w = new IndexWriter(dirTmp, + getIndexWriterConfig(matchVersion, gramAnalyzer)); + IndexWriter w2 = null; + AtomicReader r = null; + boolean success = false; + try { + + BytesRef text; + Document doc = new Document(); + FieldType ft = new FieldType(TextField.TYPE_NOT_STORED); + ft.setIndexOptions(IndexOptions.DOCS_ONLY); + ft.setOmitNorms(true); + Field textField = new Field(TEXT_FIELD_NAME, "", ft); + doc.add(textField); + + Field textGramField = new Field("textgrams", "", ft); + doc.add(textGramField); + + Field textDVField = new BinaryDocValuesField(TEXT_FIELD_NAME, new BytesRef()); + doc.add(textDVField); + + // TODO: use threads...? + Field weightField = new NumericDocValuesField("weight", 0); + doc.add(weightField); + + Field payloadField; + if (payloads != null) { + payloadField = new BinaryDocValuesField("payloads", new BytesRef()); + doc.add(payloadField); + } else { + payloadField = null; + } + + long t0 = System.nanoTime(); + while ((text = iter.next()) != null) { + String textString = text.utf8ToString(); + textField.setStringValue(textString); + textGramField.setStringValue(textString); + textDVField.setBytesValue(text); + weightField.setLongValue(iter.weight()); + if (payloads != null) { + payloadField.setBytesValue(payloads.payload()); + } + w.addDocument(doc); + } + //System.out.println("initial indexing time: " + ((System.nanoTime()-t0)/1000000) + " msec"); + + r = new SlowCompositeReaderWrapper(DirectoryReader.open(w, false)); + long t1 = System.nanoTime(); + w.rollback(); + + final int maxDoc = r.maxDoc(); + + final NumericDocValues weights = r.getNumericDocValues("weight"); + + final Sorter.DocComparator comparator = new Sorter.DocComparator() { + @Override + public int compare(int docID1, int docID2) { + final long v1 = weights.get(docID1); + final long v2 = weights.get(docID2); + // Reverse sort (highest weight first); + // java7 only: + //return Long.compare(v2, v1); + if (v1 > v2) { + return -1; + } else if (v1 < v2) { + return 1; + } else { + return 0; + } + } + }; + + r = SortingAtomicReader.wrap(r, new Sorter() { + @Override + public Sorter.DocMap sort(AtomicReader reader) throws IOException { + long t0 = System.nanoTime(); + try { + return Sorter.sort(maxDoc, comparator); + } finally { + //System.out.println("Sort took " + ((System.nanoTime() - t0)/1000000.) + " msec"); + } + } + + @Override + public String getID() { + return "Weight"; + } + }); + + w2 = new IndexWriter(dir, + getIndexWriterConfig(matchVersion, indexAnalyzer)); + w2.addIndexes(new IndexReader[] {r}); + r.close(); + + //System.out.println("sort time: " + ((System.nanoTime()-t1)/1000000) + " msec"); + + searcher = new IndexSearcher(DirectoryReader.open(w2, false)); + w2.close(); + + payloadsDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), "payloads"); + weightsDV = MultiDocValues.getNumericValues(searcher.getIndexReader(), "weight"); + textDV = MultiDocValues.getBinaryValues(searcher.getIndexReader(), TEXT_FIELD_NAME); + assert textDV != null; + success = true; + } finally { + if (success) { + IOUtils.close(w, w2, r); + } else { + IOUtils.closeWhileHandlingException(w, w2, r); + } + } + } + + @Override + public List lookup(CharSequence key, boolean onlyMorePopular, int num) { + return lookup(key, num, true, true); + } + + /** This is called if the last token isn't ended + * (e.g. user did not type a space after it). Return an + * appropriate Query clause to add to the BooleanQuery. */ + protected Query getLastTokenQuery(String token) throws IOException { + if (token.length() < minPrefixChars) { + // The leading ngram was directly indexed: + return new TermQuery(new Term("textgrams", token)); + } + + return new PrefixQuery(new Term(TEXT_FIELD_NAME, token)); + } + + public List lookup(CharSequence key, int num, boolean allTermsRequired, boolean doHighlight) { + + final BooleanClause.Occur occur; + if (allTermsRequired) { + occur = BooleanClause.Occur.MUST; + } else { + occur = BooleanClause.Occur.SHOULD; + } + + try { + //long t0 = System.currentTimeMillis(); + TokenStream ts = queryAnalyzer.tokenStream("", new StringReader(key.toString())); + ts.reset(); + final CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class); + final OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class); + String lastToken = null; + BooleanQuery query = new BooleanQuery(); + int maxEndOffset = -1; + final Set matchedTokens = new HashSet(); + while (ts.incrementToken()) { + if (lastToken != null) { + matchedTokens.add(lastToken); + query.add(new TermQuery(new Term(TEXT_FIELD_NAME, lastToken)), occur); + } + lastToken = termAtt.toString(); + if (lastToken != null) { + maxEndOffset = Math.max(maxEndOffset, offsetAtt.endOffset()); + } + } + ts.end(); + + String prefixToken = null; + + if (lastToken != null) { + Query lastQuery; + if (maxEndOffset == offsetAtt.endOffset()) { + lastQuery = getLastTokenQuery(lastToken); + prefixToken = lastToken; + } else { + matchedTokens.add(lastToken); + lastQuery = new TermQuery(new Term(TEXT_FIELD_NAME, lastToken)); + } + if (lastQuery != null) { + query.add(lastQuery, occur); + } + } + ts.close(); + + // TODO: we could allow blended sort here, combining + // weight w/ score. Now we ignore score and sort only + // by weight: + + //System.out.println("INFIX query=" + query); + + // nocommit not general: + List clauses = query.clauses(); + if (clauses.size() >= 2 && occur == BooleanClause.Occur.MUST) { + BooleanQuery sub = new BooleanQuery(); + BooleanClause other = clauses.get(clauses.size()-2); + sub.add(new BooleanClause(clauses.get(clauses.size()-2).getQuery(), BooleanClause.Occur.SHOULD)); + sub.add(new BooleanClause(clauses.get(clauses.size()-1).getQuery(), BooleanClause.Occur.SHOULD)); + clauses.subList(clauses.size()-2, clauses.size()).clear(); + clauses.add(new BooleanClause(sub, BooleanClause.Occur.MUST)); + } + + FirstNDocsCollector c = new FirstNDocsCollector(num); + try { + searcher.search(query, c); + } catch (FirstNDocsCollector.DoneException done) { + } + TopDocs hits = c.getHits(); + + // Slower way if postings are not pre-sorted by weight: + // hits = searcher.search(query, null, num, new Sort(new SortField("weight", SortField.Type.LONG, true))); + + List results = new ArrayList(); + BytesRef scratch = new BytesRef(); + for (int i=0;i matchedTokens, String prefixToken) throws IOException { + TokenStream ts = queryAnalyzer.tokenStream("text", new StringReader(text)); + CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class); + OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class); + PositionIncrementAttribute posIncAtt = ts.addAttribute(PositionIncrementAttribute.class); + ts.reset(); + StringBuilder sb = new StringBuilder(); + int upto = 0; + while (ts.incrementToken()) { + String token = termAtt.toString(); + int startOffset = offsetAtt.startOffset(); + int endOffset = offsetAtt.endOffset(); + if (upto < startOffset) { + sb.append(text.substring(upto, startOffset)); + upto = startOffset; + } else if (upto > startOffset) { + continue; + } + + if (matchedTokens.contains(token)) { + // Token matches. // TODO: we could try to be smart about prefixes + // ... but it obviously depends on the analyzer + addWholeMatch(sb, text.substring(startOffset, endOffset), token); + upto = endOffset; + } else if (prefixToken != null && token.startsWith(prefixToken)) { + addPrefixMatch(sb, text.substring(startOffset, endOffset), token, prefixToken); + upto = endOffset; + } + } + ts.end(); + int endOffset = offsetAtt.endOffset(); + if (upto < endOffset) { + sb.append(text.substring(upto)); + } + ts.close(); + + return sb.toString(); + } + + protected void addWholeMatch(StringBuilder sb, String surface, String analyzed) { + sb.append(""); + sb.append(surface); + sb.append(""); + } + + protected void addPrefixMatch(StringBuilder sb, String surface, String analyzed, String prefixToken) { + // nocommit do the hackish thing here? it will "often" work? + // TODO: apps that know their analysis is "light" can + // try to find the prefix in the surface token and + // highlight only that: + sb.append(""); + sb.append(surface); + sb.append(""); + } + + private static class FirstNDocsCollector extends Collector { + private int docBase; + private final int[] hits; + private int hitCount; + + private static class DoneException extends RuntimeException { + } + + public TopDocs getHits() { + ScoreDoc[] scoreDocs = new ScoreDoc[hitCount]; + for(int i=0;i + + + + + + + + +