mirror of https://github.com/apache/lucene.git
LUCENE-5404: Add .getCount method to all suggesters (Lookup), persist count metadata on .store(), Dictionary returns InputIterator, Dictionary.getWordIterator renamed to .getEntryIterator
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1565810 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
f9ea3f8101
commit
b5926da2d6
|
@ -77,6 +77,10 @@ New Features
|
|||
(missing the term, weight or payload). (Areek Zillur via
|
||||
Mike McCandless)
|
||||
|
||||
* LUCENE-5404: Add .getCount method to all suggesters (Lookup); persist count
|
||||
metadata on .store(); Dictionary returns InputIterator; Dictionary.getWordIterator
|
||||
renamed to .getEntryIterator. (Areek Zillur)
|
||||
|
||||
* SOLR-1871: The RangeMapFloatFunction accepts an arbitrary ValueSource
|
||||
as target and default values. (Chris Harris, shalin)
|
||||
|
||||
|
|
|
@ -17,19 +17,20 @@ package org.apache.lucene.search.spell;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.lucene.util.BytesRefIterator;
|
||||
|
||||
import org.apache.lucene.search.suggest.InputIterator;
|
||||
|
||||
/**
|
||||
* A simple interface representing a Dictionary. A Dictionary
|
||||
* here is just a list of words.
|
||||
*
|
||||
* here is a list of entries, where every entry consists of
|
||||
* term, weight and payload.
|
||||
*
|
||||
*/
|
||||
public interface Dictionary {
|
||||
|
||||
/**
|
||||
* Return all words present in the dictionary
|
||||
* Returns an iterator over all the entries
|
||||
* @return Iterator
|
||||
*/
|
||||
BytesRefIterator getWordsIterator() throws IOException;
|
||||
InputIterator getEntryIterator() throws IOException;
|
||||
}
|
||||
|
|
|
@ -56,7 +56,7 @@ public class HighFrequencyDictionary implements Dictionary {
|
|||
}
|
||||
|
||||
@Override
|
||||
public final BytesRefIterator getWordsIterator() throws IOException {
|
||||
public final InputIterator getEntryIterator() throws IOException {
|
||||
return new HighFrequencyIterator();
|
||||
}
|
||||
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.search.spell;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.search.suggest.InputIterator;
|
||||
import org.apache.lucene.util.BytesRefIterator;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.MultiFields;
|
||||
|
@ -42,12 +43,12 @@ public class LuceneDictionary implements Dictionary {
|
|||
}
|
||||
|
||||
@Override
|
||||
public final BytesRefIterator getWordsIterator() throws IOException {
|
||||
public final InputIterator getEntryIterator() throws IOException {
|
||||
final Terms terms = MultiFields.getTerms(reader, field);
|
||||
if (terms != null) {
|
||||
return terms.iterator(null);
|
||||
return new InputIterator.InputIteratorWrapper(terms.iterator(null));
|
||||
} else {
|
||||
return BytesRefIterator.EMPTY;
|
||||
return InputIterator.EMPTY;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.io.IOException;
|
|||
import java.io.InputStream;
|
||||
import java.io.Reader;
|
||||
|
||||
import org.apache.lucene.search.suggest.InputIterator;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefIterator;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
@ -66,8 +67,8 @@ public class PlainTextDictionary implements Dictionary {
|
|||
}
|
||||
|
||||
@Override
|
||||
public BytesRefIterator getWordsIterator() throws IOException {
|
||||
return new FileIterator();
|
||||
public InputIterator getEntryIterator() throws IOException {
|
||||
return new InputIterator.InputIteratorWrapper(new FileIterator());
|
||||
}
|
||||
|
||||
final class FileIterator implements BytesRefIterator {
|
||||
|
|
|
@ -512,7 +512,7 @@ public class SpellChecker implements java.io.Closeable {
|
|||
boolean isEmpty = termsEnums.isEmpty();
|
||||
|
||||
try {
|
||||
BytesRefIterator iter = dict.getWordsIterator();
|
||||
BytesRefIterator iter = dict.getEntryIterator();
|
||||
BytesRef currentTerm;
|
||||
|
||||
terms: while ((currentTerm = iter.next()) != null) {
|
||||
|
|
|
@ -86,7 +86,7 @@ public class DocumentDictionary implements Dictionary {
|
|||
}
|
||||
|
||||
@Override
|
||||
public BytesRefIterator getWordsIterator() throws IOException {
|
||||
public InputIterator getEntryIterator() throws IOException {
|
||||
return new DocumentInputIterator(payloadField!=null);
|
||||
}
|
||||
|
||||
|
@ -102,7 +102,6 @@ public class DocumentDictionary implements Dictionary {
|
|||
private BytesRef currentPayload = null;
|
||||
private final NumericDocValues weightValues;
|
||||
|
||||
|
||||
/**
|
||||
* Creates an iterator over term, weight and payload fields from the lucene
|
||||
* index. setting <code>withPayload</code> to false, implies an iterator
|
||||
|
|
|
@ -92,7 +92,7 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
|
|||
}
|
||||
|
||||
@Override
|
||||
public BytesRefIterator getWordsIterator() throws IOException {
|
||||
public InputIterator getEntryIterator() throws IOException {
|
||||
return new DocumentValueSourceInputIterator(payloadField!=null);
|
||||
}
|
||||
|
||||
|
|
|
@ -106,7 +106,7 @@ public class FileDictionary implements Dictionary {
|
|||
}
|
||||
|
||||
@Override
|
||||
public InputIterator getWordsIterator() {
|
||||
public InputIterator getEntryIterator() {
|
||||
try {
|
||||
return new FileIterator();
|
||||
} catch (IOException e) {
|
||||
|
|
|
@ -44,6 +44,9 @@ public interface InputIterator extends BytesRefIterator {
|
|||
/** Returns true if the iterator has payloads */
|
||||
public boolean hasPayloads();
|
||||
|
||||
/** Singleton InputIterator that iterates over 0 BytesRefs. */
|
||||
public static final InputIterator EMPTY = new InputIteratorWrapper(BytesRefIterator.EMPTY);
|
||||
|
||||
/**
|
||||
* Wraps a BytesRefIterator as a suggester InputIterator, with all weights
|
||||
* set to <code>1</code> and carries no payload
|
||||
|
|
|
@ -24,8 +24,12 @@ import java.util.Comparator;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.search.spell.Dictionary;
|
||||
import org.apache.lucene.store.DataInput;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.store.InputStreamDataInput;
|
||||
import org.apache.lucene.store.OutputStreamDataOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefIterator;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
|
||||
/**
|
||||
|
@ -33,6 +37,7 @@ import org.apache.lucene.util.PriorityQueue;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class Lookup {
|
||||
|
||||
/**
|
||||
* Result of a lookup.
|
||||
*/
|
||||
|
@ -157,21 +162,46 @@ public abstract class Lookup {
|
|||
* {@link UnsortedInputIterator} in such case.
|
||||
*/
|
||||
public void build(Dictionary dict) throws IOException {
|
||||
BytesRefIterator it = dict.getWordsIterator();
|
||||
InputIterator tfit;
|
||||
if (it instanceof InputIterator) {
|
||||
tfit = (InputIterator)it;
|
||||
} else {
|
||||
tfit = new InputIterator.InputIteratorWrapper(it);
|
||||
build(dict.getEntryIterator());
|
||||
}
|
||||
build(tfit);
|
||||
|
||||
/**
|
||||
* Calls {@link #load(DataInput)} after converting
|
||||
* {@link InputStream} to {@link DataInput}
|
||||
*/
|
||||
public boolean load(InputStream input) throws IOException {
|
||||
DataInput dataIn = new InputStreamDataInput(input);
|
||||
try {
|
||||
return load(dataIn);
|
||||
} finally {
|
||||
IOUtils.close(input);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Calls {@link #store(DataOutput)} after converting
|
||||
* {@link OutputStream} to {@link DataOutput}
|
||||
*/
|
||||
public boolean store(OutputStream output) throws IOException {
|
||||
DataOutput dataOut = new OutputStreamDataOutput(output);
|
||||
try {
|
||||
return store(dataOut);
|
||||
} finally {
|
||||
IOUtils.close(output);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the number of entries the lookup was built with
|
||||
* @return total number of suggester entries
|
||||
*/
|
||||
public abstract long getCount();
|
||||
|
||||
/**
|
||||
* Builds up a new internal {@link Lookup} representation based on the given {@link InputIterator}.
|
||||
* The implementation might re-sort the data internally.
|
||||
*/
|
||||
public abstract void build(InputIterator tfit) throws IOException;
|
||||
public abstract void build(InputIterator inputIterator) throws IOException;
|
||||
|
||||
/**
|
||||
* Look up a key and return possible completion for this key.
|
||||
|
@ -183,23 +213,22 @@ public abstract class Lookup {
|
|||
*/
|
||||
public abstract List<LookupResult> lookup(CharSequence key, boolean onlyMorePopular, int num);
|
||||
|
||||
|
||||
/**
|
||||
* Persist the constructed lookup data to a directory. Optional operation.
|
||||
* @param output {@link OutputStream} to write the data to.
|
||||
* @param output {@link DataOutput} to write the data to.
|
||||
* @return true if successful, false if unsuccessful or not supported.
|
||||
* @throws IOException when fatal IO error occurs.
|
||||
*/
|
||||
public abstract boolean store(OutputStream output) throws IOException;
|
||||
public abstract boolean store(DataOutput output) throws IOException;
|
||||
|
||||
/**
|
||||
* Discard current lookup data and load it from a previously saved copy.
|
||||
* Optional operation.
|
||||
* @param input the {@link InputStream} to load the lookup data.
|
||||
* @param input the {@link DataInput} to load the lookup data.
|
||||
* @return true if completed successfully, false if unsuccessful or not supported.
|
||||
* @throws IOException when fatal IO error occurs.
|
||||
*/
|
||||
public abstract boolean load(InputStream input) throws IOException;
|
||||
public abstract boolean load(DataInput input) throws IOException;
|
||||
|
||||
/**
|
||||
* Get the size of the underlying lookup implementation in memory
|
||||
|
|
|
@ -20,8 +20,6 @@ package org.apache.lucene.search.suggest.analyzing;
|
|||
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;
|
||||
|
@ -68,6 +66,8 @@ import org.apache.lucene.search.TopDocs;
|
|||
import org.apache.lucene.search.suggest.Lookup.LookupResult; // javadocs
|
||||
import org.apache.lucene.search.suggest.InputIterator;
|
||||
import org.apache.lucene.search.suggest.Lookup;
|
||||
import org.apache.lucene.store.DataInput;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -107,6 +107,8 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
|||
private final File indexPath;
|
||||
final int minPrefixChars;
|
||||
private Directory dir;
|
||||
/** Number of entries the lookup was built with */
|
||||
private long count = 0;
|
||||
|
||||
/** {@link IndexSearcher} used for lookups. */
|
||||
protected IndexSearcher searcher;
|
||||
|
@ -154,12 +156,14 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
|||
|
||||
if (DirectoryReader.indexExists(dir)) {
|
||||
// Already built; open it:
|
||||
searcher = new IndexSearcher(DirectoryReader.open(dir));
|
||||
IndexReader reader = DirectoryReader.open(dir);
|
||||
searcher = new IndexSearcher(reader);
|
||||
// This will just be null if app didn't pass payloads to build():
|
||||
// TODO: maybe 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);
|
||||
count = reader.numDocs();
|
||||
assert textDV != null;
|
||||
}
|
||||
}
|
||||
|
@ -194,6 +198,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
|||
IndexWriter w2 = null;
|
||||
AtomicReader r = null;
|
||||
boolean success = false;
|
||||
count = 0;
|
||||
try {
|
||||
Analyzer gramAnalyzer = new AnalyzerWrapper(Analyzer.PER_FIELD_REUSE_STRATEGY) {
|
||||
@Override
|
||||
|
@ -239,7 +244,6 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
|||
} else {
|
||||
payloadField = null;
|
||||
}
|
||||
|
||||
//long t0 = System.nanoTime();
|
||||
while ((text = iter.next()) != null) {
|
||||
String textString = text.utf8ToString();
|
||||
|
@ -251,6 +255,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
|||
payloadField.setBytesValue(iter.payload());
|
||||
}
|
||||
w.addDocument(doc);
|
||||
count++;
|
||||
}
|
||||
//System.out.println("initial indexing time: " + ((System.nanoTime()-t0)/1000000) + " msec");
|
||||
|
||||
|
@ -612,12 +617,12 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean store(OutputStream out) {
|
||||
public boolean store(DataOutput in) throws IOException {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean load(InputStream out) {
|
||||
public boolean load(DataInput out) throws IOException {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
@ -637,4 +642,9 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
|
|||
public long sizeInBytes() {
|
||||
return RamUsageEstimator.sizeOf(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getCount() {
|
||||
return count;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -19,8 +19,6 @@ package org.apache.lucene.search.suggest.analyzing;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
|
@ -38,8 +36,6 @@ import org.apache.lucene.store.ByteArrayDataInput;
|
|||
import org.apache.lucene.store.ByteArrayDataOutput;
|
||||
import org.apache.lucene.store.DataInput;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.store.InputStreamDataInput;
|
||||
import org.apache.lucene.store.OutputStreamDataOutput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
|
@ -187,6 +183,9 @@ public class AnalyzingSuggester extends Lookup {
|
|||
/** Whether position holes should appear in the automaton. */
|
||||
private boolean preservePositionIncrements;
|
||||
|
||||
/** Number of entries the lookup was built with */
|
||||
private long count = 0;
|
||||
|
||||
/**
|
||||
* Calls {@link #AnalyzingSuggester(Analyzer,Analyzer,int,int,int,boolean)
|
||||
* AnalyzingSuggester(analyzer, analyzer, EXACT_FIRST |
|
||||
|
@ -394,6 +393,7 @@ public class AnalyzingSuggester extends Lookup {
|
|||
TokenStreamToAutomaton ts2a = getTokenStreamToAutomaton();
|
||||
|
||||
boolean success = false;
|
||||
count = 0;
|
||||
byte buffer[] = new byte[8];
|
||||
try {
|
||||
ByteArrayDataOutput output = new ByteArrayDataOutput(buffer);
|
||||
|
@ -458,6 +458,7 @@ public class AnalyzingSuggester extends Lookup {
|
|||
|
||||
writer.write(buffer, 0, output.getPosition());
|
||||
}
|
||||
count++;
|
||||
}
|
||||
writer.close();
|
||||
|
||||
|
@ -571,32 +572,24 @@ public class AnalyzingSuggester extends Lookup {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean store(OutputStream output) throws IOException {
|
||||
DataOutput dataOut = new OutputStreamDataOutput(output);
|
||||
try {
|
||||
public boolean store(DataOutput output) throws IOException {
|
||||
output.writeVLong(count);
|
||||
if (fst == null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
fst.save(dataOut);
|
||||
dataOut.writeVInt(maxAnalyzedPathsForOneInput);
|
||||
dataOut.writeByte((byte) (hasPayloads ? 1 : 0));
|
||||
} finally {
|
||||
IOUtils.close(output);
|
||||
}
|
||||
fst.save(output);
|
||||
output.writeVInt(maxAnalyzedPathsForOneInput);
|
||||
output.writeByte((byte) (hasPayloads ? 1 : 0));
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean load(InputStream input) throws IOException {
|
||||
DataInput dataIn = new InputStreamDataInput(input);
|
||||
try {
|
||||
this.fst = new FST<Pair<Long,BytesRef>>(dataIn, new PairOutputs<Long,BytesRef>(PositiveIntOutputs.getSingleton(), ByteSequenceOutputs.getSingleton()));
|
||||
maxAnalyzedPathsForOneInput = dataIn.readVInt();
|
||||
hasPayloads = dataIn.readByte() == 1;
|
||||
} finally {
|
||||
IOUtils.close(input);
|
||||
}
|
||||
public boolean load(DataInput input) throws IOException {
|
||||
count = input.readVLong();
|
||||
this.fst = new FST<Pair<Long,BytesRef>>(input, new PairOutputs<Long,BytesRef>(PositiveIntOutputs.getSingleton(), ByteSequenceOutputs.getSingleton()));
|
||||
maxAnalyzedPathsForOneInput = input.readVInt();
|
||||
hasPayloads = input.readByte() == 1;
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -810,6 +803,11 @@ public class AnalyzingSuggester extends Lookup {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getCount() {
|
||||
return count;
|
||||
}
|
||||
|
||||
/** Returns all prefix paths to initialize the search. */
|
||||
protected List<FSTUtil.Path<Pair<Long,BytesRef>>> getFullPrefixPaths(List<FSTUtil.Path<Pair<Long,BytesRef>>> prefixPaths,
|
||||
Automaton lookupAutomaton,
|
||||
|
|
|
@ -22,8 +22,6 @@ package org.apache.lucene.search.suggest.analyzing;
|
|||
// - add pruning of low-freq ngrams?
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
//import java.io.PrintWriter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
|
@ -62,8 +60,6 @@ import org.apache.lucene.store.DataInput;
|
|||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
import org.apache.lucene.store.InputStreamDataInput;
|
||||
import org.apache.lucene.store.OutputStreamDataOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
@ -153,6 +149,10 @@ public class FreeTextSuggester extends Lookup {
|
|||
private final int grams;
|
||||
|
||||
private final byte separator;
|
||||
|
||||
/** Number of entries the lookup was built with */
|
||||
private long count = 0;
|
||||
|
||||
/** The default character used to join multiple tokens
|
||||
* into a single ngram token. The input tokens produced
|
||||
* by the analyzer must not contain this character. */
|
||||
|
@ -320,6 +320,7 @@ public class FreeTextSuggester extends Lookup {
|
|||
IndexReader reader = null;
|
||||
|
||||
boolean success = false;
|
||||
count = 0;
|
||||
try {
|
||||
while (true) {
|
||||
BytesRef surfaceForm = iterator.next();
|
||||
|
@ -328,6 +329,7 @@ public class FreeTextSuggester extends Lookup {
|
|||
}
|
||||
field.setStringValue(surfaceForm.utf8ToString());
|
||||
writer.addDocument(doc);
|
||||
count++;
|
||||
}
|
||||
reader = DirectoryReader.open(writer, false);
|
||||
|
||||
|
@ -397,31 +399,31 @@ public class FreeTextSuggester extends Lookup {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean store(OutputStream output) throws IOException {
|
||||
DataOutput out = new OutputStreamDataOutput(output);
|
||||
CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
|
||||
out.writeByte(separator);
|
||||
out.writeVInt(grams);
|
||||
out.writeVLong(totTokens);
|
||||
fst.save(out);
|
||||
public boolean store(DataOutput output) throws IOException {
|
||||
CodecUtil.writeHeader(output, CODEC_NAME, VERSION_CURRENT);
|
||||
output.writeVLong(count);
|
||||
output.writeByte(separator);
|
||||
output.writeVInt(grams);
|
||||
output.writeVLong(totTokens);
|
||||
fst.save(output);
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean load(InputStream input) throws IOException {
|
||||
DataInput in = new InputStreamDataInput(input);
|
||||
CodecUtil.checkHeader(in, CODEC_NAME, VERSION_START, VERSION_START);
|
||||
byte separatorOrig = in.readByte();
|
||||
public boolean load(DataInput input) throws IOException {
|
||||
CodecUtil.checkHeader(input, CODEC_NAME, VERSION_START, VERSION_START);
|
||||
count = input.readVLong();
|
||||
byte separatorOrig = input.readByte();
|
||||
if (separatorOrig != separator) {
|
||||
throw new IllegalStateException("separator=" + separator + " is incorrect: original model was built with separator=" + separatorOrig);
|
||||
}
|
||||
int gramsOrig = in.readVInt();
|
||||
int gramsOrig = input.readVInt();
|
||||
if (gramsOrig != grams) {
|
||||
throw new IllegalStateException("grams=" + grams + " is incorrect: original model was built with grams=" + gramsOrig);
|
||||
}
|
||||
totTokens = in.readVLong();
|
||||
totTokens = input.readVLong();
|
||||
|
||||
fst = new FST<Long>(in, PositiveIntOutputs.getSingleton());
|
||||
fst = new FST<Long>(input, PositiveIntOutputs.getSingleton());
|
||||
|
||||
return true;
|
||||
}
|
||||
|
@ -436,6 +438,11 @@ public class FreeTextSuggester extends Lookup {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getCount() {
|
||||
return count;
|
||||
}
|
||||
|
||||
private int countGrams(BytesRef token) {
|
||||
int count = 1;
|
||||
for(int i=0;i<token.length;i++) {
|
||||
|
|
|
@ -32,6 +32,8 @@ import org.apache.lucene.search.suggest.fst.FSTCompletion.Completion;
|
|||
import org.apache.lucene.search.suggest.tst.TSTLookup;
|
||||
import org.apache.lucene.store.ByteArrayDataInput;
|
||||
import org.apache.lucene.store.ByteArrayDataOutput;
|
||||
import org.apache.lucene.store.DataInput;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.store.InputStreamDataInput;
|
||||
import org.apache.lucene.store.OutputStreamDataOutput;
|
||||
import org.apache.lucene.util.*;
|
||||
|
@ -93,6 +95,9 @@ public class FSTCompletionLookup extends Lookup {
|
|||
*/
|
||||
private FSTCompletion normalCompletion;
|
||||
|
||||
/** Number of entries the lookup was built with */
|
||||
private long count = 0;
|
||||
|
||||
/**
|
||||
* This constructor prepares for creating a suggested FST using the
|
||||
* {@link #build(InputIterator)} method. The number of weight
|
||||
|
@ -140,8 +145,8 @@ public class FSTCompletionLookup extends Lookup {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void build(InputIterator tfit) throws IOException {
|
||||
if (tfit.hasPayloads()) {
|
||||
public void build(InputIterator iterator) throws IOException {
|
||||
if (iterator.hasPayloads()) {
|
||||
throw new IllegalArgumentException("this suggester doesn't support payloads");
|
||||
}
|
||||
File tempInput = File.createTempFile(
|
||||
|
@ -156,17 +161,18 @@ public class FSTCompletionLookup extends Lookup {
|
|||
// Push floats up front before sequences to sort them. For now, assume they are non-negative.
|
||||
// If negative floats are allowed some trickery needs to be done to find their byte order.
|
||||
boolean success = false;
|
||||
count = 0;
|
||||
try {
|
||||
byte [] buffer = new byte [0];
|
||||
ByteArrayDataOutput output = new ByteArrayDataOutput(buffer);
|
||||
BytesRef spare;
|
||||
while ((spare = tfit.next()) != null) {
|
||||
while ((spare = iterator.next()) != null) {
|
||||
if (spare.length + 4 >= buffer.length) {
|
||||
buffer = ArrayUtil.grow(buffer, spare.length + 4);
|
||||
}
|
||||
|
||||
output.reset(buffer);
|
||||
output.writeInt(encodeWeight(tfit.weight()));
|
||||
output.writeInt(encodeWeight(iterator.weight()));
|
||||
output.writeBytes(spare.bytes, spare.offset, spare.length);
|
||||
writer.write(buffer, 0, output.getPosition());
|
||||
}
|
||||
|
@ -207,6 +213,7 @@ public class FSTCompletionLookup extends Lookup {
|
|||
builder.add(tmp2, bucket);
|
||||
|
||||
line++;
|
||||
count++;
|
||||
}
|
||||
|
||||
// The two FSTCompletions share the same automaton.
|
||||
|
@ -264,28 +271,21 @@ public class FSTCompletionLookup extends Lookup {
|
|||
|
||||
|
||||
@Override
|
||||
public synchronized boolean store(OutputStream output) throws IOException {
|
||||
|
||||
try {
|
||||
public synchronized boolean store(DataOutput output) throws IOException {
|
||||
output.writeVLong(count);
|
||||
if (this.normalCompletion == null || normalCompletion.getFST() == null)
|
||||
return false;
|
||||
normalCompletion.getFST().save(new OutputStreamDataOutput(output));
|
||||
} finally {
|
||||
IOUtils.close(output);
|
||||
}
|
||||
normalCompletion.getFST().save(output);
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized boolean load(InputStream input) throws IOException {
|
||||
try {
|
||||
public synchronized boolean load(DataInput input) throws IOException {
|
||||
count = input.readVLong();
|
||||
this.higherWeightsCompletion = new FSTCompletion(new FST<Object>(
|
||||
new InputStreamDataInput(input), NoOutputs.getSingleton()));
|
||||
input, NoOutputs.getSingleton()));
|
||||
this.normalCompletion = new FSTCompletion(
|
||||
higherWeightsCompletion.getFST(), false, exactMatchFirst);
|
||||
} finally {
|
||||
IOUtils.close(input);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -293,4 +293,9 @@ public class FSTCompletionLookup extends Lookup {
|
|||
public long sizeInBytes() {
|
||||
return RamUsageEstimator.sizeOf(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getCount() {
|
||||
return count;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,8 +18,6 @@ package org.apache.lucene.search.suggest.fst;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
|
@ -31,12 +29,11 @@ import org.apache.lucene.search.suggest.Sort.ByteSequencesWriter;
|
|||
import org.apache.lucene.search.suggest.SortedInputIterator;
|
||||
import org.apache.lucene.store.ByteArrayDataInput;
|
||||
import org.apache.lucene.store.ByteArrayDataOutput;
|
||||
import org.apache.lucene.store.InputStreamDataInput;
|
||||
import org.apache.lucene.store.OutputStreamDataOutput;
|
||||
import org.apache.lucene.store.DataInput;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.apache.lucene.util.fst.Builder;
|
||||
|
@ -72,6 +69,9 @@ public class WFSTCompletionLookup extends Lookup {
|
|||
*/
|
||||
private final boolean exactFirst;
|
||||
|
||||
/** Number of entries the lookup was built with */
|
||||
private long count = 0;
|
||||
|
||||
/**
|
||||
* Calls {@link #WFSTCompletionLookup(boolean) WFSTCompletionLookup(true)}
|
||||
*/
|
||||
|
@ -96,6 +96,7 @@ public class WFSTCompletionLookup extends Lookup {
|
|||
if (iterator.hasPayloads()) {
|
||||
throw new IllegalArgumentException("this suggester doesn't support payloads");
|
||||
}
|
||||
count = 0;
|
||||
BytesRef scratch = new BytesRef();
|
||||
InputIterator iter = new WFSTInputIterator(iterator);
|
||||
IntsRef scratchInts = new IntsRef();
|
||||
|
@ -114,31 +115,26 @@ public class WFSTCompletionLookup extends Lookup {
|
|||
Util.toIntsRef(scratch, scratchInts);
|
||||
builder.add(scratchInts, cost);
|
||||
previous.copyBytes(scratch);
|
||||
count++;
|
||||
}
|
||||
fst = builder.finish();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean store(OutputStream output) throws IOException {
|
||||
try {
|
||||
public boolean store(DataOutput output) throws IOException {
|
||||
output.writeVLong(count);
|
||||
if (fst == null) {
|
||||
return false;
|
||||
}
|
||||
fst.save(new OutputStreamDataOutput(output));
|
||||
} finally {
|
||||
IOUtils.close(output);
|
||||
}
|
||||
fst.save(output);
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean load(InputStream input) throws IOException {
|
||||
try {
|
||||
this.fst = new FST<Long>(new InputStreamDataInput(input), PositiveIntOutputs.getSingleton());
|
||||
} finally {
|
||||
IOUtils.close(input);
|
||||
}
|
||||
public boolean load(DataInput input) throws IOException {
|
||||
count = input.readVLong();
|
||||
this.fst = new FST<Long>(input, PositiveIntOutputs.getSingleton());
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -293,4 +289,9 @@ public class WFSTCompletionLookup extends Lookup {
|
|||
public long sizeInBytes() {
|
||||
return (fst == null) ? 0 : fst.sizeInBytes();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getCount() {
|
||||
return count;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,20 +17,17 @@ package org.apache.lucene.search.suggest.jaspell;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.search.suggest.InputIterator;
|
||||
import org.apache.lucene.search.suggest.Lookup;
|
||||
import org.apache.lucene.search.suggest.jaspell.JaspellTernarySearchTrie.TSTNode;
|
||||
import org.apache.lucene.store.DataInput;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
|
||||
|
@ -45,6 +42,9 @@ public class JaspellLookup extends Lookup {
|
|||
private boolean usePrefix = true;
|
||||
private int editDistance = 2;
|
||||
|
||||
/** Number of entries the lookup was built with */
|
||||
private long count = 0;
|
||||
|
||||
/**
|
||||
* Creates a new empty trie
|
||||
* @see #build(InputIterator)
|
||||
|
@ -52,23 +52,25 @@ public class JaspellLookup extends Lookup {
|
|||
public JaspellLookup() {}
|
||||
|
||||
@Override
|
||||
public void build(InputIterator tfit) throws IOException {
|
||||
if (tfit.hasPayloads()) {
|
||||
public void build(InputIterator iterator) throws IOException {
|
||||
if (iterator.hasPayloads()) {
|
||||
throw new IllegalArgumentException("this suggester doesn't support payloads");
|
||||
}
|
||||
count = 0;
|
||||
trie = new JaspellTernarySearchTrie();
|
||||
trie.setMatchAlmostDiff(editDistance);
|
||||
BytesRef spare;
|
||||
final CharsRef charsSpare = new CharsRef();
|
||||
|
||||
while ((spare = tfit.next()) != null) {
|
||||
final long weight = tfit.weight();
|
||||
while ((spare = iterator.next()) != null) {
|
||||
final long weight = iterator.weight();
|
||||
if (spare.length == 0) {
|
||||
continue;
|
||||
}
|
||||
charsSpare.grow(spare.length);
|
||||
UnicodeUtil.UTF8toUTF16(spare.bytes, spare.offset, spare.length, charsSpare);
|
||||
trie.put(charsSpare.toString(), Long.valueOf(weight));
|
||||
count++;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -131,8 +133,8 @@ public class JaspellLookup extends Lookup {
|
|||
private static final byte HI_KID = 0x04;
|
||||
private static final byte HAS_VALUE = 0x08;
|
||||
|
||||
private void readRecursively(DataInputStream in, TSTNode node) throws IOException {
|
||||
node.splitchar = in.readChar();
|
||||
private void readRecursively(DataInput in, TSTNode node) throws IOException {
|
||||
node.splitchar = in.readString().charAt(0);
|
||||
byte mask = in.readByte();
|
||||
if ((mask & HAS_VALUE) != 0) {
|
||||
node.data = Long.valueOf(in.readLong());
|
||||
|
@ -154,11 +156,11 @@ public class JaspellLookup extends Lookup {
|
|||
}
|
||||
}
|
||||
|
||||
private void writeRecursively(DataOutputStream out, TSTNode node) throws IOException {
|
||||
private void writeRecursively(DataOutput out, TSTNode node) throws IOException {
|
||||
if (node == null) {
|
||||
return;
|
||||
}
|
||||
out.writeChar(node.splitchar);
|
||||
out.writeString(new String(new char[] {node.splitchar}, 0, 1));
|
||||
byte mask = 0;
|
||||
if (node.relatives[TSTNode.LOKID] != null) mask |= LO_KID;
|
||||
if (node.relatives[TSTNode.EQKID] != null) mask |= EQ_KID;
|
||||
|
@ -174,31 +176,22 @@ public class JaspellLookup extends Lookup {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean store(OutputStream output) throws IOException {
|
||||
public boolean store(DataOutput output) throws IOException {
|
||||
output.writeVLong(count);
|
||||
TSTNode root = trie.getRoot();
|
||||
if (root == null) { // empty tree
|
||||
return false;
|
||||
}
|
||||
DataOutputStream out = new DataOutputStream(output);
|
||||
try {
|
||||
writeRecursively(out, root);
|
||||
out.flush();
|
||||
} finally {
|
||||
IOUtils.close(out);
|
||||
}
|
||||
writeRecursively(output, root);
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean load(InputStream input) throws IOException {
|
||||
DataInputStream in = new DataInputStream(input);
|
||||
public boolean load(DataInput input) throws IOException {
|
||||
count = input.readVLong();
|
||||
TSTNode root = trie.new TSTNode('\0', null);
|
||||
try {
|
||||
readRecursively(in, root);
|
||||
readRecursively(input, root);
|
||||
trie.setRoot(root);
|
||||
} finally {
|
||||
IOUtils.close(in);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -207,4 +200,9 @@ public class JaspellLookup extends Lookup {
|
|||
public long sizeInBytes() {
|
||||
return RamUsageEstimator.sizeOf(trie);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getCount() {
|
||||
return count;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,20 +17,17 @@ package org.apache.lucene.search.suggest.tst;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.search.suggest.InputIterator;
|
||||
import org.apache.lucene.search.suggest.Lookup;
|
||||
import org.apache.lucene.search.suggest.SortedInputIterator;
|
||||
import org.apache.lucene.store.DataInput;
|
||||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
|
||||
|
@ -44,6 +41,9 @@ public class TSTLookup extends Lookup {
|
|||
TernaryTreeNode root = new TernaryTreeNode();
|
||||
TSTAutocomplete autocomplete = new TSTAutocomplete();
|
||||
|
||||
/** Number of entries the lookup was built with */
|
||||
private long count = 0;
|
||||
|
||||
/**
|
||||
* Creates a new TSTLookup with an empty Ternary Search Tree.
|
||||
* @see #build(InputIterator)
|
||||
|
@ -51,24 +51,25 @@ public class TSTLookup extends Lookup {
|
|||
public TSTLookup() {}
|
||||
|
||||
@Override
|
||||
public void build(InputIterator tfit) throws IOException {
|
||||
if (tfit.hasPayloads()) {
|
||||
public void build(InputIterator iterator) throws IOException {
|
||||
if (iterator.hasPayloads()) {
|
||||
throw new IllegalArgumentException("this suggester doesn't support payloads");
|
||||
}
|
||||
root = new TernaryTreeNode();
|
||||
|
||||
// make sure it's sorted and the comparator uses UTF16 sort order
|
||||
tfit = new SortedInputIterator(tfit, BytesRef.getUTF8SortedAsUTF16Comparator());
|
||||
|
||||
iterator = new SortedInputIterator(iterator, BytesRef.getUTF8SortedAsUTF16Comparator());
|
||||
count = 0;
|
||||
ArrayList<String> tokens = new ArrayList<String>();
|
||||
ArrayList<Number> vals = new ArrayList<Number>();
|
||||
BytesRef spare;
|
||||
CharsRef charsSpare = new CharsRef();
|
||||
while ((spare = tfit.next()) != null) {
|
||||
while ((spare = iterator.next()) != null) {
|
||||
charsSpare.grow(spare.length);
|
||||
UnicodeUtil.UTF8toUTF16(spare.bytes, spare.offset, spare.length, charsSpare);
|
||||
tokens.add(charsSpare.toString());
|
||||
vals.add(Long.valueOf(tfit.weight()));
|
||||
vals.add(Long.valueOf(iterator.weight()));
|
||||
count++;
|
||||
}
|
||||
autocomplete.balancedTree(tokens.toArray(), vals.toArray(), 0, tokens.size() - 1, root);
|
||||
}
|
||||
|
@ -148,11 +149,11 @@ public class TSTLookup extends Lookup {
|
|||
private static final byte HAS_VALUE = 0x10;
|
||||
|
||||
// pre-order traversal
|
||||
private void readRecursively(DataInputStream in, TernaryTreeNode node) throws IOException {
|
||||
node.splitchar = in.readChar();
|
||||
private void readRecursively(DataInput in, TernaryTreeNode node) throws IOException {
|
||||
node.splitchar = in.readString().charAt(0);
|
||||
byte mask = in.readByte();
|
||||
if ((mask & HAS_TOKEN) != 0) {
|
||||
node.token = in.readUTF();
|
||||
node.token = in.readString();
|
||||
}
|
||||
if ((mask & HAS_VALUE) != 0) {
|
||||
node.val = Long.valueOf(in.readLong());
|
||||
|
@ -172,9 +173,9 @@ public class TSTLookup extends Lookup {
|
|||
}
|
||||
|
||||
// pre-order traversal
|
||||
private void writeRecursively(DataOutputStream out, TernaryTreeNode node) throws IOException {
|
||||
private void writeRecursively(DataOutput out, TernaryTreeNode node) throws IOException {
|
||||
// write out the current node
|
||||
out.writeChar(node.splitchar);
|
||||
out.writeString(new String(new char[] {node.splitchar}, 0, 1));
|
||||
// prepare a mask of kids
|
||||
byte mask = 0;
|
||||
if (node.eqKid != null) mask |= EQ_KID;
|
||||
|
@ -183,7 +184,7 @@ public class TSTLookup extends Lookup {
|
|||
if (node.token != null) mask |= HAS_TOKEN;
|
||||
if (node.val != null) mask |= HAS_VALUE;
|
||||
out.writeByte(mask);
|
||||
if (node.token != null) out.writeUTF(node.token);
|
||||
if (node.token != null) out.writeString(node.token);
|
||||
if (node.val != null) out.writeLong(((Number)node.val).longValue());
|
||||
// recurse and write kids
|
||||
if (node.loKid != null) {
|
||||
|
@ -198,26 +199,17 @@ public class TSTLookup extends Lookup {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized boolean store(OutputStream output) throws IOException {
|
||||
DataOutputStream out = new DataOutputStream(output);
|
||||
try {
|
||||
writeRecursively(out, root);
|
||||
out.flush();
|
||||
} finally {
|
||||
IOUtils.close(output);
|
||||
}
|
||||
public synchronized boolean store(DataOutput output) throws IOException {
|
||||
output.writeVLong(count);
|
||||
writeRecursively(output, root);
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized boolean load(InputStream input) throws IOException {
|
||||
DataInputStream in = new DataInputStream(input);
|
||||
public synchronized boolean load(DataInput input) throws IOException {
|
||||
count = input.readVLong();
|
||||
root = new TernaryTreeNode();
|
||||
try {
|
||||
readRecursively(in, root);
|
||||
} finally {
|
||||
IOUtils.close(in);
|
||||
}
|
||||
readRecursively(input, root);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -227,4 +219,8 @@ public class TSTLookup extends Lookup {
|
|||
return RamUsageEstimator.sizeOf(autocomplete);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getCount() {
|
||||
return count;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -90,7 +90,7 @@ public class TestLuceneDictionary extends LuceneTestCase {
|
|||
indexReader = DirectoryReader.open(store);
|
||||
|
||||
ld = new LuceneDictionary(indexReader, "nonexistent_field");
|
||||
it = ld.getWordsIterator();
|
||||
it = ld.getEntryIterator();
|
||||
|
||||
assertNull("More elements than expected", spare = it.next());
|
||||
} finally {
|
||||
|
@ -103,7 +103,7 @@ public class TestLuceneDictionary extends LuceneTestCase {
|
|||
indexReader = DirectoryReader.open(store);
|
||||
|
||||
ld = new LuceneDictionary(indexReader, "aaa");
|
||||
it = ld.getWordsIterator();
|
||||
it = ld.getEntryIterator();
|
||||
assertNotNull("First element doesn't exist.", spare = it.next());
|
||||
assertTrue("First element isn't correct", spare.utf8ToString().equals("foo"));
|
||||
assertNull("More elements than expected", it.next());
|
||||
|
@ -117,7 +117,7 @@ public class TestLuceneDictionary extends LuceneTestCase {
|
|||
indexReader = DirectoryReader.open(store);
|
||||
|
||||
ld = new LuceneDictionary(indexReader, "contents");
|
||||
it = ld.getWordsIterator();
|
||||
it = ld.getEntryIterator();
|
||||
|
||||
assertNotNull("First element doesn't exist.", spare = it.next());
|
||||
assertTrue("First element isn't correct", spare.utf8ToString().equals("Jerry"));
|
||||
|
@ -126,7 +126,7 @@ public class TestLuceneDictionary extends LuceneTestCase {
|
|||
assertNull("More elements than expected", it.next());
|
||||
|
||||
ld = new LuceneDictionary(indexReader, "contents");
|
||||
it = ld.getWordsIterator();
|
||||
it = ld.getEntryIterator();
|
||||
|
||||
int counter = 2;
|
||||
while (it.next() != null) {
|
||||
|
@ -145,7 +145,7 @@ public class TestLuceneDictionary extends LuceneTestCase {
|
|||
indexReader = DirectoryReader.open(store);
|
||||
|
||||
ld = new LuceneDictionary(indexReader, "contents");
|
||||
it = ld.getWordsIterator();
|
||||
it = ld.getEntryIterator();
|
||||
|
||||
// just iterate through words
|
||||
assertEquals("First element isn't correct", "Jerry", it.next().utf8ToString());
|
||||
|
@ -162,7 +162,7 @@ public class TestLuceneDictionary extends LuceneTestCase {
|
|||
indexReader = DirectoryReader.open(store);
|
||||
|
||||
ld = new LuceneDictionary(indexReader, "zzz");
|
||||
it = ld.getWordsIterator();
|
||||
it = ld.getEntryIterator();
|
||||
|
||||
assertNotNull("First element doesn't exist.", spare = it.next());
|
||||
assertEquals("First element isn't correct", "bar", spare.utf8ToString());
|
||||
|
|
|
@ -109,7 +109,7 @@ public class DocumentDictionaryTest extends LuceneTestCase {
|
|||
writer.close();
|
||||
IndexReader ir = DirectoryReader.open(dir);
|
||||
Dictionary dictionary = new DocumentDictionary(ir, FIELD_NAME, WEIGHT_FIELD_NAME, PAYLOAD_FIELD_NAME);
|
||||
InputIterator inputIterator = (InputIterator) dictionary.getWordsIterator();
|
||||
InputIterator inputIterator = dictionary.getEntryIterator();
|
||||
|
||||
assertNull(inputIterator.next());
|
||||
assertEquals(inputIterator.weight(), 0);
|
||||
|
@ -135,7 +135,7 @@ public class DocumentDictionaryTest extends LuceneTestCase {
|
|||
writer.close();
|
||||
IndexReader ir = DirectoryReader.open(dir);
|
||||
Dictionary dictionary = new DocumentDictionary(ir, FIELD_NAME, WEIGHT_FIELD_NAME, PAYLOAD_FIELD_NAME);
|
||||
InputIterator inputIterator = (InputIterator) dictionary.getWordsIterator();
|
||||
InputIterator inputIterator = dictionary.getEntryIterator();
|
||||
BytesRef f;
|
||||
while((f = inputIterator.next())!=null) {
|
||||
Document doc = docs.remove(f.utf8ToString());
|
||||
|
@ -170,7 +170,7 @@ public class DocumentDictionaryTest extends LuceneTestCase {
|
|||
writer.close();
|
||||
IndexReader ir = DirectoryReader.open(dir);
|
||||
Dictionary dictionary = new DocumentDictionary(ir, FIELD_NAME, WEIGHT_FIELD_NAME);
|
||||
InputIterator inputIterator = (InputIterator) dictionary.getWordsIterator();
|
||||
InputIterator inputIterator = dictionary.getEntryIterator();
|
||||
BytesRef f;
|
||||
while((f = inputIterator.next())!=null) {
|
||||
Document doc = docs.remove(f.utf8ToString());
|
||||
|
@ -228,7 +228,7 @@ public class DocumentDictionaryTest extends LuceneTestCase {
|
|||
IndexReader ir = DirectoryReader.open(dir);
|
||||
assertEquals(ir.numDocs(), docs.size());
|
||||
Dictionary dictionary = new DocumentDictionary(ir, FIELD_NAME, WEIGHT_FIELD_NAME);
|
||||
InputIterator inputIterator = (InputIterator) dictionary.getWordsIterator();
|
||||
InputIterator inputIterator = dictionary.getEntryIterator();
|
||||
BytesRef f;
|
||||
while((f = inputIterator.next())!=null) {
|
||||
Document doc = docs.remove(f.utf8ToString());
|
||||
|
|
|
@ -83,7 +83,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
|
|||
writer.close();
|
||||
IndexReader ir = DirectoryReader.open(dir);
|
||||
Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, new DoubleConstValueSource(10), PAYLOAD_FIELD_NAME);
|
||||
InputIterator inputIterator = (InputIterator) dictionary.getWordsIterator();
|
||||
InputIterator inputIterator = (InputIterator) dictionary.getEntryIterator();
|
||||
|
||||
assertNull(inputIterator.next());
|
||||
assertEquals(inputIterator.weight(), 0);
|
||||
|
@ -109,7 +109,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
|
|||
IndexReader ir = DirectoryReader.open(dir);
|
||||
ValueSource[] toAdd = new ValueSource[] {new LongFieldSource(WEIGHT_FIELD_NAME_1), new LongFieldSource(WEIGHT_FIELD_NAME_2), new LongFieldSource(WEIGHT_FIELD_NAME_3)};
|
||||
Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, new SumFloatFunction(toAdd), PAYLOAD_FIELD_NAME);
|
||||
InputIterator inputIterator = (InputIterator) dictionary.getWordsIterator();
|
||||
InputIterator inputIterator = (InputIterator) dictionary.getEntryIterator();
|
||||
BytesRef f;
|
||||
while((f = inputIterator.next())!=null) {
|
||||
Document doc = docs.remove(f.utf8ToString());
|
||||
|
@ -141,7 +141,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
|
|||
IndexReader ir = DirectoryReader.open(dir);
|
||||
ValueSource[] toAdd = new ValueSource[] {new LongFieldSource(WEIGHT_FIELD_NAME_1), new LongFieldSource(WEIGHT_FIELD_NAME_2), new LongFieldSource(WEIGHT_FIELD_NAME_3)};
|
||||
Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, new SumFloatFunction(toAdd));
|
||||
InputIterator inputIterator = (InputIterator) dictionary.getWordsIterator();
|
||||
InputIterator inputIterator = (InputIterator) dictionary.getEntryIterator();
|
||||
BytesRef f;
|
||||
while((f = inputIterator.next())!=null) {
|
||||
Document doc = docs.remove(f.utf8ToString());
|
||||
|
@ -195,7 +195,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
|
|||
ValueSource[] toAdd = new ValueSource[] {new LongFieldSource(WEIGHT_FIELD_NAME_1), new LongFieldSource(WEIGHT_FIELD_NAME_2)};
|
||||
|
||||
Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, new SumFloatFunction(toAdd), PAYLOAD_FIELD_NAME);
|
||||
InputIterator inputIterator = (InputIterator) dictionary.getWordsIterator();
|
||||
InputIterator inputIterator = (InputIterator) dictionary.getEntryIterator();
|
||||
BytesRef f;
|
||||
while((f = inputIterator.next())!=null) {
|
||||
Document doc = docs.remove(f.utf8ToString());
|
||||
|
@ -226,7 +226,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
|
|||
|
||||
IndexReader ir = DirectoryReader.open(dir);
|
||||
Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, new DoubleConstValueSource(10), PAYLOAD_FIELD_NAME);
|
||||
InputIterator inputIterator = (InputIterator) dictionary.getWordsIterator();
|
||||
InputIterator inputIterator = (InputIterator) dictionary.getEntryIterator();
|
||||
BytesRef f;
|
||||
while((f = inputIterator.next())!=null) {
|
||||
Document doc = docs.remove(f.utf8ToString());
|
||||
|
|
|
@ -76,7 +76,7 @@ public class FileDictionaryTest extends LuceneTestCase {
|
|||
InputStream inputReader = new ByteArrayInputStream(fileInput.getValue().getBytes("UTF-8"));
|
||||
FileDictionary dictionary = new FileDictionary(inputReader);
|
||||
List<List<String>> entries = fileInput.getKey();
|
||||
InputIterator inputIter = dictionary.getWordsIterator();
|
||||
InputIterator inputIter = dictionary.getEntryIterator();
|
||||
assertFalse(inputIter.hasPayloads());
|
||||
BytesRef term;
|
||||
int count = 0;
|
||||
|
@ -98,7 +98,7 @@ public class FileDictionaryTest extends LuceneTestCase {
|
|||
InputStream inputReader = new ByteArrayInputStream(fileInput.getValue().getBytes("UTF-8"));
|
||||
FileDictionary dictionary = new FileDictionary(inputReader);
|
||||
List<List<String>> entries = fileInput.getKey();
|
||||
InputIterator inputIter = dictionary.getWordsIterator();
|
||||
InputIterator inputIter = dictionary.getEntryIterator();
|
||||
assertFalse(inputIter.hasPayloads());
|
||||
BytesRef term;
|
||||
int count = 0;
|
||||
|
@ -120,7 +120,7 @@ public class FileDictionaryTest extends LuceneTestCase {
|
|||
InputStream inputReader = new ByteArrayInputStream(fileInput.getValue().getBytes("UTF-8"));
|
||||
FileDictionary dictionary = new FileDictionary(inputReader);
|
||||
List<List<String>> entries = fileInput.getKey();
|
||||
InputIterator inputIter = dictionary.getWordsIterator();
|
||||
InputIterator inputIter = dictionary.getEntryIterator();
|
||||
assertTrue(inputIter.hasPayloads());
|
||||
BytesRef term;
|
||||
int count = 0;
|
||||
|
@ -146,7 +146,7 @@ public class FileDictionaryTest extends LuceneTestCase {
|
|||
InputStream inputReader = new ByteArrayInputStream(fileInput.getValue().getBytes("UTF-8"));
|
||||
FileDictionary dictionary = new FileDictionary(inputReader);
|
||||
List<List<String>> entries = fileInput.getKey();
|
||||
InputIterator inputIter = dictionary.getWordsIterator();
|
||||
InputIterator inputIter = dictionary.getEntryIterator();
|
||||
assertTrue(inputIter.hasPayloads());
|
||||
BytesRef term;
|
||||
int count = 0;
|
||||
|
@ -173,7 +173,7 @@ public class FileDictionaryTest extends LuceneTestCase {
|
|||
InputStream inputReader = new ByteArrayInputStream(fileInput.getValue().getBytes("UTF-8"));
|
||||
FileDictionary dictionary = new FileDictionary(inputReader, " , ");
|
||||
List<List<String>> entries = fileInput.getKey();
|
||||
InputIterator inputIter = dictionary.getWordsIterator();
|
||||
InputIterator inputIter = dictionary.getEntryIterator();
|
||||
assertTrue(inputIter.hasPayloads());
|
||||
BytesRef term;
|
||||
int count = 0;
|
||||
|
|
|
@ -35,7 +35,7 @@ public class TestHighFrequencyDictionary extends LuceneTestCase {
|
|||
writer.close();
|
||||
IndexReader ir = DirectoryReader.open(dir);
|
||||
Dictionary dictionary = new HighFrequencyDictionary(ir, "bogus", 0.1f);
|
||||
BytesRefIterator tf = dictionary.getWordsIterator();
|
||||
BytesRefIterator tf = dictionary.getEntryIterator();
|
||||
assertNull(tf.next());
|
||||
dir.close();
|
||||
}
|
||||
|
|
|
@ -110,6 +110,7 @@ public class AnalyzingInfixSuggesterTest extends LuceneTestCase {
|
|||
}
|
||||
};
|
||||
suggester.build(new InputArrayIterator(keys));
|
||||
assertEquals(2, suggester.getCount());
|
||||
suggester.close();
|
||||
|
||||
suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, tempDir, a, a, 3) {
|
||||
|
@ -123,6 +124,7 @@ public class AnalyzingInfixSuggesterTest extends LuceneTestCase {
|
|||
assertEquals("a penny saved is a penny <b>ear</b>ned", results.get(0).key);
|
||||
assertEquals(10, results.get(0).value);
|
||||
assertEquals(new BytesRef("foobaz"), results.get(0).payload);
|
||||
assertEquals(2, suggester.getCount());
|
||||
suggester.close();
|
||||
}
|
||||
|
||||
|
|
|
@ -912,6 +912,7 @@ public class AnalyzingSuggesterTest extends LuceneTestCase {
|
|||
new Input("a c", 4),
|
||||
}));
|
||||
|
||||
assertEquals(3, suggester.getCount());
|
||||
List<LookupResult> results = suggester.lookup("a", false, 3);
|
||||
assertEquals(3, results.size());
|
||||
assertEquals("a", results.get(0).key);
|
||||
|
@ -935,6 +936,7 @@ public class AnalyzingSuggesterTest extends LuceneTestCase {
|
|||
suggester.load(is);
|
||||
is.close();
|
||||
|
||||
assertEquals(3, suggester.getCount());
|
||||
results = suggester.lookup("a", false, 3);
|
||||
assertEquals(3, results.size());
|
||||
assertEquals("a", results.get(0).key);
|
||||
|
|
|
@ -62,6 +62,7 @@ public class TestFreeTextSuggester extends LuceneTestCase {
|
|||
Analyzer a = new MockAnalyzer(random());
|
||||
FreeTextSuggester sug = new FreeTextSuggester(a, a, 2, (byte) 0x20);
|
||||
sug.build(new InputArrayIterator(keys));
|
||||
assertEquals(2, sug.getCount());
|
||||
|
||||
for(int i=0;i<2;i++) {
|
||||
|
||||
|
@ -95,6 +96,7 @@ public class TestFreeTextSuggester extends LuceneTestCase {
|
|||
sug = new FreeTextSuggester(a, a, 2, (byte) 0x20);
|
||||
sug.load(is);
|
||||
is.close();
|
||||
assertEquals(2, sug.getCount());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -171,6 +173,7 @@ public class TestFreeTextSuggester extends LuceneTestCase {
|
|||
public boolean hasPayloads() {
|
||||
return false;
|
||||
}
|
||||
|
||||
});
|
||||
if (VERBOSE) {
|
||||
System.out.println(sug.sizeInBytes() + " bytes");
|
||||
|
|
|
@ -181,6 +181,7 @@ public class FSTCompletionTest extends LuceneTestCase {
|
|||
|
||||
FSTCompletionLookup lookup = new FSTCompletionLookup();
|
||||
lookup.build(new InputArrayIterator(input));
|
||||
assertEquals(input.size(), lookup.getCount());
|
||||
for (Input tf : input) {
|
||||
assertNotNull("Not found: " + tf.term.toString(), lookup.get(_TestUtil.bytesToCharSequence(tf.term, random())));
|
||||
assertEquals(tf.term.utf8ToString(), lookup.lookup(_TestUtil.bytesToCharSequence(tf.term, random()), true, 1).get(0).key.toString());
|
||||
|
|
|
@ -156,6 +156,7 @@ public class WFSTCompletionTest extends LuceneTestCase {
|
|||
WFSTCompletionLookup suggester = new WFSTCompletionLookup(false);
|
||||
suggester.build(new InputArrayIterator(keys));
|
||||
|
||||
assertEquals(numWords, suggester.getCount());
|
||||
Random random = new Random(random().nextLong());
|
||||
for (String prefix : allPrefixes) {
|
||||
final int topN = _TestUtil.nextInt(random, 1, 10);
|
||||
|
@ -215,6 +216,7 @@ public class WFSTCompletionTest extends LuceneTestCase {
|
|||
WFSTCompletionLookup suggester = new WFSTCompletionLookup(false);
|
||||
|
||||
suggester.build(new InputArrayIterator(new Input[0]));
|
||||
assertEquals(0, suggester.getCount());
|
||||
List<LookupResult> result = suggester.lookup("a", false, 20);
|
||||
assertTrue(result.isEmpty());
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue