LUCENE-3030: add BlockTreeTermsReader/Writer, a more efficient tree-structure for the terms dictionary

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1159905 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2011-08-20 19:20:45 +00:00
parent ea946d6d43
commit b7846ffaa1
83 changed files with 7001 additions and 992 deletions

View File

@ -209,7 +209,7 @@ public class FuzzyLikeThisQuery extends Query
AttributeSource atts = new AttributeSource();
MaxNonCompetitiveBoostAttribute maxBoostAtt =
atts.addAttribute(MaxNonCompetitiveBoostAttribute.class);
FuzzyTermsEnum fe = new FuzzyTermsEnum(MultiFields.getTerms(reader, startTerm.field()).iterator(), atts, startTerm, f.minSimilarity, f.prefixLength);
FuzzyTermsEnum fe = new FuzzyTermsEnum(MultiFields.getTerms(reader, startTerm.field()), atts, startTerm, f.minSimilarity, f.prefixLength);
//store the df so all variants use same idf
int df = reader.docFreq(startTerm);
int numVariants=0;

View File

@ -1,4 +1,4 @@
package org.apache.lucene.search;
package org.apache.lucene.index;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -20,14 +20,12 @@ package org.apache.lucene.search;
import java.io.IOException;
import java.util.Comparator;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.FilteredTermsEnum;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.ByteRunAutomaton;
import org.apache.lucene.util.automaton.SpecialOperations;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.Transition;
import org.apache.lucene.util.automaton.UTF32ToUTF8;
/**
* A FilteredTermsEnum that enumerates terms based upon what is accepted by a
@ -46,7 +44,7 @@ import org.apache.lucene.util.automaton.UTF32ToUTF8;
* </p>
* @lucene.experimental
*/
public class AutomatonTermsEnum extends FilteredTermsEnum {
class AutomatonTermsEnum extends FilteredTermsEnum {
// a tableized array-based form of the DFA
private final ByteRunAutomaton runAutomaton;
// common suffix of the automaton
@ -81,6 +79,7 @@ public class AutomatonTermsEnum extends FilteredTermsEnum {
super(tenum);
this.finite = compiled.finite;
this.runAutomaton = compiled.runAutomaton;
assert this.runAutomaton != null;
this.commonSuffixRef = compiled.commonSuffixRef;
this.allTransitions = compiled.sortedTransitions;
@ -110,6 +109,7 @@ public class AutomatonTermsEnum extends FilteredTermsEnum {
@Override
protected BytesRef nextSeekTerm(final BytesRef term) throws IOException {
//System.out.println("ATE.nextSeekTerm term=" + term);
if (term == null) {
assert seekBytesRef.length == 0;
// return the empty term, as its valid
@ -318,26 +318,4 @@ public class AutomatonTermsEnum extends FilteredTermsEnum {
}
return -1; /* all solutions exhausted */
}
/**
* immutable class with everything this enum needs.
*/
public static class CompiledAutomaton {
public final ByteRunAutomaton runAutomaton;
public final Transition[][] sortedTransitions;
public final BytesRef commonSuffixRef;
public final boolean finite;
public CompiledAutomaton(Automaton automaton, boolean finite) {
Automaton utf8 = new UTF32ToUTF8().convert(automaton);
runAutomaton = new ByteRunAutomaton(utf8, true);
sortedTransitions = utf8.getSortedTransitions();
this.finite = finite;
if (finite) {
commonSuffixRef = null;
} else {
commonSuffixRef = SpecialOperations.getCommonSuffixBytesRef(utf8);
}
}
}
}

View File

@ -398,6 +398,7 @@ class BufferedDeletesStream {
if (termsEnum.seekExact(term.bytes(), false)) {
DocsEnum docsEnum = termsEnum.docs(reader.getLiveDocs(), docs);
//System.out.println("BDS: got docsEnum=" + docsEnum);
if (docsEnum != null) {
while (true) {

View File

@ -17,13 +17,6 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import java.io.File;
import java.io.IOException;
import java.io.PrintStream;
@ -31,16 +24,25 @@ import java.text.NumberFormat;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.lucene.document.AbstractField; // for javadocs
import org.apache.lucene.document.Document;
import org.apache.lucene.index.codecs.BlockTreeTermsReader;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
import org.apache.lucene.index.codecs.PerDocValues;
import org.apache.lucene.index.values.IndexDocValues;
import org.apache.lucene.index.values.ValuesEnum;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.StringHelper;
@ -237,6 +239,8 @@ public class CheckIndex {
/** Exception thrown during term index test (null on success) */
public Throwable error = null;
public Map<String,BlockTreeTermsReader.Stats> blockTreeStats = null;
}
/**
@ -285,10 +289,19 @@ public class CheckIndex {
infoStream = null;
}
private boolean verbose;
/** Set infoStream where messages should go. If null, no
* messages are printed */
public void setInfoStream(PrintStream out) {
* messages are printed. If verbose is true then more
* details are printed. */
public void setInfoStream(PrintStream out, boolean verbose) {
infoStream = out;
this.verbose = verbose;
}
/** Set infoStream where messages should go. See {@link setInfoStream(PrintStream,boolean)}. */
public void setInfoStream(PrintStream out) {
setInfoStream(out, false);
}
private void msg(String msg) {
@ -872,6 +885,16 @@ public class CheckIndex {
}
}
final Terms fieldTerms = fields.terms(field);
if (fieldTerms instanceof BlockTreeTermsReader.FieldReader) {
final BlockTreeTermsReader.Stats stats = ((BlockTreeTermsReader.FieldReader) fieldTerms).computeStats();
assert stats != null;
if (status.blockTreeStats == null) {
status.blockTreeStats = new HashMap<String,BlockTreeTermsReader.Stats>();
}
status.blockTreeStats.put(field, stats);
}
if (sumTotalTermFreq != 0) {
final long v = fields.terms(field).getSumTotalTermFreq();
if (v != -1 && sumTotalTermFreq != v) {
@ -951,6 +974,13 @@ public class CheckIndex {
msg("OK [" + status.termCount + " terms; " + status.totFreq + " terms/docs pairs; " + status.totPos + " tokens]");
if (verbose && status.blockTreeStats != null && infoStream != null && status.termCount > 0) {
for(Map.Entry<String,BlockTreeTermsReader.Stats> ent : status.blockTreeStats.entrySet()) {
infoStream.println(" field \"" + ent.getKey() + "\":");
infoStream.println(" " + ent.getValue().toString().replace("\n", "\n "));
}
}
} catch (Throwable e) {
msg("ERROR: " + e);
status.error = e;
@ -1131,7 +1161,7 @@ public class CheckIndex {
<p>
Run it like this:
<pre>
java -ea:org.apache.lucene... org.apache.lucene.index.CheckIndex pathToIndex [-fix] [-segment X] [-segment Y]
java -ea:org.apache.lucene... org.apache.lucene.index.CheckIndex pathToIndex [-fix] [-verbose] [-segment X] [-segment Y]
</pre>
<ul>
<li><code>-fix</code>: actually write a new segments_N file, removing any problematic segments
@ -1161,6 +1191,7 @@ public class CheckIndex {
public static void main(String[] args) throws IOException, InterruptedException {
boolean doFix = false;
boolean verbose = false;
List<String> onlySegments = new ArrayList<String>();
String indexPath = null;
int i = 0;
@ -1168,6 +1199,9 @@ public class CheckIndex {
if (args[i].equals("-fix")) {
doFix = true;
i++;
} else if (args[i].equals("-verbose")) {
verbose = true;
i++;
} else if (args[i].equals("-segment")) {
if (i == args.length-1) {
System.out.println("ERROR: missing name for -segment option");
@ -1190,6 +1224,7 @@ public class CheckIndex {
System.out.println("\nUsage: java org.apache.lucene.index.CheckIndex pathToIndex [-fix] [-segment X] [-segment Y]\n" +
"\n" +
" -fix: actually write a new segments_N file, removing any problematic segments\n" +
" -verbose: print additional details\n" +
" -segment X: only check the specified segments. This can be specified multiple\n" +
" times, to check more than one segment, eg '-segment _2 -segment _a'.\n" +
" You can't use this with the -fix option\n" +
@ -1231,7 +1266,7 @@ public class CheckIndex {
}
CheckIndex checker = new CheckIndex(dir);
checker.setInfoStream(System.out);
checker.setInfoStream(System.out, verbose);
Status result = checker.checkIndex(onlySegments);
if (result.missingSegments) {

View File

@ -26,6 +26,7 @@ import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.codecs.FieldsConsumer;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CollectionUtil;
import org.apache.lucene.util.IOUtils;
final class FreqProxTermsWriter extends TermsHashConsumer {
@ -58,6 +59,8 @@ final class FreqProxTermsWriter extends TermsHashConsumer {
final FieldsConsumer consumer = state.segmentCodecs.codec().fieldsConsumer(state);
boolean success = false;
try {
TermsHash termsHash = null;
@ -100,8 +103,9 @@ final class FreqProxTermsWriter extends TermsHashConsumer {
if (termsHash != null) {
termsHash.reset();
}
success = true;
} finally {
consumer.close();
IOUtils.closeSafely(!success, consumer);
}
}

View File

@ -17,13 +17,15 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.List;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.ReaderUtil;
import java.io.IOException;
import java.util.List;
import java.util.ArrayList;
import java.util.Comparator;
import org.apache.lucene.util.automaton.CompiledAutomaton;
/**
* Exposes flex API, merged from flex API of
@ -58,6 +60,23 @@ public final class MultiTerms extends Terms {
termComp = _termComp;
}
@Override
public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
final List<MultiTermsEnum.TermsEnumIndex> termsEnums = new ArrayList<MultiTermsEnum.TermsEnumIndex>();
for(int i=0;i<subs.length;i++) {
final TermsEnum termsEnum = subs[i].intersect(compiled, startTerm);
if (termsEnum != null) {
termsEnums.add(new MultiTermsEnum.TermsEnumIndex(termsEnum, i));
}
}
if (termsEnums.size() > 0) {
return new MultiTermsEnum(subSlices).reset(termsEnums.toArray(MultiTermsEnum.TermsEnumIndex.EMPTY_ARRAY));
} else {
return TermsEnum.EMPTY;
}
}
@Override
public TermsEnum iterator() throws IOException {

View File

@ -562,12 +562,14 @@ final class SegmentMerger {
}
codec = segmentWriteState.segmentCodecs.codec();
final FieldsConsumer consumer = codec.fieldsConsumer(segmentWriteState);
boolean success = false;
try {
consumer.merge(mergeState,
new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY)));
success = true;
} finally {
consumer.close();
IOUtils.closeSafely(!success, consumer);
}
}

View File

@ -19,9 +19,11 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.Comparator;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.automaton.CompiledAutomaton;
/**
* Access to the terms in a specific field. See {@link Fields}.
@ -38,6 +40,39 @@ public abstract class Terms {
* terms. This method will not return null.*/
public abstract TermsEnum iterator() throws IOException;
/** Returns a TermsEnum that iterates over all terms that
* are accepted by the provided {@link
* CompiledAutomaton}. If the <code>startTerm</code> is
* provided then the returned enum will only accept terms
* > <code>startTerm</code>, but you still must call
* next() first to get to the first term. Note that the
* provided <code>startTerm</code> must be accepted by
* the automaton.
*
* <p><b>NOTE</b>: the returned TermsEnum cannot
* seek</p>. */
public TermsEnum intersect(CompiledAutomaton compiled, final BytesRef startTerm) throws IOException {
// TODO: eventually we could support seekCeil/Exact on
// the returned enum, instead of only being able to seek
// at the start
if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
}
if (startTerm == null) {
return new AutomatonTermsEnum(iterator(), compiled);
} else {
return new AutomatonTermsEnum(iterator(), compiled) {
@Override
protected BytesRef nextSeekTerm(BytesRef term) throws IOException {
if (term == null) {
term = startTerm;
}
return super.nextSeekTerm(term);
}
};
}
}
/** Return the BytesRef Comparator used to sort terms
* provided by the iterator. This method may return null
* if there are no terms. This method may be invoked

View File

@ -29,10 +29,8 @@ public class BlockTermState extends OrdTermState {
public int docFreq; // how many docs have this term
public long totalTermFreq; // total number of occurrences of this term
public int termCount; // term ord are in the current block
public long blockFilePointer; // fp into the terms dict primary file (_X.tib) that holds this term
public int blockTermCount; // how many terms in current block
public int termBlockOrd; // the term's ord in the current block
public long blockFilePointer; // fp into the terms dict primary file (_X.tim) that holds this term
@Override
public void copyFrom(TermState _other) {
@ -41,7 +39,7 @@ public class BlockTermState extends OrdTermState {
super.copyFrom(_other);
docFreq = other.docFreq;
totalTermFreq = other.totalTermFreq;
termCount = other.termCount;
termBlockOrd = other.termBlockOrd;
blockFilePointer = other.blockFilePointer;
// NOTE: don't copy blockTermCount;
@ -51,6 +49,6 @@ public class BlockTermState extends OrdTermState {
@Override
public String toString() {
return "ord=" + ord + " docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termCount=" + termCount + " blockFP=" + blockFilePointer;
return "docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termBlockOrd=" + termBlockOrd + " blockFP=" + blockFilePointer;
}
}

View File

@ -106,7 +106,7 @@ public class BlockTermsReader extends FieldsProducer {
}
}
//private String segment;
// private String segment;
public BlockTermsReader(TermsIndexReaderBase indexReader, Directory dir, FieldInfos fieldInfos, String segment, PostingsReaderBase postingsReader, IOContext context,
int termsCacheSize, int codecId)
@ -115,7 +115,7 @@ public class BlockTermsReader extends FieldsProducer {
this.postingsReader = postingsReader;
termsCache = new DoubleBarrelLRUCache<FieldAndTerm,BlockTermState>(termsCacheSize);
//this.segment = segment;
// this.segment = segment;
in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTermsWriter.TERMS_EXTENSION),
context);
@ -321,6 +321,9 @@ public class BlockTermsReader extends FieldsProducer {
/* Common prefix used for all terms in this block. */
private int termBlockPrefix;
/* How many terms in current block */
private int blockTermCount;
private byte[] docFreqBytes;
private final ByteArrayDataInput freqReader = new ByteArrayDataInput();
private int metaDataUpto;
@ -358,16 +361,14 @@ public class BlockTermsReader extends FieldsProducer {
throw new IllegalStateException("terms index was not loaded");
}
/*
System.out.println("BTR.seek seg=" + segment + " target=" + fieldInfo.name + ":" + target.utf8ToString() + " " + target + " current=" + term().utf8ToString() + " " + term() + " useCache=" + useCache + " indexIsCurrent=" + indexIsCurrent + " didIndexNext=" + didIndexNext + " seekPending=" + seekPending + " divisor=" + indexReader.getDivisor() + " this=" + this);
//System.out.println("BTR.seek seg=" + segment + " target=" + fieldInfo.name + ":" + target.utf8ToString() + " " + target + " current=" + term().utf8ToString() + " " + term() + " useCache=" + useCache + " indexIsCurrent=" + indexIsCurrent + " didIndexNext=" + didIndexNext + " seekPending=" + seekPending + " divisor=" + indexReader.getDivisor() + " this=" + this);
if (didIndexNext) {
if (nextIndexTerm == null) {
System.out.println(" nextIndexTerm=null");
//System.out.println(" nextIndexTerm=null");
} else {
System.out.println(" nextIndexTerm=" + nextIndexTerm.utf8ToString());
//System.out.println(" nextIndexTerm=" + nextIndexTerm.utf8ToString());
}
}
*/
// Check cache
if (useCache) {
@ -444,7 +445,7 @@ public class BlockTermsReader extends FieldsProducer {
//System.out.println(" seek: term=" + term.utf8ToString());
} else {
//System.out.println(" skip seek");
if (state.termCount == state.blockTermCount && !nextBlock()) {
if (state.termBlockOrd == blockTermCount && !nextBlock()) {
indexIsCurrent = false;
return SeekStatus.END;
}
@ -480,9 +481,9 @@ public class BlockTermsReader extends FieldsProducer {
// but it could be in next block. We
// must scan to end-of-block to set common
// prefix for next block:
if (state.termCount < state.blockTermCount) {
while(state.termCount < state.blockTermCount-1) {
state.termCount++;
if (state.termBlockOrd < blockTermCount) {
while(state.termBlockOrd < blockTermCount-1) {
state.termBlockOrd++;
state.ord++;
termSuffixesReader.skipBytes(termSuffixesReader.readVInt());
}
@ -505,7 +506,7 @@ public class BlockTermsReader extends FieldsProducer {
// Target's prefix is before the common prefix
// of this block, so we position to start of
// block and return NOT_FOUND:
assert state.termCount == 0;
assert state.termBlockOrd == 0;
final int suffix = termSuffixesReader.readVInt();
term.length = termBlockPrefix + suffix;
@ -523,7 +524,7 @@ public class BlockTermsReader extends FieldsProducer {
// Test every term in this block
while (true) {
state.termCount++;
state.termBlockOrd++;
state.ord++;
final int suffix = termSuffixesReader.readVInt();
@ -581,7 +582,7 @@ public class BlockTermsReader extends FieldsProducer {
}
}
if (state.termCount == state.blockTermCount) {
if (state.termBlockOrd == blockTermCount) {
// Must pre-fill term for next block's common prefix
term.length = termBlockPrefix + suffix;
if (term.bytes.length < term.length) {
@ -613,7 +614,7 @@ public class BlockTermsReader extends FieldsProducer {
@Override
public BytesRef next() throws IOException {
//System.out.println("BTR.next() seekPending=" + seekPending + " pendingSeekCount=" + state.termCount);
//System.out.println("BTR.next() seekPending=" + seekPending + " pendingSeekCount=" + state.termBlockOrd);
// If seek was previously called and the term was cached,
// usually caller is just going to pull a D/&PEnum or get
@ -623,7 +624,7 @@ public class BlockTermsReader extends FieldsProducer {
if (seekPending) {
assert !indexIsCurrent;
in.seek(state.blockFilePointer);
final int pendingSeekCount = state.termCount;
final int pendingSeekCount = state.termBlockOrd;
boolean result = nextBlock();
final long savOrd = state.ord;
@ -633,7 +634,7 @@ public class BlockTermsReader extends FieldsProducer {
// on a real term:
assert result;
while(state.termCount < pendingSeekCount) {
while(state.termBlockOrd < pendingSeekCount) {
BytesRef nextResult = _next();
assert nextResult != null;
}
@ -647,8 +648,8 @@ public class BlockTermsReader extends FieldsProducer {
metadata, ie docFreq, totalTermFreq or pulls a D/&PEnum, we then (lazily)
decode all metadata up to the current term. */
private BytesRef _next() throws IOException {
//System.out.println("BTR._next seg=" + segment + " this=" + this + " termCount=" + state.termCount + " (vs " + state.blockTermCount + ")");
if (state.termCount == state.blockTermCount && !nextBlock()) {
//System.out.println("BTR._next seg=" + segment + " this=" + this + " termCount=" + state.termBlockOrd + " (vs " + blockTermCount + ")");
if (state.termBlockOrd == blockTermCount && !nextBlock()) {
//System.out.println(" eof");
indexIsCurrent = false;
return null;
@ -663,12 +664,12 @@ public class BlockTermsReader extends FieldsProducer {
term.grow(term.length);
}
termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
state.termCount++;
state.termBlockOrd++;
// NOTE: meaningless in the non-ord case
state.ord++;
//System.out.println(" return term=" + fieldInfo.name + ":" + term.utf8ToString() + " " + term);
//System.out.println(" return term=" + fieldInfo.name + ":" + term.utf8ToString() + " " + term + " tbOrd=" + state.termBlockOrd);
return term;
}
@ -695,9 +696,10 @@ public class BlockTermsReader extends FieldsProducer {
public DocsEnum docs(Bits liveDocs, DocsEnum reuse) throws IOException {
//System.out.println("BTR.docs this=" + this);
decodeMetaData();
//System.out.println(" state.docFreq=" + state.docFreq);
//System.out.println("BTR.docs: state.docFreq=" + state.docFreq);
final DocsEnum docsEnum = postingsReader.docs(fieldInfo, state, liveDocs, reuse);
assert docsEnum != null;
//System.out.println("BTR.docs: return docsEnum=" + docsEnum);
return docsEnum;
}
@ -716,7 +718,7 @@ public class BlockTermsReader extends FieldsProducer {
@Override
public void seekExact(BytesRef target, TermState otherState) throws IOException {
//System.out.println("BTR.seek termState target=" + target.utf8ToString() + " " + target + " this=" + this);
//System.out.println("BTR.seekExact termState target=" + target.utf8ToString() + " " + target + " this=" + this);
assert otherState != null && otherState instanceof BlockTermState;
assert !doOrd || ((BlockTermState) otherState).ord < numTerms;
state.copyFrom(otherState);
@ -800,9 +802,9 @@ public class BlockTermsReader extends FieldsProducer {
//System.out.println("BTR.nextBlock() fp=" + in.getFilePointer() + " this=" + this);
state.blockFilePointer = in.getFilePointer();
state.blockTermCount = in.readVInt();
//System.out.println(" blockTermCount=" + state.blockTermCount);
if (state.blockTermCount == 0) {
blockTermCount = in.readVInt();
//System.out.println(" blockTermCount=" + blockTermCount);
if (blockTermCount == 0) {
return false;
}
termBlockPrefix = in.readVInt();
@ -826,7 +828,7 @@ public class BlockTermsReader extends FieldsProducer {
freqReader.reset(docFreqBytes, 0, len);
metaDataUpto = 0;
state.termCount = 0;
state.termBlockOrd = 0;
postingsReader.readTermsBlock(in, fieldInfo, state);
@ -838,7 +840,7 @@ public class BlockTermsReader extends FieldsProducer {
}
private void decodeMetaData() throws IOException {
//System.out.println("BTR.decodeMetadata mdUpto=" + metaDataUpto + " vs termCount=" + state.termCount + " state=" + state);
//System.out.println("BTR.decodeMetadata mdUpto=" + metaDataUpto + " vs termCount=" + state.termBlockOrd + " state=" + state);
if (!seekPending) {
// TODO: cutover to random-access API
// here.... really stupid that we have to decode N
@ -846,10 +848,10 @@ public class BlockTermsReader extends FieldsProducer {
// that we really need...
// lazily catch up on metadata decode:
final int limit = state.termCount;
final int limit = state.termBlockOrd;
// We must set/incr state.termCount because
// postings impl can look at this
state.termCount = metaDataUpto;
state.termBlockOrd = metaDataUpto;
// TODO: better API would be "jump straight to term=N"???
while (metaDataUpto < limit) {
//System.out.println(" decode mdUpto=" + metaDataUpto);
@ -870,9 +872,9 @@ public class BlockTermsReader extends FieldsProducer {
postingsReader.nextTerm(fieldInfo, state);
metaDataUpto++;
state.termCount++;
state.termBlockOrd++;
}
//} else {
} else {
//System.out.println(" skip! seekPending");
}
}

View File

@ -66,7 +66,7 @@ public class BlockTermsWriter extends FieldsConsumer {
private final TermsIndexWriterBase termsIndexWriter;
private final List<TermsWriter> fields = new ArrayList<TermsWriter>();
//private final String segment;
// private final String segment;
public BlockTermsWriter(TermsIndexWriterBase termsIndexWriter,
SegmentWriteState state, PostingsWriterBase postingsWriter)
@ -80,7 +80,7 @@ public class BlockTermsWriter extends FieldsConsumer {
writeHeader(out);
currentField = null;
this.postingsWriter = postingsWriter;
//segment = state.segmentName;
// segment = state.segmentName;
//System.out.println("BTW.init seg=" + state.segmentName);
@ -188,7 +188,7 @@ public class BlockTermsWriter extends FieldsConsumer {
@Override
public PostingsConsumer startTerm(BytesRef text) throws IOException {
//System.out.println("BTW.startTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment);
//System.out.println("BTW: startTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment);
postingsWriter.startTerm();
return postingsWriter;
}
@ -199,7 +199,7 @@ public class BlockTermsWriter extends FieldsConsumer {
public void finishTerm(BytesRef text, TermStats stats) throws IOException {
assert stats.docFreq > 0;
//System.out.println("BTW.finishTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment + " df=" + stats.docFreq);
//System.out.println("BTW: finishTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment + " df=" + stats.docFreq);
final boolean isIndexTerm = fieldIndexWriter.checkIndexTerm(text, stats);
@ -308,7 +308,7 @@ public class BlockTermsWriter extends FieldsConsumer {
bytesWriter.writeTo(out);
bytesWriter.reset();
postingsWriter.flushTermsBlock();
postingsWriter.flushTermsBlock(pendingCount, pendingCount);
lastPrevTerm.copy(pendingTerms[pendingCount-1].term);
pendingCount = 0;
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,943 @@
package org.apache.lucene.index.codecs;
/**
* 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.FileOutputStream;
import java.io.IOException;
import java.io.OutputStreamWriter;
import java.io.Writer;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.List;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.fst.Builder;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.BytesRefFSTEnum;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.NoOutputs;
import org.apache.lucene.util.fst.Util;
/*
TODO:
- Currently there is a one-to-one mapping of indexed
term to term block, but we could decouple the two, ie,
put more terms into the index than there are blocks.
The index would take up more RAM but then it'd be able
to avoid seeking more often and could make PK/FuzzyQ
faster if the additional indexed terms could store
the offset into the terms block.
- The blocks are not written in true depth-first
order, meaning if you just next() the file pointer will
sometimes jump backwards. For example, block foo* will
be written before block f* because it finished before.
This could possibly hurt performance if the terms dict is
not hot, since OSs anticipate sequential file access. We
could fix the writer to re-order the blocks as a 2nd
pass.
- Each block encodes the term suffixes packed
sequentially using a separate vInt per term, which is
1) wasteful and 2) slow (must linear scan to find a
particular suffix). We should instead 1) make
random-access array so we can directly access the Nth
suffix, and 2) bulk-encode this array using bulk int[]
codecs; then at search time we can binary search when
we seek a particular term.
/**
* Writes terms dict and index, block-encoding (column
* stride) each term's metadata for each set of terms
* between two index terms.
*
* @lucene.experimental
*/
/** See {@link BlockTreeTermsReader}.
*
* @lucene.experimental
*/
public class BlockTreeTermsWriter extends FieldsConsumer {
public final static int DEFAULT_MIN_BLOCK_SIZE = 25;
public final static int DEFAULT_MAX_BLOCK_SIZE = 48;
//public final static boolean DEBUG = false;
public final static boolean SAVE_DOT_FILES = false;
static final int OUTPUT_FLAGS_NUM_BITS = 2;
static final int OUTPUT_FLAGS_MASK = 0x3;
static final int OUTPUT_FLAG_IS_FLOOR = 0x1;
static final int OUTPUT_FLAG_HAS_TERMS = 0x2;
final static String CODEC_NAME = "BLOCK_TREE_TERMS_DICT";
// Initial format
public static final int VERSION_START = 0;
public static final int VERSION_CURRENT = VERSION_START;
/** Extension of terms file */
static final String TERMS_EXTENSION = "tim";
static final String TERMS_INDEX_EXTENSION = "tip";
protected final IndexOutput out;
private final IndexOutput indexOut;
final int minItemsInBlock;
final int maxItemsInBlock;
final PostingsWriterBase postingsWriter;
final FieldInfos fieldInfos;
FieldInfo currentField;
private final List<TermsWriter> fields = new ArrayList<TermsWriter>();
// private final String segment;
/** Create a new writer. The number of items (terms or
* sub-blocks) per block will aim to be between
* minItemsPerBlock and maxItemsPerBlock, though in some
* cases the blocks may be smaller than the min. */
public BlockTreeTermsWriter(
SegmentWriteState state,
PostingsWriterBase postingsWriter,
int minItemsInBlock,
int maxItemsInBlock)
throws IOException
{
if (minItemsInBlock <= 1) {
throw new IllegalArgumentException("minItemsInBlock must be >= 2; got " + minItemsInBlock);
}
if (maxItemsInBlock <= 0) {
throw new IllegalArgumentException("maxItemsInBlock must be >= 1; got " + maxItemsInBlock);
}
if (minItemsInBlock > maxItemsInBlock) {
throw new IllegalArgumentException("maxItemsInBlock must be >= minItemsInBlock; got maxItemsInBlock=" + maxItemsInBlock + " minItemsInBlock=" + minItemsInBlock);
}
if (2*(minItemsInBlock-1) > maxItemsInBlock) {
throw new IllegalArgumentException("maxItemsInBlock must be at least 2*(minItemsInBlock-1); got maxItemsInBlock=" + maxItemsInBlock + " minItemsInBlock=" + minItemsInBlock);
}
final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_EXTENSION);
out = state.directory.createOutput(termsFileName, state.context);
boolean success = false;
IndexOutput indexOut = null;
try {
fieldInfos = state.fieldInfos;
this.minItemsInBlock = minItemsInBlock;
this.maxItemsInBlock = maxItemsInBlock;
writeHeader(out);
//DEBUG = state.segmentName.equals("_4a");
final String termsIndexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION);
indexOut = state.directory.createOutput(termsIndexFileName, state.context);
writeIndexHeader(indexOut);
currentField = null;
this.postingsWriter = postingsWriter;
// segment = state.segmentName;
// System.out.println("BTW.init seg=" + state.segmentName);
postingsWriter.start(out); // have consumer write its format/header
success = true;
} finally {
if (!success) {
IOUtils.closeSafely(true, out, indexOut);
}
}
this.indexOut = indexOut;
}
protected void writeHeader(IndexOutput out) throws IOException {
CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
out.writeLong(0); // leave space for end index pointer
}
protected void writeIndexHeader(IndexOutput out) throws IOException {
CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
out.writeLong(0); // leave space for end index pointer
}
protected void writeTrailer(long dirStart) throws IOException {
out.seek(CodecUtil.headerLength(CODEC_NAME));
out.writeLong(dirStart);
}
protected void writeIndexTrailer(long dirStart) throws IOException {
indexOut.seek(CodecUtil.headerLength(CODEC_NAME));
indexOut.writeLong(dirStart);
}
@Override
public TermsConsumer addField(FieldInfo field) throws IOException {
//DEBUG = field.name.equals("id");
//if (DEBUG) System.out.println("\nBTTW.addField seg=" + segment + " field=" + field.name);
assert currentField == null || currentField.name.compareTo(field.name) < 0;
currentField = field;
final TermsWriter terms = new TermsWriter(field);
fields.add(terms);
return terms;
}
static long encodeOutput(long fp, boolean hasTerms, boolean isFloor) {
assert fp < (1L << 62);
return (fp << 2) | (hasTerms ? OUTPUT_FLAG_HAS_TERMS : 0) | (isFloor ? OUTPUT_FLAG_IS_FLOOR : 0);
}
private static class PendingEntry {
public final boolean isTerm;
protected PendingEntry(boolean isTerm) {
this.isTerm = isTerm;
}
}
private static final class PendingTerm extends PendingEntry {
public final BytesRef term;
public final TermStats stats;
public PendingTerm(BytesRef term, TermStats stats) {
super(true);
this.term = term;
this.stats = stats;
}
@Override
public String toString() {
return term.utf8ToString();
}
}
private static final class PendingBlock extends PendingEntry {
public final BytesRef prefix;
public final long fp;
public FST<BytesRef> index;
public List<FST<BytesRef>> subIndices;
public final boolean hasTerms;
public final boolean isFloor;
public final int floorLeadByte;
public PendingBlock(BytesRef prefix, long fp, boolean hasTerms, boolean isFloor, int floorLeadByte, List<FST<BytesRef>> subIndices) {
super(false);
this.prefix = prefix;
this.fp = fp;
this.hasTerms = hasTerms;
this.isFloor = isFloor;
this.floorLeadByte = floorLeadByte;
this.subIndices = subIndices;
}
@Override
public String toString() {
return "BLOCK: " + prefix.utf8ToString();
}
public void compileIndex(List<PendingBlock> floorBlocks, RAMOutputStream scratchBytes) throws IOException {
assert (isFloor && floorBlocks != null && floorBlocks.size() != 0) || (!isFloor && floorBlocks == null): "isFloor=" + isFloor + " floorBlocks=" + floorBlocks;
assert scratchBytes.getFilePointer() == 0;
// TODO: try writing the leading vLong in MSB order
// (opposite of what Lucene does today), for better
// outputs sharing in the FST
scratchBytes.writeVLong(encodeOutput(fp, hasTerms, isFloor));
if (isFloor) {
scratchBytes.writeVInt(floorBlocks.size());
for (PendingBlock sub : floorBlocks) {
assert sub.floorLeadByte != -1;
//if (DEBUG) {
// System.out.println(" write floorLeadByte=" + Integer.toHexString(sub.floorLeadByte&0xff));
//}
scratchBytes.writeByte((byte) sub.floorLeadByte);
assert sub.fp > fp;
scratchBytes.writeVLong((sub.fp - fp) << 1 | (sub.hasTerms ? 1 : 0));
}
}
final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
final Builder<BytesRef> indexBuilder = new Builder<BytesRef>(FST.INPUT_TYPE.BYTE1,
0, 0, true, false, Integer.MAX_VALUE,
outputs, null);
//if (DEBUG) {
// System.out.println(" compile index for prefix=" + prefix);
//}
//indexBuilder.DEBUG = false;
final byte[] bytes = new byte[(int) scratchBytes.getFilePointer()];
assert bytes.length > 0;
scratchBytes.writeTo(bytes, 0);
indexBuilder.add(prefix, new BytesRef(bytes, 0, bytes.length));
scratchBytes.reset();
// Copy over index for all sub-blocks
if (subIndices != null) {
for(FST<BytesRef> subIndex : subIndices) {
append(indexBuilder, subIndex);
}
}
if (floorBlocks != null) {
for (PendingBlock sub : floorBlocks) {
if (sub.subIndices != null) {
for(FST<BytesRef> subIndex : sub.subIndices) {
append(indexBuilder, subIndex);
}
}
sub.subIndices = null;
}
}
index = indexBuilder.finish();
subIndices = null;
/*
Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"));
Util.toDot(index, w, false, false);
System.out.println("SAVED to out.dot");
w.close();
*/
}
// TODO: maybe we could add bulk-add method to
// Builder? Takes FST and unions it w/ current
// FST.
private void append(Builder<BytesRef> builder, FST<BytesRef> subIndex) throws IOException {
final BytesRefFSTEnum<BytesRef> subIndexEnum = new BytesRefFSTEnum<BytesRef>(subIndex);
BytesRefFSTEnum.InputOutput<BytesRef> indexEnt;
while((indexEnt = subIndexEnum.next()) != null) {
//if (DEBUG) {
// System.out.println(" add sub=" + indexEnt.input + " " + indexEnt.input + " output=" + indexEnt.output);
//}
builder.add(indexEnt.input, indexEnt.output);
}
}
}
final RAMOutputStream scratchBytes = new RAMOutputStream();
class TermsWriter extends TermsConsumer {
private final FieldInfo fieldInfo;
private long numTerms;
long sumTotalTermFreq;
long sumDocFreq;
long indexStartFP;
// Used only to partition terms into the block tree; we
// don't pull an FST from this builder:
private final NoOutputs noOutputs;
private final Builder<Object> blockBuilder;
// PendingTerm or PendingBlock:
private final List<PendingEntry> pending = new ArrayList<PendingEntry>();
// Index into pending of most recently written block
private int lastBlockIndex = -1;
// Re-used when segmenting a too-large block into floor
// blocks:
private int[] subBytes = new int[10];
private int[] subTermCounts = new int[10];
private int[] subTermCountSums = new int[10];
private int[] subSubCounts = new int[10];
// This class assigns terms to blocks "naturally", ie,
// according to the number of terms under a given prefix
// that we encounter:
private class FindBlocks extends Builder.FreezeTail<Object> {
@Override
public void freeze(final Builder.UnCompiledNode<Object>[] frontier, int prefixLenPlus1, final IntsRef lastInput) throws IOException {
//if (DEBUG) System.out.println(" freeze prefixLenPlus1=" + prefixLenPlus1);
for(int idx=lastInput.length; idx >= prefixLenPlus1; idx--) {
final Builder.UnCompiledNode<Object> node = frontier[idx];
long totCount = 0;
if (node.isFinal) {
totCount++;
}
for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
@SuppressWarnings("unchecked") final Builder.UnCompiledNode<Object> target = (Builder.UnCompiledNode<Object>) node.arcs[arcIdx].target;
totCount += target.inputCount;
target.clear();
node.arcs[arcIdx].target = null;
}
node.numArcs = 0;
if (totCount >= minItemsInBlock || idx == 0) {
// We are on a prefix node that has enough
// entries (terms or sub-blocks) under it to let
// us write a new block or multiple blocks (main
// block + follow on floor blocks):
//if (DEBUG) {
// if (totCount < minItemsInBlock && idx != 0) {
// System.out.println(" force block has terms");
// }
//}
writeBlocks(lastInput, idx, (int) totCount);
node.inputCount = 1;
} else {
// stragglers! carry count upwards
node.inputCount = totCount;
}
frontier[idx] = new Builder.UnCompiledNode<Object>(blockBuilder, idx);
}
}
}
// Write the top count entries on the pending stack as
// one or more blocks. Returns how many blocks were
// written. If the entry count is <= maxItemsPerBlock
// we just write a single block; else we break into
// primary (initial) block and then one or more
// following floor blocks:
void writeBlocks(IntsRef prevTerm, int prefixLength, int count) throws IOException {
if (prefixLength == 0 || count <= maxItemsInBlock) {
// Easy case: not floor block. Eg, prefix is "foo",
// and we found 30 terms/sub-blocks starting w/ that
// prefix, and minItemsInBlock <= 30 <=
// maxItemsInBlock.
final PendingBlock nonFloorBlock = writeBlock(prevTerm, prefixLength, prefixLength, count, count, 0, false, -1, true);
nonFloorBlock.compileIndex(null, scratchBytes);
pending.add(nonFloorBlock);
} else {
// Floor block case. Eg, prefix is "foo" but we
// have 100 terms/sub-blocks starting w/ that
// prefix. We segment the entries into a primary
// block and following floor blocks using the first
// label in the suffix to assign to floor blocks.
// TODO: we could store min & max suffix start byte
// in each block, to make floor blocks authoritative
//if (DEBUG) {
// final BytesRef prefix = new BytesRef(prefixLength);
// for(int m=0;m<prefixLength;m++) {
// prefix.bytes[m] = (byte) prevTerm.ints[m];
// }
// prefix.length = prefixLength;
// //System.out.println("\nWBS count=" + count + " prefix=" + prefix.utf8ToString() + " " + prefix);
// System.out.println("writeBlocks: prefix=" + prefix + " " + prefix + " count=" + count + " pending.size()=" + pending.size());
//}
//System.out.println("\nwbs count=" + count);
final int savLabel = prevTerm.ints[prevTerm.offset + prefixLength];
// Count up how many items fall under
// each unique label after the prefix.
// TODO: this is wasteful since the builder had
// already done this (partitioned these sub-terms
// according to their leading prefix byte)
final List<PendingEntry> slice = pending.subList(pending.size()-count, pending.size());
int lastSuffixLeadLabel = -1;
int termCount = 0;
int subCount = 0;
int numSubs = 0;
for(PendingEntry ent : slice) {
// First byte in the suffix of this term
final int suffixLeadLabel;
if (ent.isTerm) {
PendingTerm term = (PendingTerm) ent;
if (term.term.length == prefixLength) {
// Suffix is 0, ie prefix 'foo' and term is
// 'foo' so the term has empty string suffix
// in this block
assert lastSuffixLeadLabel == -1;
assert numSubs == 0;
suffixLeadLabel = -1;
} else {
suffixLeadLabel = term.term.bytes[term.term.offset + prefixLength] & 0xff;
}
} else {
PendingBlock block = (PendingBlock) ent;
assert block.prefix.length > prefixLength;
suffixLeadLabel = block.prefix.bytes[block.prefix.offset + prefixLength] & 0xff;
}
if (suffixLeadLabel != lastSuffixLeadLabel && (termCount + subCount) != 0) {
if (subBytes.length == numSubs) {
subBytes = ArrayUtil.grow(subBytes);
subTermCounts = ArrayUtil.grow(subTermCounts);
subSubCounts = ArrayUtil.grow(subSubCounts);
}
subBytes[numSubs] = lastSuffixLeadLabel;
lastSuffixLeadLabel = suffixLeadLabel;
subTermCounts[numSubs] = termCount;
subSubCounts[numSubs] = subCount;
/*
if (suffixLeadLabel == -1) {
System.out.println(" sub " + -1 + " termCount=" + termCount + " subCount=" + subCount);
} else {
System.out.println(" sub " + Integer.toHexString(suffixLeadLabel) + " termCount=" + termCount + " subCount=" + subCount);
}
*/
termCount = subCount = 0;
numSubs++;
}
if (ent.isTerm) {
termCount++;
} else {
subCount++;
}
}
if (subBytes.length == numSubs) {
subBytes = ArrayUtil.grow(subBytes);
subTermCounts = ArrayUtil.grow(subTermCounts);
subSubCounts = ArrayUtil.grow(subSubCounts);
}
subBytes[numSubs] = lastSuffixLeadLabel;
subTermCounts[numSubs] = termCount;
subSubCounts[numSubs] = subCount;
numSubs++;
/*
if (lastSuffixLeadLabel == -1) {
System.out.println(" sub " + -1 + " termCount=" + termCount + " subCount=" + subCount);
} else {
System.out.println(" sub " + Integer.toHexString(lastSuffixLeadLabel) + " termCount=" + termCount + " subCount=" + subCount);
}
*/
if (subTermCountSums.length < numSubs) {
subTermCountSums = ArrayUtil.grow(subTermCountSums, numSubs);
}
// Roll up (backwards) the termCounts; postings impl
// needs this to know where to pull the term slice
// from its pending terms stack:
int sum = 0;
for(int idx=numSubs-1;idx>=0;idx--) {
sum += subTermCounts[idx];
subTermCountSums[idx] = sum;
}
// TODO: make a better segmenter? It'd have to
// absorb the too-small end blocks backwards into
// the previous blocks
// Naive greedy segmentation; this is not always
// best (it can produce a too-small block as the
// last block):
int pendingCount = 0;
int startLabel = subBytes[0];
int curStart = count;
subCount = 0;
final List<PendingBlock> floorBlocks = new ArrayList<PendingBlock>();
PendingBlock firstBlock = null;
for(int sub=0;sub<numSubs;sub++) {
pendingCount += subTermCounts[sub] + subSubCounts[sub];
//System.out.println(" " + (subTermCounts[sub] + subSubCounts[sub]));
subCount++;
// Greedily make a floor block as soon as we've
// crossed the min count
if (pendingCount >= minItemsInBlock) {
final int curPrefixLength;
if (startLabel == -1) {
curPrefixLength = prefixLength;
} else {
curPrefixLength = 1+prefixLength;
// floor term:
prevTerm.ints[prevTerm.offset + prefixLength] = startLabel;
}
//System.out.println(" " + subCount + " subs");
final PendingBlock floorBlock = writeBlock(prevTerm, prefixLength, curPrefixLength, curStart, pendingCount, subTermCountSums[1+sub], true, startLabel, curStart == pendingCount);
if (firstBlock == null) {
firstBlock = floorBlock;
} else {
floorBlocks.add(floorBlock);
}
curStart -= pendingCount;
//System.out.println(" = " + pendingCount);
pendingCount = 0;
assert minItemsInBlock == 1 || subCount > 1: "minItemsInBlock=" + minItemsInBlock + " subCount=" + subCount + " sub=" + sub + " of " + numSubs + " subTermCount=" + subTermCountSums[sub] + " subSubCount=" + subSubCounts[sub] + " depth=" + prefixLength;
subCount = 0;
startLabel = subBytes[sub+1];
if (curStart == 0) {
break;
}
if (curStart <= maxItemsInBlock) {
// remainder is small enough to fit into a
// block. NOTE that this may be too small (<
// minItemsInBlock); need a true segmenter
// here
assert startLabel != -1;
assert firstBlock != null;
prevTerm.ints[prevTerm.offset + prefixLength] = startLabel;
//System.out.println(" final " + (numSubs-sub-1) + " subs");
/*
for(sub++;sub < numSubs;sub++) {
System.out.println(" " + (subTermCounts[sub] + subSubCounts[sub]));
}
System.out.println(" = " + curStart);
if (curStart < minItemsInBlock) {
System.out.println(" **");
}
*/
floorBlocks.add(writeBlock(prevTerm, prefixLength, prefixLength+1, curStart, curStart, 0, true, startLabel, true));
break;
}
}
}
prevTerm.ints[prevTerm.offset + prefixLength] = savLabel;
assert firstBlock != null;
firstBlock.compileIndex(floorBlocks, scratchBytes);
pending.add(firstBlock);
//if (DEBUG) System.out.println(" done pending.size()=" + pending.size());
}
lastBlockIndex = pending.size()-1;
}
// for debugging
private String toString(BytesRef b) {
try {
return b.utf8ToString() + " " + b;
} catch (Throwable t) {
// If BytesRef isn't actually UTF8, or it's eg a
// prefix of UTF8 that ends mid-unicode-char, we
// fallback to hex:
return b.toString();
}
}
// Writes all entries in the pending slice as a single
// block:
private PendingBlock writeBlock(IntsRef prevTerm, int prefixLength, int indexPrefixLength, int startBackwards, int length,
int futureTermCount, boolean isFloor, int floorLeadByte, boolean isLastInFloor) throws IOException {
assert length > 0;
final int start = pending.size()-startBackwards;
assert start >= 0: "pending.size()=" + pending.size() + " startBackwards=" + startBackwards + " length=" + length;
final List<PendingEntry> slice = pending.subList(start, start + length);
final long startFP = out.getFilePointer();
final BytesRef prefix = new BytesRef(indexPrefixLength);
for(int m=0;m<indexPrefixLength;m++) {
prefix.bytes[m] = (byte) prevTerm.ints[m];
}
prefix.length = indexPrefixLength;
// Write block header:
out.writeVInt((length<<1)|(isLastInFloor ? 1:0));
// if (DEBUG) {
// System.out.println(" writeBlock " + (isFloor ? "(floor) " : "") + "seg=" + segment + " pending.size()=" + pending.size() + " prefixLength=" + prefixLength + " indexPrefix=" + toString(prefix) + " entCount=" + length + " startFP=" + startFP + " futureTermCount=" + futureTermCount + (isFloor ? (" floorLeadByte=" + Integer.toHexString(floorLeadByte&0xff)) : "") + " isLastInFloor=" + isLastInFloor);
// }
// 1st pass: pack term suffix bytes into byte[] blob
// TODO: cutover to bulk int codec... simple64?
final boolean isLeafBlock;
if (lastBlockIndex < start) {
// This block definitely does not contain sub-blocks:
isLeafBlock = true;
//System.out.println("no scan true isFloor=" + isFloor);
} else if (!isFloor) {
// This block definitely does contain at least one sub-block:
isLeafBlock = false;
//System.out.println("no scan false " + lastBlockIndex + " vs start=" + start + " len=" + length);
} else {
// Must scan up-front to see if there is a sub-block
boolean v = true;
//System.out.println("scan " + lastBlockIndex + " vs start=" + start + " len=" + length);
for (PendingEntry ent : slice) {
if (!ent.isTerm) {
v = false;
break;
}
}
isLeafBlock = v;
}
final List<FST<BytesRef>> subIndices;
int termCount;
if (isLeafBlock) {
subIndices = null;
for (PendingEntry ent : slice) {
assert ent.isTerm;
PendingTerm term = (PendingTerm) ent;
final int suffix = term.term.length - prefixLength;
// if (DEBUG) {
// BytesRef suffixBytes = new BytesRef(suffix);
// System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
// suffixBytes.length = suffix;
// System.out.println(" write term suffix=" + suffixBytes);
// }
// For leaf block we write suffix straight
bytesWriter.writeVInt(suffix);
bytesWriter.writeBytes(term.term.bytes, prefixLength, suffix);
// Write term stats, to separate byte[] blob:
bytesWriter2.writeVInt(term.stats.docFreq);
if (fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) {
assert term.stats.totalTermFreq >= term.stats.docFreq;
bytesWriter2.writeVLong(term.stats.totalTermFreq - term.stats.docFreq);
}
}
termCount = length;
} else {
subIndices = new ArrayList<FST<BytesRef>>();
termCount = 0;
for (PendingEntry ent : slice) {
if (ent.isTerm) {
PendingTerm term = (PendingTerm) ent;
final int suffix = term.term.length - prefixLength;
// if (DEBUG) {
// BytesRef suffixBytes = new BytesRef(suffix);
// System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
// suffixBytes.length = suffix;
// System.out.println(" write term suffix=" + suffixBytes);
// }
// For non-leaf block we borrow 1 bit to record
// if entry is term or sub-block
bytesWriter.writeVInt(suffix<<1);
bytesWriter.writeBytes(term.term.bytes, prefixLength, suffix);
// Write term stats, to separate byte[] blob:
bytesWriter2.writeVInt(term.stats.docFreq);
if (fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) {
assert term.stats.totalTermFreq >= term.stats.docFreq;
bytesWriter2.writeVLong(term.stats.totalTermFreq - term.stats.docFreq);
}
termCount++;
} else {
PendingBlock block = (PendingBlock) ent;
final int suffix = block.prefix.length - prefixLength;
assert suffix > 0;
// For non-leaf block we borrow 1 bit to record
// if entry is term or sub-block
bytesWriter.writeVInt((suffix<<1)|1);
bytesWriter.writeBytes(block.prefix.bytes, prefixLength, suffix);
assert block.fp < startFP;
// if (DEBUG) {
// BytesRef suffixBytes = new BytesRef(suffix);
// System.arraycopy(block.prefix.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
// suffixBytes.length = suffix;
// System.out.println(" write sub-block suffix=" + toString(suffixBytes) + " subFP=" + block.fp + " subCode=" + (startFP-block.fp) + " floor=" + block.isFloor);
// }
bytesWriter.writeVLong(startFP - block.fp);
subIndices.add(block.index);
}
}
assert subIndices.size() != 0;
}
// TODO: we could block-write the term suffix pointers;
// this would take more space but would enable binary
// search on lookup
// Write suffixes byte[] blob to terms dict output:
out.writeVInt((int) (bytesWriter.getFilePointer() << 1) | (isLeafBlock ? 1:0));
bytesWriter.writeTo(out);
bytesWriter.reset();
// Write term stats byte[] blob
out.writeVInt((int) bytesWriter2.getFilePointer());
bytesWriter2.writeTo(out);
bytesWriter2.reset();
// Have postings writer write block
postingsWriter.flushTermsBlock(futureTermCount+termCount, termCount);
// Remove slice replaced by block:
slice.clear();
if (lastBlockIndex >= start) {
if (lastBlockIndex < start+length) {
lastBlockIndex = start;
} else {
lastBlockIndex -= length;
}
}
// if (DEBUG) {
// System.out.println(" fpEnd=" + out.getFilePointer());
// }
return new PendingBlock(prefix, startFP, termCount != 0, isFloor, floorLeadByte, subIndices);
}
TermsWriter(FieldInfo fieldInfo) {
this.fieldInfo = fieldInfo;
noOutputs = NoOutputs.getSingleton();
// This Builder is just used transiently to fragment
// terms into "good" blocks; we don't save the
// resulting FST:
blockBuilder = new Builder<Object>(FST.INPUT_TYPE.BYTE1,
0, 0, true,
true, Integer.MAX_VALUE,
noOutputs,
new FindBlocks());
postingsWriter.setField(fieldInfo);
}
@Override
public Comparator<BytesRef> getComparator() {
return BytesRef.getUTF8SortedAsUnicodeComparator();
}
@Override
public PostingsConsumer startTerm(BytesRef text) throws IOException {
//if (DEBUG) System.out.println("\nBTTW.startTerm term=" + fieldInfo.name + ":" + toString(text) + " seg=" + segment);
postingsWriter.startTerm();
/*
if (fieldInfo.name.equals("id")) {
postingsWriter.termID = Integer.parseInt(text.utf8ToString());
} else {
postingsWriter.termID = -1;
}
*/
return postingsWriter;
}
@Override
public void finishTerm(BytesRef text, TermStats stats) throws IOException {
assert stats.docFreq > 0;
//if (DEBUG) System.out.println("BTTW.finishTerm term=" + fieldInfo.name + ":" + toString(text) + " seg=" + segment + " df=" + stats.docFreq);
blockBuilder.add(text, noOutputs.getNoOutput());
pending.add(new PendingTerm(new BytesRef(text), stats));
postingsWriter.finishTerm(stats);
numTerms++;
}
// Finishes all terms in this field
@Override
public void finish(long sumTotalTermFreq, long sumDocFreq) throws IOException {
if (numTerms > 0) {
blockBuilder.finish();
// We better have one final "root" block:
assert pending.size() == 1 && !pending.get(0).isTerm: "pending.size()=" + pending.size() + " pending=" + pending;
final PendingBlock root = (PendingBlock) pending.get(0);
assert root.prefix.length == 0;
assert root.index.getEmptyOutput() != null;
this.sumTotalTermFreq = sumTotalTermFreq;
this.sumDocFreq = sumDocFreq;
// Write FST to index
indexStartFP = indexOut.getFilePointer();
root.index.save(indexOut);
//System.out.println(" write FST " + indexStartFP + " field=" + fieldInfo.name);
// if (SAVE_DOT_FILES || DEBUG) {
// final String dotFileName = segment + "_" + fieldInfo.name + ".dot";
// Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName));
// Util.toDot(root.index, w, false, false);
// System.out.println("SAVED to " + dotFileName);
// w.close();
// }
}
}
private final RAMOutputStream bytesWriter = new RAMOutputStream();
private final RAMOutputStream bytesWriter2 = new RAMOutputStream();
}
@Override
public void close() throws IOException {
IOException ioe = null;
try {
int nonZeroCount = 0;
for(TermsWriter field : fields) {
if (field.numTerms > 0) {
nonZeroCount++;
}
}
final long dirStart = out.getFilePointer();
final long indexDirStart = indexOut.getFilePointer();
out.writeVInt(nonZeroCount);
for(TermsWriter field : fields) {
if (field.numTerms > 0) {
//System.out.println(" field " + field.fieldInfo.name + " " + field.numTerms + " terms");
out.writeVInt(field.fieldInfo.number);
out.writeVLong(field.numTerms);
final BytesRef rootCode = ((PendingBlock) field.pending.get(0)).index.getEmptyOutput();
assert rootCode != null: "field=" + field.fieldInfo.name + " numTerms=" + field.numTerms;
out.writeVInt(rootCode.length);
out.writeBytes(rootCode.bytes, rootCode.offset, rootCode.length);
if (field.fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) {
out.writeVLong(field.sumTotalTermFreq);
}
out.writeVLong(field.sumDocFreq);
indexOut.writeVLong(field.indexStartFP);
}
}
writeTrailer(dirStart);
writeIndexTrailer(indexDirStart);
} catch (IOException ioe2) {
ioe = ioe2;
} finally {
IOUtils.closeSafely(ioe, out, indexOut, postingsWriter);
}
}
}

View File

@ -84,7 +84,7 @@ public class CodecProvider {
public synchronized Codec lookup(String name) {
final Codec codec = codecs.get(name);
if (codec == null) {
throw new IllegalArgumentException("required codec '" + name + "' not found");
throw new IllegalArgumentException("required codec '" + name + "' not found; known codecs: " + codecs.keySet());
}
return codec;
}

View File

@ -26,9 +26,8 @@ import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Bits;
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter; // javadocs
/** BlockTermsReader interacts with a single instance
/** The core terms dictionaries (BlockTermsReader,
* BlockTreeTermsReader) interact with a single instance
* of this class to manage creation of {@link DocsEnum} and
* {@link DocsAndPositionsEnum} instances. It provides an
* IndexInput (termsIn) where this class may read any
@ -49,11 +48,11 @@ public abstract class PostingsReaderBase implements Closeable {
/** Must fully consume state, since after this call that
* TermState may be reused. */
public abstract DocsEnum docs(FieldInfo fieldInfo, BlockTermState state, Bits liveDocs, DocsEnum reuse) throws IOException;
public abstract DocsEnum docs(FieldInfo fieldInfo, BlockTermState state, Bits skipDocs, DocsEnum reuse) throws IOException;
/** Must fully consume state, since after this call that
* TermState may be reused. */
public abstract DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState state, Bits liveDocs, DocsAndPositionsEnum reuse) throws IOException;
public abstract DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState state, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException;
public abstract void close() throws IOException;

View File

@ -33,7 +33,11 @@ public abstract class PostingsWriterBase extends PostingsConsumer implements Clo
public abstract void startTerm() throws IOException;
public abstract void flushTermsBlock() throws IOException;
/** Flush count terms starting at start "backwards", as a
* block. start is a negative offset from the end of the
* terms stack, ie bigger start means further back in
* the stack. */
public abstract void flushTermsBlock(int start, int count) throws IOException;
/** Finishes the current term */
public abstract void finishTerm(TermStats stats) throws IOException;

View File

@ -68,10 +68,14 @@ public abstract class FixedIntBlockIndexOutput extends IntIndexOutput {
}
@Override
public void set(IntIndexOutput.Index other) throws IOException {
public void copyFrom(IntIndexOutput.Index other, boolean copyLast) throws IOException {
Index idx = (Index) other;
lastFP = fp = idx.fp;
lastUpto = upto = idx.upto;
fp = idx.fp;
upto = idx.upto;
if (copyLast) {
lastFP = fp;
lastUpto = upto;
}
}
@Override

View File

@ -77,10 +77,14 @@ public abstract class VariableIntBlockIndexOutput extends IntIndexOutput {
}
@Override
public void set(IntIndexOutput.Index other) throws IOException {
public void copyFrom(IntIndexOutput.Index other, boolean copyLast) throws IOException {
Index idx = (Index) other;
lastFP = fp = idx.fp;
lastUpto = upto = idx.upto;
fp = idx.fp;
upto = idx.upto;
if (copyLast) {
lastFP = fp;
lastUpto = upto;
}
}
@Override

View File

@ -22,28 +22,23 @@ import java.util.Set;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.PostingsWriterBase;
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.codecs.PostingsReaderBase;
import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
import org.apache.lucene.index.codecs.PostingsWriterBase;
import org.apache.lucene.index.codecs.BlockTreeTermsReader;
import org.apache.lucene.index.codecs.BlockTreeTermsWriter;
import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
import org.apache.lucene.index.codecs.FieldsConsumer;
import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.index.codecs.PerDocConsumer;
import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
import org.apache.lucene.index.codecs.PerDocValues;
import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
import org.apache.lucene.index.codecs.BlockTermsReader;
import org.apache.lucene.index.codecs.BlockTermsWriter;
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
import org.apache.lucene.index.codecs.standard.StandardCodec;
import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
/** This codec "inlines" the postings for terms that have
* low docFreq. It wraps another codec, which is used for
@ -56,64 +51,52 @@ import org.apache.lucene.util.IOUtils;
public class PulsingCodec extends Codec {
private final int freqCutoff;
private final int minBlockSize;
private final int maxBlockSize;
/**
* Creates a {@link PulsingCodec} with <tt>freqCutoff = 1</tt>
*
* @see PulsingCodec#PulsingCodec(int)
*/
public PulsingCodec() {
this(1);
}
/** @lucene.internal */
public int getFreqCutoff() {
return freqCutoff;
public PulsingCodec(int freqCutoff) {
this(freqCutoff, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
}
/** Terms with freq <= freqCutoff are inlined into terms
* dict. */
public PulsingCodec(int freqCutoff) {
public PulsingCodec(int freqCutoff, int minBlockSize, int maxBlockSize) {
super("Pulsing");
this.freqCutoff = freqCutoff;
this.minBlockSize = minBlockSize;
assert minBlockSize > 1;
this.maxBlockSize = maxBlockSize;
}
@Override
public String toString() {
return name + "(freqCutoff=" + freqCutoff + ")";
return name + "(freqCutoff=" + freqCutoff + " minBlockSize=" + minBlockSize + " maxBlockSize=" + maxBlockSize + ")";
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
// We wrap StandardPostingsWriter, but any StandardPostingsWriter
// We wrap StandardPostingsWriter, but any PostingsWriterBase
// will work:
PostingsWriterBase docsWriter = new StandardPostingsWriter(state);
// Terms that have <= freqCutoff number of docs are
// "pulsed" (inlined):
PostingsWriterBase pulsingWriter = new PulsingPostingsWriterImpl(freqCutoff, docsWriter);
// Terms dict index
TermsIndexWriterBase indexWriter;
boolean success = false;
try {
indexWriter = new VariableGapTermsIndexWriter(state, new VariableGapTermsIndexWriter.EveryNTermSelector(state.termIndexInterval));
success = true;
} finally {
if (!success) {
IOUtils.closeSafely(true, pulsingWriter);
}
}
PostingsWriterBase pulsingWriter = new PulsingPostingsWriter(freqCutoff, docsWriter);
// Terms dict
success = false;
boolean success = false;
try {
FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, pulsingWriter);
FieldsConsumer ret = new BlockTreeTermsWriter(state, pulsingWriter, minBlockSize, maxBlockSize);
success = true;
return ret;
} finally {
if (!success) {
IOUtils.closeSafely(true, pulsingWriter, indexWriter);
pulsingWriter.close();
}
}
}
@ -124,53 +107,34 @@ public class PulsingCodec extends Codec {
// We wrap StandardPostingsReader, but any StandardPostingsReader
// will work:
PostingsReaderBase docsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
PostingsReaderBase pulsingReader = new PulsingPostingsReaderImpl(docsReader);
// Terms dict index reader
TermsIndexReaderBase indexReader;
PostingsReaderBase pulsingReader = new PulsingPostingsReader(docsReader);
boolean success = false;
try {
indexReader = new VariableGapTermsIndexReader(state.dir,
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
state.codecId, state.context);
FieldsProducer ret = new BlockTreeTermsReader(
state.dir, state.fieldInfos, state.segmentInfo.name,
pulsingReader,
state.context,
state.codecId,
state.termsIndexDivisor);
success = true;
return ret;
} finally {
if (!success) {
pulsingReader.close();
}
}
}
// Terms dict reader
success = false;
try {
FieldsProducer ret = new BlockTermsReader(indexReader,
state.dir, state.fieldInfos, state.segmentInfo.name,
pulsingReader,
state.context,
StandardCodec.TERMS_CACHE_SIZE,
state.codecId);
success = true;
return ret;
} finally {
if (!success) {
try {
pulsingReader.close();
} finally {
indexReader.close();
}
}
}
public int getFreqCutoff() {
return freqCutoff;
}
@Override
public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
StandardPostingsReader.files(dir, segmentInfo, id, files);
BlockTermsReader.files(dir, segmentInfo, id, files);
VariableGapTermsIndexReader.files(dir, segmentInfo, id, files);
DefaultDocValuesConsumer.files(dir, segmentInfo, id, files, getDocValuesUseCFS());
public void files(Directory dir, SegmentInfo segmentInfo, int codecID, Set<String> files) throws IOException {
StandardPostingsReader.files(dir, segmentInfo, codecID, files);
BlockTreeTermsReader.files(dir, segmentInfo, codecID, files);
DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files, getDocValuesUseCFS());
}
@Override

View File

@ -41,20 +41,20 @@ import org.apache.lucene.util.CodecUtil;
// create two separate docs readers, one that also reads
// prox and one that doesn't?
public class PulsingPostingsReaderImpl extends PostingsReaderBase {
public class PulsingPostingsReader extends PostingsReaderBase {
// Fallback reader for non-pulsed terms:
final PostingsReaderBase wrappedPostingsReader;
int maxPositions;
public PulsingPostingsReaderImpl(PostingsReaderBase wrappedPostingsReader) throws IOException {
public PulsingPostingsReader(PostingsReaderBase wrappedPostingsReader) throws IOException {
this.wrappedPostingsReader = wrappedPostingsReader;
}
@Override
public void init(IndexInput termsIn) throws IOException {
CodecUtil.checkHeader(termsIn, PulsingPostingsWriterImpl.CODEC,
PulsingPostingsWriterImpl.VERSION_START, PulsingPostingsWriterImpl.VERSION_START);
CodecUtil.checkHeader(termsIn, PulsingPostingsWriter.CODEC,
PulsingPostingsWriter.VERSION_START, PulsingPostingsWriter.VERSION_START);
maxPositions = termsIn.readVInt();
wrappedPostingsReader.init(termsIn);
}
@ -69,8 +69,15 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
@Override
public Object clone() {
PulsingTermState clone = new PulsingTermState();
clone.copyFrom(this);
PulsingTermState clone;
clone = (PulsingTermState) super.clone();
if (postingsSize != -1) {
clone.postings = new byte[postingsSize];
System.arraycopy(postings, 0, clone.postings, 0, postingsSize);
} else {
assert wrappedTermState != null;
clone.wrappedTermState = (BlockTermState) wrappedTermState.clone();
}
return clone;
}
@ -84,10 +91,8 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
postings = new byte[ArrayUtil.oversize(other.postingsSize, 1)];
}
System.arraycopy(other.postings, 0, postings, 0, other.postingsSize);
} else if (wrappedTermState != null) {
wrappedTermState.copyFrom(other.wrappedTermState);
} else {
wrappedTermState = (BlockTermState) other.wrappedTermState.clone();
wrappedTermState.copyFrom(other.wrappedTermState);
}
// NOTE: we do not copy the
@ -108,18 +113,20 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
@Override
public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
//System.out.println("PR.readTermsBlock state=" + _termState);
final PulsingTermState termState = (PulsingTermState) _termState;
if (termState.inlinedBytes == null) {
termState.inlinedBytes = new byte[128];
termState.inlinedBytesReader = new ByteArrayDataInput();
}
int len = termsIn.readVInt();
//System.out.println(" len=" + len + " fp=" + termsIn.getFilePointer());
if (termState.inlinedBytes.length < len) {
termState.inlinedBytes = new byte[ArrayUtil.oversize(len, 1)];
}
termsIn.readBytes(termState.inlinedBytes, 0, len);
termState.inlinedBytesReader.reset(termState.inlinedBytes);
termState.wrappedTermState.termCount = 0;
termState.wrappedTermState.termBlockOrd = 0;
wrappedPostingsReader.readTermsBlock(termsIn, fieldInfo, termState.wrappedTermState);
}
@ -140,7 +147,6 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
//System.out.println(" count=" + count + " threshold=" + maxPositions);
if (count <= maxPositions) {
//System.out.println(" inlined pos=" + termState.inlinedBytesReader.getPosition());
// Inlined into terms dict -- just read the byte[] blob in,
// but don't decode it now (we only decode when a DocsEnum
@ -154,6 +160,7 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
// current term block) into another byte[] (just the
// blob for this term)...
termState.inlinedBytesReader.readBytes(termState.postings, 0, termState.postingsSize);
//System.out.println(" inlined bytes=" + termState.postingsSize);
} else {
//System.out.println(" not inlined");
termState.postingsSize = -1;
@ -161,7 +168,7 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
termState.wrappedTermState.docFreq = termState.docFreq;
termState.wrappedTermState.totalTermFreq = termState.totalTermFreq;
wrappedPostingsReader.nextTerm(fieldInfo, termState.wrappedTermState);
termState.wrappedTermState.termCount++;
termState.wrappedTermState.termBlockOrd++;
}
}
@ -223,6 +230,7 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
}
private static class PulsingDocsEnum extends DocsEnum {
private byte[] postingsBytes;
private final ByteArrayDataInput postings = new ByteArrayDataInput();
private final IndexOptions indexOptions;
private final boolean storePayloads;
@ -239,9 +247,16 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
public PulsingDocsEnum reset(Bits liveDocs, PulsingTermState termState) {
//System.out.println("PR docsEnum termState=" + termState + " docFreq=" + termState.docFreq);
assert termState.postingsSize != -1;
final byte[] bytes = new byte[termState.postingsSize];
System.arraycopy(termState.postings, 0, bytes, 0, termState.postingsSize);
postings.reset(bytes);
// Must make a copy of termState's byte[] so that if
// app does TermsEnum.next(), this DocsEnum is not affected
if (postingsBytes == null) {
postingsBytes = new byte[termState.postingsSize];
} else if (postingsBytes.length < termState.postingsSize) {
postingsBytes = ArrayUtil.grow(postingsBytes, termState.postingsSize);
}
System.arraycopy(termState.postings, 0, postingsBytes, 0, termState.postingsSize);
postings.reset(postingsBytes, 0, termState.postingsSize);
docID = 0;
payloadLength = 0;
freq = 1;
@ -263,6 +278,7 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
}
final int code = postings.readVInt();
//System.out.println(" read code=" + code);
if (indexOptions == IndexOptions.DOCS_ONLY) {
docID += code;
} else {
@ -295,7 +311,6 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
}
if (liveDocs == null || liveDocs.get(docID)) {
//System.out.println(" return docID=" + docID + " freq=" + freq);
return docID;
}
}
@ -323,6 +338,7 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
}
private static class PulsingDocsAndPositionsEnum extends DocsAndPositionsEnum {
private byte[] postingsBytes;
private final ByteArrayDataInput postings = new ByteArrayDataInput();
private final boolean storePayloads;
@ -346,9 +362,13 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
public PulsingDocsAndPositionsEnum reset(Bits liveDocs, PulsingTermState termState) {
assert termState.postingsSize != -1;
final byte[] bytes = new byte[termState.postingsSize];
System.arraycopy(termState.postings, 0, bytes, 0, termState.postingsSize);
postings.reset(bytes);
if (postingsBytes == null) {
postingsBytes = new byte[termState.postingsSize];
} else if (postingsBytes.length < termState.postingsSize) {
postingsBytes = ArrayUtil.grow(postingsBytes, termState.postingsSize);
}
System.arraycopy(termState.postings, 0, postingsBytes, 0, termState.postingsSize);
postings.reset(postingsBytes, 0, termState.postingsSize);
this.liveDocs = liveDocs;
payloadLength = 0;
posPending = 0;
@ -359,7 +379,7 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
@Override
public int nextDoc() throws IOException {
//System.out.println("PR.nextDoc this=" + this);
//System.out.println("PR d&p nextDoc this=" + this);
while(true) {
//System.out.println(" cycle skip posPending=" + posPending);
@ -367,16 +387,15 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
skipPositions();
if (postings.eof()) {
//System.out.println(" END");
//System.out.println("PR END");
return docID = NO_MORE_DOCS;
}
//System.out.println(" read doc code");
final int code = postings.readVInt();
docID += code >>> 1; // shift off low bit
if ((code & 1) != 0) { // if low bit is set
freq = 1; // freq is one
} else {
//System.out.println(" read freq");
freq = postings.readVInt(); // else read freq
}
posPending = freq;
@ -401,10 +420,8 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
@Override
public int advance(int target) throws IOException {
//System.out.println("PR.advance target=" + target);
int doc;
while((doc=nextDoc()) != NO_MORE_DOCS) {
//System.out.println(" nextDoc got doc=" + doc);
if (doc >= target) {
return docID = doc;
}
@ -414,7 +431,7 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
@Override
public int nextPosition() throws IOException {
//System.out.println("PR.nextPosition posPending=" + posPending + " vs freq=" + freq);
//System.out.println("PR d&p nextPosition posPending=" + posPending + " vs freq=" + freq);
assert posPending > 0;
posPending--;
@ -424,7 +441,6 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
//System.out.println("PR skip payload=" + payloadLength);
postings.skipBytes(payloadLength);
}
//System.out.println(" read pos code");
final int code = postings.readVInt();
//System.out.println("PR code=" + code);
if ((code & 1) != 0) {
@ -437,17 +453,16 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
position += postings.readVInt();
}
//System.out.println(" return pos=" + position + " hasPayload=" + !payloadRetrieved + " posPending=" + posPending + " this=" + this);
//System.out.println("PR d&p nextPos return pos=" + position + " this=" + this);
return position;
}
private void skipPositions() throws IOException {
//System.out.println("PR.skipPositions: posPending=" + posPending);
while(posPending != 0) {
nextPosition();
}
if (storePayloads && !payloadRetrieved) {
//System.out.println(" skip last payload len=" + payloadLength);
//System.out.println(" skip payload len=" + payloadLength);
postings.skipBytes(payloadLength);
payloadRetrieved = true;
}

View File

@ -18,6 +18,8 @@ package org.apache.lucene.index.codecs.pulsing;
*/
import java.io.IOException;
import java.util.List;
import java.util.ArrayList;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions;
@ -35,9 +37,9 @@ import org.apache.lucene.util.CodecUtil;
// presumably rare in practice...
/** @lucene.experimental */
public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
public final class PulsingPostingsWriter extends PostingsWriterBase {
final static String CODEC = "PulsedPostings";
final static String CODEC = "PulsedPostingsWriter";
// To add a new version, increment from the last one, and
// change VERSION_CURRENT to point to your new version:
@ -50,6 +52,15 @@ public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
private IndexOptions indexOptions;
private boolean storePayloads;
private static class PendingTerm {
private final byte[] bytes;
public PendingTerm(byte[] bytes) {
this.bytes = bytes;
}
}
private final List<PendingTerm> pendingTerms = new ArrayList<PendingTerm>();
// one entry per position
private final Position[] pending;
private int pendingCount = 0; // -1 once we've hit too many positions
@ -71,7 +82,7 @@ public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
/** If the total number of positions (summed across all docs
* for this term) is <= maxPositions, then the postings are
* inlined into terms dict */
public PulsingPostingsWriterImpl(int maxPositions, PostingsWriterBase wrappedPostingsWriter) throws IOException {
public PulsingPostingsWriter(int maxPositions, PostingsWriterBase wrappedPostingsWriter) throws IOException {
pending = new Position[maxPositions];
for(int i=0;i<maxPositions;i++) {
pending[i] = new Position();
@ -92,7 +103,7 @@ public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
@Override
public void startTerm() {
//System.out.println("PW startTerm");
if (DEBUG) System.out.println("PW startTerm");
assert pendingCount == 0;
}
@ -104,19 +115,33 @@ public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
@Override
public void setField(FieldInfo fieldInfo) {
this.indexOptions = fieldInfo.indexOptions;
//System.out.println("PW field=" + fieldInfo.name + " omitTF=" + omitTF);
if (DEBUG) System.out.println("PW field=" + fieldInfo.name + " indexOptions=" + indexOptions);
storePayloads = fieldInfo.storePayloads;
wrappedPostingsWriter.setField(fieldInfo);
//DEBUG = BlockTreeTermsWriter.DEBUG;
}
private boolean DEBUG;
@Override
public void startDoc(int docID, int termDocFreq) throws IOException {
assert docID >= 0: "got docID=" + docID;
//System.out.println("PW doc=" + docID);
/*
if (termID != -1) {
if (docID == 0) {
baseDocID = termID;
} else if (baseDocID + docID != termID) {
throw new RuntimeException("WRITE: baseDocID=" + baseDocID + " docID=" + docID + " termID=" + termID);
}
}
*/
if (DEBUG) System.out.println("PW doc=" + docID);
if (pendingCount == pending.length) {
push();
//System.out.println("PW: wrapped.finishDoc");
if (DEBUG) System.out.println("PW: wrapped.finishDoc");
wrappedPostingsWriter.finishDoc();
}
@ -142,7 +167,7 @@ public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
@Override
public void addPosition(int position, BytesRef payload) throws IOException {
//System.out.println("PW pos=" + position + " payload=" + (payload == null ? "null" : payload.length + " bytes"));
if (DEBUG) System.out.println("PW pos=" + position + " payload=" + (payload == null ? "null" : payload.length + " bytes"));
if (pendingCount == pending.length) {
push();
}
@ -170,24 +195,28 @@ public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
@Override
public void finishDoc() throws IOException {
//System.out.println("PW finishDoc");
if (DEBUG) System.out.println("PW finishDoc");
if (pendingCount == -1) {
wrappedPostingsWriter.finishDoc();
}
}
private final RAMOutputStream buffer = new RAMOutputStream();
private final RAMOutputStream buffer2 = new RAMOutputStream();
// private int baseDocID;
/** Called when we are done adding docs to this term */
@Override
public void finishTerm(TermStats stats) throws IOException {
//System.out.println("PW finishTerm docCount=" + stats.docFreq);
if (DEBUG) System.out.println("PW finishTerm docCount=" + stats.docFreq + " pendingCount=" + pendingCount + " pendingTerms.size()=" + pendingTerms.size());
assert pendingCount > 0 || pendingCount == -1;
if (pendingCount == -1) {
wrappedPostingsWriter.finishTerm(stats);
// Must add null entry to record terms that our
// wrapped postings impl added
pendingTerms.add(null);
} else {
// There were few enough total occurrences for this
@ -210,7 +239,7 @@ public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
final int delta = doc.docID - lastDocID;
lastDocID = doc.docID;
//System.out.println(" write doc=" + doc.docID + " freq=" + doc.termFreq);
if (DEBUG) System.out.println(" write doc=" + doc.docID + " freq=" + doc.termFreq);
if (doc.termFreq == 1) {
buffer.writeVInt((delta<<1)|1);
@ -225,7 +254,7 @@ public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
assert pos.docID == doc.docID;
final int posDelta = pos.pos - lastPos;
lastPos = pos.pos;
//System.out.println(" write pos=" + pos.pos);
if (DEBUG) System.out.println(" write pos=" + pos.pos);
if (storePayloads) {
final int payloadLength = pos.payload == null ? 0 : pos.payload.length;
if (payloadLength != lastPayloadLength) {
@ -248,6 +277,7 @@ public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
for(int posIDX=0;posIDX<pendingCount;posIDX++) {
final Position doc = pending[posIDX];
final int delta = doc.docID - lastDocID;
assert doc.termFreq != 0;
if (doc.termFreq == 1) {
buffer.writeVInt((delta<<1)|1);
} else {
@ -265,9 +295,9 @@ public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
}
}
//System.out.println(" bytes=" + buffer.getFilePointer());
buffer2.writeVInt((int) buffer.getFilePointer());
buffer.writeTo(buffer2);
final byte[] bytes = new byte[(int) buffer.getFilePointer()];
buffer.writeTo(bytes, 0);
pendingTerms.add(new PendingTerm(bytes));
buffer.reset();
}
@ -280,20 +310,51 @@ public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
}
@Override
public void flushTermsBlock() throws IOException {
termsOut.writeVInt((int) buffer2.getFilePointer());
buffer2.writeTo(termsOut);
buffer2.reset();
public void flushTermsBlock(int start, int count) throws IOException {
if (DEBUG) System.out.println("PW: flushTermsBlock start=" + start + " count=" + count + " pendingTerms.size()=" + pendingTerms.size());
int wrappedCount = 0;
assert buffer.getFilePointer() == 0;
assert start >= count;
final int limit = pendingTerms.size() - start + count;
for(int idx=pendingTerms.size()-start; idx<limit; idx++) {
final PendingTerm term = pendingTerms.get(idx);
if (term == null) {
wrappedCount++;
} else {
buffer.writeVInt(term.bytes.length);
buffer.writeBytes(term.bytes, 0, term.bytes.length);
}
}
termsOut.writeVInt((int) buffer.getFilePointer());
buffer.writeTo(termsOut);
buffer.reset();
// TDOO: this could be somewhat costly since
// pendingTerms.size() could be biggish?
int futureWrappedCount = 0;
final int limit2 = pendingTerms.size();
for(int idx=limit;idx<limit2;idx++) {
if (pendingTerms.get(idx) == null) {
futureWrappedCount++;
}
}
// Remove the terms we just wrote:
pendingTerms.subList(pendingTerms.size()-start, limit).clear();
if (DEBUG) System.out.println("PW: len=" + buffer.getFilePointer() + " fp=" + termsOut.getFilePointer() + " futureWrappedCount=" + futureWrappedCount + " wrappedCount=" + wrappedCount);
// TODO: can we avoid calling this if all terms
// were inlined...? Eg for a "primary key" field, the
// wrapped codec is never invoked...
wrappedPostingsWriter.flushTermsBlock();
wrappedPostingsWriter.flushTermsBlock(futureWrappedCount+wrappedCount, wrappedCount);
}
// Pushes pending positions to the wrapped codec
private void push() throws IOException {
//System.out.println("PW now push @ " + pendingCount + " wrapped=" + wrappedPostingsWriter);
if (DEBUG) System.out.println("PW now push @ " + pendingCount + " wrapped=" + wrappedPostingsWriter);
assert pendingCount == pending.length;
wrappedPostingsWriter.startTerm();
@ -304,17 +365,17 @@ public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
for(Position pos : pending) {
if (doc == null) {
doc = pos;
//System.out.println("PW: wrapped.startDoc docID=" + doc.docID + " tf=" + doc.termFreq);
if (DEBUG) System.out.println("PW: wrapped.startDoc docID=" + doc.docID + " tf=" + doc.termFreq);
wrappedPostingsWriter.startDoc(doc.docID, doc.termFreq);
} else if (doc.docID != pos.docID) {
assert pos.docID > doc.docID;
//System.out.println("PW: wrapped.finishDoc");
if (DEBUG) System.out.println("PW: wrapped.finishDoc");
wrappedPostingsWriter.finishDoc();
doc = pos;
//System.out.println("PW: wrapped.startDoc docID=" + doc.docID + " tf=" + doc.termFreq);
if (DEBUG) System.out.println("PW: wrapped.startDoc docID=" + doc.docID + " tf=" + doc.termFreq);
wrappedPostingsWriter.startDoc(doc.docID, doc.termFreq);
}
//System.out.println("PW: wrapped.addPos pos=" + pos.pos);
if (DEBUG) System.out.println("PW: wrapped.addPos pos=" + pos.pos);
wrappedPostingsWriter.addPosition(pos.pos, pos.payload);
}
//wrappedPostingsWriter.finishDoc();

View File

@ -44,7 +44,7 @@ public abstract class IntIndexOutput implements Closeable {
public abstract void mark() throws IOException;
/** Copies index from other */
public abstract void set(Index other) throws IOException;
public abstract void copyFrom(Index other, boolean copyLast) throws IOException;
/** Writes "location" of current output pointer of primary
* output to different output (out) */

View File

@ -48,7 +48,7 @@ import org.apache.lucene.util.CodecUtil;
// create two separate docs readers, one that also reads
// prox and one that doesn't?
public class SepPostingsReaderImpl extends PostingsReaderBase {
public class SepPostingsReader extends PostingsReaderBase {
final IntIndexInput freqIn;
final IntIndexInput docIn;
@ -60,23 +60,23 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
int maxSkipLevels;
int skipMinimum;
public SepPostingsReaderImpl(Directory dir, SegmentInfo segmentInfo, IOContext context, IntStreamFactory intFactory, int codecId) throws IOException {
public SepPostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext context, IntStreamFactory intFactory, int codecId) throws IOException {
boolean success = false;
try {
final String docFileName = IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.DOC_EXTENSION);
final String docFileName = IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.DOC_EXTENSION);
docIn = intFactory.openInput(dir, docFileName, context);
skipIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.SKIP_EXTENSION), context);
skipIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.SKIP_EXTENSION), context);
if (segmentInfo.getFieldInfos().hasFreq()) {
freqIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.FREQ_EXTENSION), context);
freqIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.FREQ_EXTENSION), context);
} else {
freqIn = null;
}
if (segmentInfo.getHasProx()) {
posIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.POS_EXTENSION), context);
payloadIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.PAYLOAD_EXTENSION), context);
posIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.POS_EXTENSION), context);
payloadIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.PAYLOAD_EXTENSION), context);
} else {
posIn = null;
payloadIn = null;
@ -90,24 +90,24 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
}
public static void files(SegmentInfo segmentInfo, int codecId, Collection<String> files) throws IOException {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.DOC_EXTENSION));
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.SKIP_EXTENSION));
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.DOC_EXTENSION));
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.SKIP_EXTENSION));
if (segmentInfo.getFieldInfos().hasFreq()) {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.FREQ_EXTENSION));
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.FREQ_EXTENSION));
}
if (segmentInfo.getHasProx()) {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.POS_EXTENSION));
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.PAYLOAD_EXTENSION));
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.POS_EXTENSION));
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriter.PAYLOAD_EXTENSION));
}
}
@Override
public void init(IndexInput termsIn) throws IOException {
// Make sure we are talking to the matching past writer
CodecUtil.checkHeader(termsIn, SepPostingsWriterImpl.CODEC,
SepPostingsWriterImpl.VERSION_START, SepPostingsWriterImpl.VERSION_START);
CodecUtil.checkHeader(termsIn, SepPostingsWriter.CODEC,
SepPostingsWriter.VERSION_START, SepPostingsWriter.VERSION_START);
skipInterval = termsIn.readInt();
maxSkipLevels = termsIn.readInt();
skipMinimum = termsIn.readInt();
@ -153,6 +153,10 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
// Only used for "primary" term state; these are never
// copied on clone:
// TODO: these should somehow be stored per-TermsEnum
// not per TermState; maybe somehow the terms dict
// should load/manage the byte[]/DataReader for us?
byte[] bytes;
ByteArrayDataInput bytesReader;
@ -216,8 +220,9 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
@Override
public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
final SepTermState termState = (SepTermState) _termState;
//System.out.println("SEPR: readTermsBlock termsIn.fp=" + termsIn.getFilePointer());
final int len = termsIn.readVInt();
//System.out.println("SepR.readTermsBlock len=" + len);
//System.out.println(" numBytes=" + len);
if (termState.bytes == null) {
termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
termState.bytesReader = new ByteArrayDataInput(termState.bytes);
@ -231,30 +236,30 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
@Override
public void nextTerm(FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
final SepTermState termState = (SepTermState) _termState;
//System.out.println("SepR.nextTerm termCount=" + termState.termCount);
final boolean isFirstTerm = termState.termBlockOrd == 0;
//System.out.println("SEPR.nextTerm termCount=" + termState.termBlockOrd + " isFirstTerm=" + isFirstTerm + " bytesReader.pos=" + termState.bytesReader.getPosition());
//System.out.println(" docFreq=" + termState.docFreq);
final boolean isFirstTerm = termState.termCount == 0;
termState.docIndex.read(termState.bytesReader, isFirstTerm);
//System.out.println(" docIndex=" + termState.docIndex);
if (fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) {
termState.freqIndex.read(termState.bytesReader, isFirstTerm);
}
if (fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
//System.out.println(" freqIndex=" + termState.freqIndex);
termState.posIndex.read(termState.bytesReader, isFirstTerm);
//System.out.println(" posIndex=" + termState.posIndex);
if (fieldInfo.storePayloads) {
if (isFirstTerm) {
termState.payloadFP = termState.bytesReader.readVLong();
} else {
termState.payloadFP += termState.bytesReader.readVLong();
if (fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
//System.out.println(" freqIndex=" + termState.freqIndex);
termState.posIndex.read(termState.bytesReader, isFirstTerm);
//System.out.println(" posIndex=" + termState.posIndex);
if (fieldInfo.storePayloads) {
if (isFirstTerm) {
termState.payloadFP = termState.bytesReader.readVLong();
} else {
termState.payloadFP += termState.bytesReader.readVLong();
}
//System.out.println(" payloadFP=" + termState.payloadFP);
}
//System.out.println(" payloadFP=" + termState.payloadFP);
}
}
if (termState.docFreq >= skipMinimum) {
//System.out.println(" readSkip @ " + termState.bytesReader.pos);
//System.out.println(" readSkip @ " + termState.bytesReader.getPosition());
if (isFirstTerm) {
termState.skipFP = termState.bytesReader.readVLong();
} else {
@ -538,7 +543,7 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
freqIndex = freqIn.index();
posReader = posIn.reader();
posIndex = posIn.index();
payloadIn = (IndexInput) SepPostingsReaderImpl.this.payloadIn.clone();
payloadIn = (IndexInput) SepPostingsReader.this.payloadIn.clone();
}
SepDocsAndPositionsEnum init(FieldInfo fieldInfo, SepTermState termState, Bits liveDocs) throws IOException {
@ -656,6 +661,7 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
// Skipper did move
skipper.getFreqIndex().seek(freqReader);
skipper.getDocIndex().seek(docReader);
//System.out.println(" doc seek'd to " + skipper.getDocIndex());
// NOTE: don't seek pos here; do it lazily
// instead. Eg a PhraseQuery may skip to many
// docs before finally asking for positions...

View File

@ -18,6 +18,8 @@ package org.apache.lucene.index.codecs.sep;
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import org.apache.lucene.index.CorruptIndexException;
@ -38,8 +40,8 @@ import org.apache.lucene.util.IOUtils;
* to .pyl, skip data to .skp
*
* @lucene.experimental */
public final class SepPostingsWriterImpl extends PostingsWriterBase {
final static String CODEC = "SepDocFreqSkip";
public final class SepPostingsWriter extends PostingsWriterBase {
final static String CODEC = "SepPostingsWriter";
final static String DOC_EXTENSION = "doc";
final static String SKIP_EXTENSION = "skp";
@ -89,26 +91,22 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
boolean storePayloads;
IndexOptions indexOptions;
long lastSkipFP;
FieldInfo fieldInfo;
int lastPayloadLength;
int lastPosition;
long payloadStart;
long lastPayloadStart;
int lastDocID;
int df;
private int pendingTermCount;
// Holds pending byte[] blob for the current terms block
private final RAMOutputStream indexBytesWriter = new RAMOutputStream();
public SepPostingsWriterImpl(SegmentWriteState state, IntStreamFactory factory) throws IOException {
public SepPostingsWriter(SegmentWriteState state, IntStreamFactory factory) throws IOException {
this(state, factory, DEFAULT_SKIP_INTERVAL);
}
public SepPostingsWriterImpl(SegmentWriteState state, IntStreamFactory factory, int skipInterval) throws IOException {
public SepPostingsWriter(SegmentWriteState state, IntStreamFactory factory, int skipInterval) throws IOException {
freqOut = null;
freqIndex = null;
posOut = null;
@ -171,6 +169,7 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
@Override
public void startTerm() throws IOException {
docIndex.mark();
//System.out.println("SEPW: startTerm docIndex=" + docIndex);
if (indexOptions != IndexOptions.DOCS_ONLY) {
freqIndex.mark();
@ -201,7 +200,7 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
public void startDoc(int docID, int termDocFreq) throws IOException {
final int delta = docID - lastDocID;
//System.out.println("SepW startDoc: write doc=" + docID + " delta=" + delta);
//System.out.println("SEPW: startDoc: write doc=" + docID + " delta=" + delta + " out.fp=" + docOut);
if (docID < 0 || (df > 0 && delta <= 0)) {
throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " )");
@ -223,15 +222,6 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
}
}
@Override
public void flushTermsBlock() throws IOException {
//System.out.println("SepW.flushTermsBlock: pendingTermCount=" + pendingTermCount + " bytesUsed=" + indexBytesWriter.getFilePointer());
termsOut.writeVLong((int) indexBytesWriter.getFilePointer());
indexBytesWriter.writeTo(termsOut);
indexBytesWriter.reset();
pendingTermCount = 0;
}
/** Add a new position & payload */
@Override
public void addPosition(int position, BytesRef payload) throws IOException {
@ -269,6 +259,24 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
lastPosition = 0;
}
private static class PendingTerm {
public final IntIndexOutput.Index docIndex;
public final IntIndexOutput.Index freqIndex;
public final IntIndexOutput.Index posIndex;
public final long payloadFP;
public final long skipFP;
public PendingTerm(IntIndexOutput.Index docIndex, IntIndexOutput.Index freqIndex, IntIndexOutput.Index posIndex, long payloadFP, long skipFP) {
this.docIndex = docIndex;
this.freqIndex = freqIndex;
this.posIndex = posIndex;
this.payloadFP = payloadFP;
this.skipFP = skipFP;
}
}
private final List<PendingTerm> pendingTerms = new ArrayList<PendingTerm>();
/** Called when we are done adding docs to this term */
@Override
public void finishTerm(TermStats stats) throws IOException {
@ -276,50 +284,107 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
assert stats.docFreq > 0;
assert stats.docFreq == df;
final boolean isFirstTerm = pendingTermCount == 0;
//System.out.println("SepW.finishTerm: isFirstTerm=" + isFirstTerm);
docIndex.write(indexBytesWriter, isFirstTerm);
//System.out.println(" docIndex=" + docIndex);
final IntIndexOutput.Index docIndexCopy = docOut.index();
docIndexCopy.copyFrom(docIndex, false);
final IntIndexOutput.Index freqIndexCopy;
final IntIndexOutput.Index posIndexCopy;
if (indexOptions != IndexOptions.DOCS_ONLY) {
freqIndex.write(indexBytesWriter, isFirstTerm);
//System.out.println(" freqIndex=" + freqIndex);
}
if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
posIndex.write(indexBytesWriter, isFirstTerm);
//System.out.println(" posIndex=" + posIndex);
if (storePayloads) {
if (isFirstTerm) {
indexBytesWriter.writeVLong(payloadStart);
} else {
indexBytesWriter.writeVLong(payloadStart - lastPayloadStart);
}
lastPayloadStart = payloadStart;
//System.out.println(" payloadFP=" + payloadStart);
}
}
if (df >= skipMinimum) {
//System.out.println(" skipFP=" + skipStart);
final long skipFP = skipOut.getFilePointer();
skipListWriter.writeSkip(skipOut);
//System.out.println(" writeSkip @ " + indexBytesWriter.getFilePointer());
if (isFirstTerm) {
indexBytesWriter.writeVLong(skipFP);
freqIndexCopy = freqOut.index();
freqIndexCopy.copyFrom(freqIndex, false);
if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
posIndexCopy = posOut.index();
posIndexCopy.copyFrom(posIndex, false);
} else {
indexBytesWriter.writeVLong(skipFP - lastSkipFP);
posIndexCopy = null;
}
lastSkipFP = skipFP;
} else if (isFirstTerm) {
// lazily write an absolute delta if a term in this block requires skip data.
lastSkipFP = 0;
} else {
freqIndexCopy = null;
posIndexCopy = null;
}
final long skipFP;
if (df >= skipMinimum) {
skipFP = skipOut.getFilePointer();
//System.out.println(" skipFP=" + skipFP);
skipListWriter.writeSkip(skipOut);
//System.out.println(" numBytes=" + (skipOut.getFilePointer()-skipFP));
} else {
skipFP = -1;
}
lastDocID = 0;
df = 0;
pendingTermCount++;
pendingTerms.add(new PendingTerm(docIndexCopy,
freqIndexCopy,
posIndexCopy,
payloadStart,
skipFP));
}
@Override
public void flushTermsBlock(int start, int count) throws IOException {
//System.out.println("SEPW: flushTermsBlock: start=" + start + " count=" + count + " pendingTerms.size()=" + pendingTerms.size() + " termsOut.fp=" + termsOut.getFilePointer());
assert indexBytesWriter.getFilePointer() == 0;
final int absStart = pendingTerms.size() - start;
final List<PendingTerm> slice = pendingTerms.subList(absStart, absStart+count);
long lastPayloadFP = 0;
long lastSkipFP = 0;
if (count == 0) {
termsOut.writeByte((byte) 0);
return;
}
final PendingTerm firstTerm = slice.get(0);
final IntIndexOutput.Index docIndexFlush = firstTerm.docIndex;
final IntIndexOutput.Index freqIndexFlush = firstTerm.freqIndex;
final IntIndexOutput.Index posIndexFlush = firstTerm.posIndex;
for(int idx=0;idx<slice.size();idx++) {
final boolean isFirstTerm = idx == 0;
final PendingTerm t = slice.get(idx);
//System.out.println(" write idx=" + idx + " docIndex=" + t.docIndex);
docIndexFlush.copyFrom(t.docIndex, false);
docIndexFlush.write(indexBytesWriter, isFirstTerm);
if (indexOptions != IndexOptions.DOCS_ONLY) {
freqIndexFlush.copyFrom(t.freqIndex, false);
freqIndexFlush.write(indexBytesWriter, isFirstTerm);
//System.out.println(" freqIndex=" + t.freqIndex);
if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
posIndexFlush.copyFrom(t.posIndex, false);
posIndexFlush.write(indexBytesWriter, isFirstTerm);
//System.out.println(" posIndex=" + t.posIndex);
if (storePayloads) {
//System.out.println(" payloadFP=" + t.payloadFP);
if (isFirstTerm) {
indexBytesWriter.writeVLong(t.payloadFP);
} else {
indexBytesWriter.writeVLong(t.payloadFP - lastPayloadFP);
}
lastPayloadFP = t.payloadFP;
}
}
}
if (t.skipFP != -1) {
if (isFirstTerm) {
indexBytesWriter.writeVLong(t.skipFP);
} else {
indexBytesWriter.writeVLong(t.skipFP - lastSkipFP);
}
lastSkipFP = t.skipFP;
//System.out.println(" skipFP=" + t.skipFP);
}
}
//System.out.println(" numBytes=" + indexBytesWriter.getFilePointer());
termsOut.writeVLong((int) indexBytesWriter.getFilePointer());
indexBytesWriter.writeTo(termsOut);
indexBytesWriter.reset();
slice.clear();
}
@Override

View File

@ -124,12 +124,12 @@ class SepSkipListWriter extends MultiLevelSkipListWriter {
Arrays.fill(lastSkipDoc, 0);
Arrays.fill(lastSkipPayloadLength, -1); // we don't have to write the first length in the skip list
for(int i=0;i<numberOfSkipLevels;i++) {
docIndex[i].set(topDocIndex);
docIndex[i].copyFrom(topDocIndex, true);
if (freqOutput != null) {
freqIndex[i].set(topFreqIndex);
freqIndex[i].copyFrom(topFreqIndex, true);
}
if (posOutput != null) {
posIndex[i].set(topPosIndex);
posIndex[i].copyFrom(topPosIndex, true);
}
}
if (payloadOutput != null) {

View File

@ -22,31 +22,37 @@ import java.util.Set;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.codecs.PostingsReaderBase;
import org.apache.lucene.index.codecs.PostingsWriterBase;
import org.apache.lucene.index.codecs.BlockTreeTermsReader;
import org.apache.lucene.index.codecs.BlockTreeTermsWriter;
import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
import org.apache.lucene.index.codecs.FieldsConsumer;
import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.index.codecs.PerDocConsumer;
import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
import org.apache.lucene.index.codecs.PerDocValues;
import org.apache.lucene.index.codecs.PostingsWriterBase;
import org.apache.lucene.index.codecs.PostingsReaderBase;
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
import org.apache.lucene.index.codecs.BlockTermsWriter;
import org.apache.lucene.index.codecs.BlockTermsReader;
import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
import org.apache.lucene.store.Directory;
/** Default codec.
* @lucene.experimental */
public class StandardCodec extends Codec {
private final int minBlockSize;
private final int maxBlockSize;
public StandardCodec() {
this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
}
public StandardCodec(int minBlockSize, int maxBlockSize) {
super("Standard");
this.minBlockSize = minBlockSize;
assert minBlockSize > 1;
this.maxBlockSize = maxBlockSize;
}
@Override
@ -57,29 +63,14 @@ public class StandardCodec extends Codec {
// pluggable? Ie so that this codec would record which
// index impl was used, and switch on loading?
// Or... you must make a new Codec for this?
TermsIndexWriterBase indexWriter;
boolean success = false;
try {
indexWriter = new VariableGapTermsIndexWriter(state, new VariableGapTermsIndexWriter.EveryNTermSelector(state.termIndexInterval));
success = true;
} finally {
if (!success) {
docs.close();
}
}
success = false;
try {
FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, docs);
FieldsConsumer ret = new BlockTreeTermsWriter(state, docs, minBlockSize, maxBlockSize);
success = true;
return ret;
} finally {
if (!success) {
try {
docs.close();
} finally {
indexWriter.close();
}
docs.close();
}
}
}
@ -89,41 +80,22 @@ public class StandardCodec extends Codec {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postings = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
TermsIndexReaderBase indexReader;
boolean success = false;
try {
indexReader = new VariableGapTermsIndexReader(state.dir,
FieldsProducer ret = new BlockTreeTermsReader(
state.dir,
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
state.codecId, state.context);
success = true;
} finally {
if (!success) {
postings.close();
}
}
success = false;
try {
FieldsProducer ret = new BlockTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,
postings,
state.context,
TERMS_CACHE_SIZE,
state.codecId);
postings,
state.context,
state.codecId,
state.termsIndexDivisor);
success = true;
return ret;
} finally {
if (!success) {
try {
postings.close();
} finally {
indexReader.close();
}
postings.close();
}
}
}
@ -135,11 +107,10 @@ public class StandardCodec extends Codec {
static final String PROX_EXTENSION = "prx";
@Override
public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
StandardPostingsReader.files(dir, segmentInfo, id, files);
BlockTermsReader.files(dir, segmentInfo, id, files);
VariableGapTermsIndexReader.files(dir, segmentInfo, id, files);
DefaultDocValuesConsumer.files(dir, segmentInfo, id, files, getDocValuesUseCFS());
public void files(Directory dir, SegmentInfo segmentInfo, int codecID, Set<String> files) throws IOException {
StandardPostingsReader.files(dir, segmentInfo, codecID, files);
BlockTreeTermsReader.files(dir, segmentInfo, codecID, files);
DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files, getDocValuesUseCFS());
}
@Override
@ -151,8 +122,12 @@ public class StandardCodec extends Codec {
public static void getStandardExtensions(Set<String> extensions) {
extensions.add(FREQ_EXTENSION);
extensions.add(PROX_EXTENSION);
BlockTermsReader.getExtensions(extensions);
VariableGapTermsIndexReader.getIndexExtensions(extensions);
BlockTreeTermsReader.getExtensions(extensions);
}
@Override
public String toString() {
return name + "(minBlockSize=" + minBlockSize + " maxBlockSize=" + maxBlockSize + ")";
}
@Override

View File

@ -27,8 +27,8 @@ import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.codecs.BlockTermState;
import org.apache.lucene.index.codecs.PostingsReaderBase;
import org.apache.lucene.index.codecs.BlockTermState;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
@ -46,22 +46,23 @@ public class StandardPostingsReader extends PostingsReaderBase {
private final IndexInput freqIn;
private final IndexInput proxIn;
// public static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
int skipInterval;
int maxSkipLevels;
int skipMinimum;
//private String segment;
// private String segment;
public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext context, int codecId) throws IOException {
public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext ioContext, int codecId) throws IOException {
freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.FREQ_EXTENSION),
context);
//this.segment = segmentInfo.name;
ioContext);
// this.segment = segmentInfo.name;
if (segmentInfo.getHasProx()) {
boolean success = false;
try {
proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.PROX_EXTENSION),
context);
ioContext);
success = true;
} finally {
if (!success) {
@ -73,10 +74,10 @@ public class StandardPostingsReader extends PostingsReaderBase {
}
}
public static void files(Directory dir, SegmentInfo segmentInfo, int id, Collection<String> files) throws IOException {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, StandardCodec.FREQ_EXTENSION));
public static void files(Directory dir, SegmentInfo segmentInfo, int codecID, Collection<String> files) throws IOException {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, StandardCodec.FREQ_EXTENSION));
if (segmentInfo.getHasProx()) {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, StandardCodec.PROX_EXTENSION));
files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, StandardCodec.PROX_EXTENSION));
}
}
@ -100,7 +101,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
// Only used by the "primary" TermState -- clones don't
// copy this (basically they are "transient"):
ByteArrayDataInput bytesReader;
ByteArrayDataInput bytesReader; // TODO: should this NOT be in the TermState...?
byte[] bytes;
@Override
@ -155,7 +156,8 @@ public class StandardPostingsReader extends PostingsReaderBase {
final StandardTermState termState = (StandardTermState) _termState;
final int len = termsIn.readVInt();
//System.out.println("SPR.readTermsBlock termsIn.fp=" + termsIn.getFilePointer());
// if (DEBUG) System.out.println(" SPR.readTermsBlock bytes=" + len + " ts=" + _termState);
if (termState.bytes == null) {
termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
termState.bytesReader = new ByteArrayDataInput();
@ -171,21 +173,25 @@ public class StandardPostingsReader extends PostingsReaderBase {
public void nextTerm(FieldInfo fieldInfo, BlockTermState _termState)
throws IOException {
final StandardTermState termState = (StandardTermState) _termState;
//System.out.println("StandardR.nextTerm seg=" + segment);
final boolean isFirstTerm = termState.termCount == 0;
// if (DEBUG) System.out.println("SPR: nextTerm seg=" + segment + " tbOrd=" + termState.termBlockOrd + " bytesReader.fp=" + termState.bytesReader.getPosition());
final boolean isFirstTerm = termState.termBlockOrd == 0;
if (isFirstTerm) {
termState.freqOffset = termState.bytesReader.readVLong();
} else {
termState.freqOffset += termState.bytesReader.readVLong();
}
//System.out.println(" dF=" + termState.docFreq);
//System.out.println(" freqFP=" + termState.freqOffset);
/*
if (DEBUG) {
System.out.println(" dF=" + termState.docFreq);
System.out.println(" freqFP=" + termState.freqOffset);
}
*/
assert termState.freqOffset < freqIn.length();
if (termState.docFreq >= skipMinimum) {
termState.skipOffset = termState.bytesReader.readVInt();
//System.out.println(" skipOffset=" + termState.skipOffset + " vs freqIn.length=" + freqIn.length());
// if (DEBUG) System.out.println(" skipOffset=" + termState.skipOffset + " vs freqIn.length=" + freqIn.length());
assert termState.freqOffset + termState.skipOffset < freqIn.length();
} else {
// undefined
@ -197,7 +203,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
} else {
termState.proxOffset += termState.bytesReader.readVLong();
}
//System.out.println(" proxFP=" + termState.proxOffset);
// if (DEBUG) System.out.println(" proxFP=" + termState.proxOffset);
}
}
@ -215,6 +221,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
docsEnum = new SegmentDocsEnum(freqIn);
}
}
// if (DEBUG) System.out.println("SPR.docs ts=" + termState);
return docsEnum.reset(fieldInfo, (StandardTermState) termState, liveDocs);
}
@ -300,7 +307,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
assert limit > 0;
ord = 0;
doc = 0;
//System.out.println(" sde limit=" + limit + " freqFP=" + freqOffset);
// if (DEBUG) System.out.println(" sde limit=" + limit + " freqFP=" + freqOffset);
skipped = false;
@ -309,8 +316,10 @@ public class StandardPostingsReader extends PostingsReaderBase {
@Override
public int nextDoc() throws IOException {
//if (DEBUG) System.out.println(" stpr.nextDoc seg=" + segment + " fp=" + freqIn.getFilePointer());
while(true) {
if (ord == limit) {
//if (DEBUG) System.out.println(" return doc=" + NO_MORE_DOCS);
return doc = NO_MORE_DOCS;
}
@ -318,6 +327,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
// Decode next doc/freq pair
final int code = freqIn.readVInt();
// if (DEBUG) System.out.println(" code=" + code);
if (omitTF) {
doc += code;
} else {
@ -334,6 +344,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
}
}
//if (DEBUG) System.out.println(" stpr.nextDoc return doc=" + doc);
return doc;
}
@ -480,16 +491,17 @@ public class StandardPostingsReader extends PostingsReaderBase {
freqOffset = termState.freqOffset;
proxOffset = termState.proxOffset;
skipOffset = termState.skipOffset;
//System.out.println("StandardR.D&PE reset seg=" + segment + " limit=" + limit + " freqFP=" + freqOffset + " proxFP=" + proxOffset);
// if (DEBUG) System.out.println("StandardR.D&PE reset seg=" + segment + " limit=" + limit + " freqFP=" + freqOffset + " proxFP=" + proxOffset);
return this;
}
@Override
public int nextDoc() throws IOException {
// if (DEBUG) System.out.println("SPR.nextDoc seg=" + segment + " freqIn.fp=" + freqIn.getFilePointer());
while(true) {
if (ord == limit) {
//System.out.println("StandardR.D&PE seg=" + segment + " nextDoc return doc=END");
// if (DEBUG) System.out.println(" return END");
return doc = NO_MORE_DOCS;
}
@ -513,7 +525,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
position = 0;
//System.out.println("StandardR.D&PE nextDoc seg=" + segment + " return doc=" + doc);
// if (DEBUG) System.out.println(" return doc=" + doc);
return doc;
}

View File

@ -21,6 +21,8 @@ package org.apache.lucene.index.codecs.standard;
* index file format */
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DocsEnum;
@ -34,18 +36,19 @@ import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.util.IOUtils;
/** @lucene.experimental */
public final class StandardPostingsWriter extends PostingsWriterBase {
final static String CODEC = "StandardPostingsWriterImpl";
final static String CODEC = "StandardPostingsWriter";
//private static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
// Increment version to change it:
final static int VERSION_START = 0;
final static int VERSION_CURRENT = VERSION_START;
IndexOutput freqOut;
IndexOutput proxOut;
final IndexOutput freqOut;
final IndexOutput proxOut;
final DefaultSkipListWriter skipListWriter;
/** Expert: The fraction of TermDocs entries stored in skip tables,
* used to accelerate {@link DocsEnum#advance(int)}. Larger values result in
@ -70,52 +73,42 @@ public final class StandardPostingsWriter extends PostingsWriterBase {
IndexOptions indexOptions;
boolean storePayloads;
// Starts a new term
long lastFreqStart;
long freqStart;
long lastProxStart;
long proxStart;
FieldInfo fieldInfo;
int lastPayloadLength;
int lastPosition;
private int pendingCount;
//private String segment;
private RAMOutputStream bytesWriter = new RAMOutputStream();
// private String segment;
public StandardPostingsWriter(SegmentWriteState state) throws IOException {
this(state, DEFAULT_SKIP_INTERVAL);
}
public StandardPostingsWriter(SegmentWriteState state, int skipInterval) throws IOException {
super();
this.skipInterval = skipInterval;
this.skipMinimum = skipInterval; /* set to the same for now */
//this.segment = state.segmentName;
// this.segment = state.segmentName;
String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.FREQ_EXTENSION);
freqOut = state.directory.createOutput(fileName, state.context);
boolean success = false;
try {
if (state.fieldInfos.hasProx()) {
// At least one field does not omit TF, so create the
// prox file
fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.PROX_EXTENSION);
proxOut = state.directory.createOutput(fileName, state.context);
} else {
// Every field omits TF so we will write no prox file
proxOut = null;
}
totalNumDocs = state.numDocs;
skipListWriter = new DefaultSkipListWriter(skipInterval, maxSkipLevels,
state.numDocs, freqOut, proxOut);
success = true;
} finally {
if (!success) {
IOUtils.closeSafely(true, freqOut, proxOut);
}
if (state.fieldInfos.hasProx()) {
// At least one field does not omit TF, so create the
// prox file
fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.PROX_EXTENSION);
proxOut = state.directory.createOutput(fileName, state.context);
} else {
// Every field omits TF so we will write no prox file
proxOut = null;
}
totalNumDocs = state.numDocs;
skipListWriter = new DefaultSkipListWriter(skipInterval,
maxSkipLevels,
state.numDocs,
freqOut,
proxOut);
}
@Override
@ -129,8 +122,8 @@ public final class StandardPostingsWriter extends PostingsWriterBase {
@Override
public void startTerm() {
//System.out.println("StandardW: startTerm seg=" + segment + " pendingCount=" + pendingCount);
freqStart = freqOut.getFilePointer();
//if (DEBUG) System.out.println("SPW: startTerm freqOut.fp=" + freqStart);
if (proxOut != null) {
proxStart = proxOut.getFilePointer();
// force first payload to write its length
@ -144,6 +137,13 @@ public final class StandardPostingsWriter extends PostingsWriterBase {
@Override
public void setField(FieldInfo fieldInfo) {
//System.out.println("SPW: setField");
/*
if (BlockTreeTermsWriter.DEBUG && fieldInfo.name.equals("id")) {
DEBUG = true;
} else {
DEBUG = false;
}
*/
this.fieldInfo = fieldInfo;
indexOptions = fieldInfo.indexOptions;
storePayloads = fieldInfo.storePayloads;
@ -158,7 +158,7 @@ public final class StandardPostingsWriter extends PostingsWriterBase {
* then we just skip consuming positions/payloads. */
@Override
public void startDoc(int docID, int termDocFreq) throws IOException {
//System.out.println("StandardW: startDoc seg=" + segment + " docID=" + docID + " tf=" + termDocFreq);
// if (DEBUG) System.out.println("SPW: startDoc seg=" + segment + " docID=" + docID + " tf=" + termDocFreq + " freqOut.fp=" + freqOut.getFilePointer());
final int delta = docID - lastDocID;
@ -189,13 +189,13 @@ public final class StandardPostingsWriter extends PostingsWriterBase {
/** Add a new position & payload */
@Override
public void addPosition(int position, BytesRef payload) throws IOException {
//System.out.println("StandardW: addPos pos=" + position + " payload=" + (payload == null ? "null" : (payload.length + " bytes")) + " proxFP=" + proxOut.getFilePointer());
//if (DEBUG) System.out.println("SPW: addPos pos=" + position + " payload=" + (payload == null ? "null" : (payload.length + " bytes")) + " proxFP=" + proxOut.getFilePointer());
assert indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS: "invalid indexOptions: " + indexOptions;
assert proxOut != null;
final int delta = position - lastPosition;
assert delta >= 0: "position=" + position + " lastPosition=" + lastPosition;
assert delta >= 0: "position=" + position + " lastPosition=" + lastPosition; // not quite right (if pos=0 is repeated twice we don't catch it)
lastPosition = position;
@ -222,57 +222,104 @@ public final class StandardPostingsWriter extends PostingsWriterBase {
public void finishDoc() {
}
private static class PendingTerm {
public final long freqStart;
public final long proxStart;
public final int skipOffset;
public PendingTerm(long freqStart, long proxStart, int skipOffset) {
this.freqStart = freqStart;
this.proxStart = proxStart;
this.skipOffset = skipOffset;
}
}
private final List<PendingTerm> pendingTerms = new ArrayList<PendingTerm>();
/** Called when we are done adding docs to this term */
@Override
public void finishTerm(TermStats stats) throws IOException {
//System.out.println("StandardW.finishTerm seg=" + segment);
// if (DEBUG) System.out.println("SPW: finishTerm seg=" + segment + " freqStart=" + freqStart);
assert stats.docFreq > 0;
// TODO: wasteful we are counting this (counting # docs
// for this term) in two places?
assert stats.docFreq == df;
final boolean isFirstTerm = pendingCount == 0;
//System.out.println(" isFirstTerm=" + isFirstTerm);
//System.out.println(" freqFP=" + freqStart);
if (isFirstTerm) {
bytesWriter.writeVLong(freqStart);
} else {
bytesWriter.writeVLong(freqStart-lastFreqStart);
}
lastFreqStart = freqStart;
final int skipOffset;
if (df >= skipMinimum) {
bytesWriter.writeVInt((int) (skipListWriter.writeSkip(freqOut)-freqStart));
skipOffset = (int) (skipListWriter.writeSkip(freqOut)-freqStart);
} else {
skipOffset = -1;
}
if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
//System.out.println(" proxFP=" + proxStart);
if (isFirstTerm) {
bytesWriter.writeVLong(proxStart);
} else {
bytesWriter.writeVLong(proxStart - lastProxStart);
}
lastProxStart = proxStart;
}
pendingTerms.add(new PendingTerm(freqStart, proxStart, skipOffset));
lastDocID = 0;
df = 0;
pendingCount++;
}
private final RAMOutputStream bytesWriter = new RAMOutputStream();
@Override
public void flushTermsBlock() throws IOException {
//System.out.println("SPW.flushBlock pendingCount=" + pendingCount);
public void flushTermsBlock(int start, int count) throws IOException {
//if (DEBUG) System.out.println("SPW: flushTermsBlock start=" + start + " count=" + count + " left=" + (pendingTerms.size()-count) + " pendingTerms.size()=" + pendingTerms.size());
if (count == 0) {
termsOut.writeByte((byte) 0);
return;
}
assert start <= pendingTerms.size();
assert count <= start;
final int limit = pendingTerms.size() - start + count;
final PendingTerm firstTerm = pendingTerms.get(limit - count);
// First term in block is abs coded:
bytesWriter.writeVLong(firstTerm.freqStart);
if (firstTerm.skipOffset != -1) {
assert firstTerm.skipOffset > 0;
bytesWriter.writeVInt(firstTerm.skipOffset);
}
if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
bytesWriter.writeVLong(firstTerm.proxStart);
}
long lastFreqStart = firstTerm.freqStart;
long lastProxStart = firstTerm.proxStart;
for(int idx=limit-count+1; idx<limit; idx++) {
final PendingTerm term = pendingTerms.get(idx);
//if (DEBUG) System.out.println(" write term freqStart=" + term.freqStart);
// The rest of the terms term are delta coded:
bytesWriter.writeVLong(term.freqStart - lastFreqStart);
lastFreqStart = term.freqStart;
if (term.skipOffset != -1) {
assert term.skipOffset > 0;
bytesWriter.writeVInt(term.skipOffset);
}
if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
bytesWriter.writeVLong(term.proxStart - lastProxStart);
lastProxStart = term.proxStart;
}
}
termsOut.writeVInt((int) bytesWriter.getFilePointer());
bytesWriter.writeTo(termsOut);
bytesWriter.reset();
pendingCount = 0;
// Remove the terms we just wrote:
pendingTerms.subList(limit-count, limit).clear();
}
@Override
public void close() throws IOException {
IOUtils.closeSafely(false, freqOut, proxOut);
try {
freqOut.close();
} finally {
if (proxOut != null) {
proxOut.close();
}
}
}
}

View File

@ -22,14 +22,11 @@ import java.io.IOException;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.AutomatonTermsEnum.CompiledAutomaton;
import org.apache.lucene.util.ToStringUtils;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.ToStringUtils;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.BasicAutomata;
import org.apache.lucene.util.automaton.BasicOperations;
import org.apache.lucene.util.automaton.MinimizationOperations;
import org.apache.lucene.util.automaton.SpecialOperations;
import org.apache.lucene.util.automaton.CompiledAutomaton;
/**
* A {@link Query} that will match terms against a finite-state machine.
@ -41,7 +38,7 @@ import org.apache.lucene.util.automaton.SpecialOperations;
* the standard Lucene wildcard syntax with {@link WildcardQuery}.
* </p>
* <p>
* When the query is executed, it will create an equivalent minimal DFA of the
* When the query is executed, it will create an equivalent DFA of the
* finite-state machine, and will enumerate the term dictionary in an
* intelligent way to reduce the number of comparisons. For example: the regular
* expression of <code>[dl]og?</code> will make approximately four comparisons:
@ -52,20 +49,10 @@ import org.apache.lucene.util.automaton.SpecialOperations;
public class AutomatonQuery extends MultiTermQuery {
/** the automaton to match index terms against */
protected final Automaton automaton;
protected final CompiledAutomaton compiled;
/** term containing the field, and possibly some pattern structure */
protected final Term term;
/**
* abstraction for returning a termsenum:
* in the ctor the query computes one of these, the actual
* implementation depends upon the automaton's structure.
*/
private abstract class TermsEnumFactory {
protected abstract TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException;
}
private final TermsEnumFactory factory;
/**
* Create a new AutomatonQuery from an {@link Automaton}.
*
@ -78,73 +65,12 @@ public class AutomatonQuery extends MultiTermQuery {
super(term.field());
this.term = term;
this.automaton = automaton;
MinimizationOperations.minimize(automaton);
if (BasicOperations.isEmpty(automaton)) {
// matches nothing
factory = new TermsEnumFactory() {
@Override
protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
return TermsEnum.EMPTY;
}
};
} else if (BasicOperations.isTotal(automaton)) {
// matches all possible strings
factory = new TermsEnumFactory() {
@Override
protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
return terms.iterator();
}
};
} else {
final String singleton;
final String commonPrefix;
if (automaton.getSingleton() == null) {
commonPrefix = SpecialOperations.getCommonPrefix(automaton);
if (commonPrefix.length() > 0 && BasicOperations.sameLanguage(automaton, BasicAutomata.makeString(commonPrefix))) {
singleton = commonPrefix;
} else {
singleton = null;
}
} else {
commonPrefix = null;
singleton = automaton.getSingleton();
}
if (singleton != null) {
// matches a fixed string in singleton or expanded representation
factory = new TermsEnumFactory() {
@Override
protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
return new SingleTermsEnum(terms.iterator(), new Term(field, singleton));
}
};
} else if (BasicOperations.sameLanguage(automaton, BasicOperations.concatenate(
BasicAutomata.makeString(commonPrefix), BasicAutomata.makeAnyString()))) {
// matches a constant prefix
factory = new TermsEnumFactory() {
@Override
protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
return new PrefixTermsEnum(terms.iterator(), new Term(field, commonPrefix));
}
};
} else {
final AutomatonTermsEnum.CompiledAutomaton compiled =
new CompiledAutomaton(automaton, SpecialOperations.isFinite(automaton));
factory = new TermsEnumFactory() {
@Override
protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
return new AutomatonTermsEnum(terms.iterator(), compiled);
}
};
}
}
this.compiled = new CompiledAutomaton(automaton);
}
@Override
protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
return factory.getTermsEnum(terms, atts);
return compiled.getTermsEnum(terms);
}
@Override

View File

@ -28,6 +28,8 @@ import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.Bits;
// TODO: move this class to oal.index
/**
* Abstract class for enumerating a subset of all terms.
*
@ -42,7 +44,7 @@ import org.apache.lucene.util.Bits;
public abstract class FilteredTermsEnum extends TermsEnum {
private BytesRef initialSeekTerm = null;
private boolean doSeek = true;
private boolean doSeek;
private BytesRef actualTerm = null;
private final TermsEnum tenum;
@ -64,8 +66,17 @@ public abstract class FilteredTermsEnum extends TermsEnum {
* @param tenum the terms enumeration to filter.
*/
public FilteredTermsEnum(final TermsEnum tenum) {
this(tenum, true);
}
/**
* Creates a filtered {@link TermsEnum} on a terms enum.
* @param tenum the terms enumeration to filter.
*/
public FilteredTermsEnum(final TermsEnum tenum, final boolean startWithSeek) {
assert tenum != null;
this.tenum = tenum;
doSeek = startWithSeek;
}
/**
@ -190,18 +201,23 @@ public abstract class FilteredTermsEnum extends TermsEnum {
@SuppressWarnings("fallthrough")
@Override
public BytesRef next() throws IOException {
//System.out.println("FTE.next doSeek=" + doSeek);
//new Throwable().printStackTrace(System.out);
for (;;) {
// Seek or forward the iterator
if (doSeek) {
doSeek = false;
final BytesRef t = nextSeekTerm(actualTerm);
//System.out.println(" seek to t=" + (t == null ? "null" : t.utf8ToString()) + " tenum=" + tenum);
// Make sure we always seek forward:
assert actualTerm == null || t == null || getComparator().compare(t, actualTerm) > 0: "curTerm=" + actualTerm + " seekTerm=" + t;
if (t == null || tenum.seekCeil(t, false) == SeekStatus.END) {
// no more terms to seek to or enum exhausted
//System.out.println(" return null");
return null;
}
actualTerm = tenum.term();
//System.out.println(" got term=" + actualTerm.utf8ToString());
} else {
actualTerm = tenum.next();
if (actualTerm == null) {

View File

@ -137,12 +137,10 @@ public class FuzzyQuery extends MultiTermQuery {
@Override
protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
TermsEnum tenum = terms.iterator();
if (!termLongEnough) { // can only match if it's exact
return new SingleTermsEnum(tenum, term);
return new SingleTermsEnum(terms.iterator(), term.bytes());
}
return new FuzzyTermsEnum(tenum, atts, getTerm(), minimumSimilarity, prefixLength);
return new FuzzyTermsEnum(terms, atts, getTerm(), minimumSimilarity, prefixLength);
}
/**

View File

@ -17,12 +17,17 @@ package org.apache.lucene.search;
* limitations under the License.
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.List;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.AutomatonTermsEnum.CompiledAutomaton;
import org.apache.lucene.util.Attribute;
import org.apache.lucene.util.AttributeImpl;
import org.apache.lucene.util.AttributeSource;
@ -34,13 +39,9 @@ import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.BasicAutomata;
import org.apache.lucene.util.automaton.BasicOperations;
import org.apache.lucene.util.automaton.ByteRunAutomaton;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.LevenshteinAutomata;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.List;
/** Subclass of TermsEnum for enumerating all terms that are similar
* to the specified filter term.
*
@ -72,7 +73,7 @@ public final class FuzzyTermsEnum extends TermsEnum {
private int maxEdits;
private final boolean raw;
private final TermsEnum tenum;
private final Terms terms;
private final Term term;
private final int termText[];
private final int realPrefixLength;
@ -94,7 +95,7 @@ public final class FuzzyTermsEnum extends TermsEnum {
* @param prefixLength Length of required common prefix. Default value is 0.
* @throws IOException
*/
public FuzzyTermsEnum(TermsEnum tenum, AttributeSource atts, Term term,
public FuzzyTermsEnum(Terms terms, AttributeSource atts, Term term,
final float minSimilarity, final int prefixLength) throws IOException {
if (minSimilarity >= 1.0f && minSimilarity != (int)minSimilarity)
throw new IllegalArgumentException("fractional edit distances are not allowed");
@ -102,7 +103,7 @@ public final class FuzzyTermsEnum extends TermsEnum {
throw new IllegalArgumentException("minimumSimilarity cannot be less than 0");
if(prefixLength < 0)
throw new IllegalArgumentException("prefixLength cannot be less than 0");
this.tenum = tenum;
this.terms = terms;
this.term = term;
// convert the string into a utf32 int[] representation for fast comparisons
@ -143,8 +144,10 @@ public final class FuzzyTermsEnum extends TermsEnum {
throws IOException {
final List<CompiledAutomaton> runAutomata = initAutomata(editDistance);
if (editDistance < runAutomata.size()) {
return new AutomatonFuzzyTermsEnum(runAutomata.subList(0, editDistance + 1)
.toArray(new CompiledAutomaton[editDistance + 1]), lastTerm);
//if (BlockTreeTermsWriter.DEBUG) System.out.println("FuzzyTE.getAEnum: ed=" + editDistance + " lastTerm=" + (lastTerm==null ? "null" : lastTerm.utf8ToString()));
final CompiledAutomaton compiled = runAutomata.get(editDistance);
return new AutomatonFuzzyTermsEnum(terms.intersect(compiled, lastTerm == null ? null : compiled.floor(lastTerm, new BytesRef())),
runAutomata.subList(0, editDistance + 1).toArray(new CompiledAutomaton[editDistance + 1]));
} else {
return null;
}
@ -153,6 +156,7 @@ public final class FuzzyTermsEnum extends TermsEnum {
/** initialize levenshtein DFAs up to maxDistance, if possible */
private List<CompiledAutomaton> initAutomata(int maxDistance) {
final List<CompiledAutomaton> runAutomata = dfaAtt.automata();
//System.out.println("cached automata size: " + runAutomata.size());
if (runAutomata.size() <= maxDistance &&
maxDistance <= LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE) {
LevenshteinAutomata builder =
@ -160,13 +164,14 @@ public final class FuzzyTermsEnum extends TermsEnum {
for (int i = runAutomata.size(); i <= maxDistance; i++) {
Automaton a = builder.toAutomaton(i);
//System.out.println("compute automaton n=" + i);
// constant prefix
if (realPrefixLength > 0) {
Automaton prefix = BasicAutomata.makeString(
UnicodeUtil.newString(termText, 0, realPrefixLength));
a = BasicOperations.concatenate(prefix, a);
}
runAutomata.add(new CompiledAutomaton(a, true));
runAutomata.add(new CompiledAutomaton(a, true, false));
}
}
return runAutomata;
@ -303,63 +308,63 @@ public final class FuzzyTermsEnum extends TermsEnum {
}
/**
* Implement fuzzy enumeration with automaton.
* Implement fuzzy enumeration with Terms.intersect.
* <p>
* This is the fastest method as opposed to LinearFuzzyTermsEnum:
* as enumeration is logarithmic to the number of terms (instead of linear)
* and comparison is linear to length of the term (rather than quadratic)
*/
private class AutomatonFuzzyTermsEnum extends AutomatonTermsEnum {
private class AutomatonFuzzyTermsEnum extends FilteredTermsEnum {
private final ByteRunAutomaton matchers[];
private final BytesRef termRef;
private final BytesRef lastTerm;
private final BoostAttribute boostAtt =
attributes().addAttribute(BoostAttribute.class);
public AutomatonFuzzyTermsEnum(CompiledAutomaton compiled[],
BytesRef lastTerm) throws IOException {
super(tenum, compiled[compiled.length - 1]);
public AutomatonFuzzyTermsEnum(TermsEnum tenum, CompiledAutomaton compiled[])
throws IOException {
super(tenum, false);
this.matchers = new ByteRunAutomaton[compiled.length];
for (int i = 0; i < compiled.length; i++)
this.matchers[i] = compiled[i].runAutomaton;
this.lastTerm = lastTerm;
termRef = new BytesRef(term.text());
}
/** finds the smallest Lev(n) DFA that accepts the term. */
@Override
protected AcceptStatus accept(BytesRef term) {
//System.out.println("AFTE.accept term=" + term);
int ed = matchers.length - 1;
if (matches(term, ed)) { // we match the outer dfa
// now compute exact edit distance
while (ed > 0) {
if (matches(term, ed - 1)) {
ed--;
} else {
break;
}
}
// scale to a boost and return (if similarity > minSimilarity)
if (ed == 0) { // exact match
boostAtt.setBoost(1.0F);
return AcceptStatus.YES_AND_SEEK;
// we are wrapping either an intersect() TermsEnum or an AutomatonTermsENum,
// so we know the outer DFA always matches.
// now compute exact edit distance
while (ed > 0) {
if (matches(term, ed - 1)) {
ed--;
} else {
final int codePointCount = UnicodeUtil.codePointCount(term);
final float similarity = 1.0f - ((float) ed / (float)
(Math.min(codePointCount, termLength)));
if (similarity > minSimilarity) {
boostAtt.setBoost((similarity - minSimilarity) * scale_factor);
return AcceptStatus.YES_AND_SEEK;
} else {
return AcceptStatus.NO_AND_SEEK;
}
break;
}
}
//System.out.println("CHECK term=" + term.utf8ToString() + " ed=" + ed);
// scale to a boost and return (if similarity > minSimilarity)
if (ed == 0) { // exact match
boostAtt.setBoost(1.0F);
//System.out.println(" yes");
return AcceptStatus.YES;
} else {
return AcceptStatus.NO_AND_SEEK;
final int codePointCount = UnicodeUtil.codePointCount(term);
final float similarity = 1.0f - ((float) ed / (float)
(Math.min(codePointCount, termLength)));
if (similarity > minSimilarity) {
boostAtt.setBoost((similarity - minSimilarity) * scale_factor);
//System.out.println(" yes");
return AcceptStatus.YES;
} else {
return AcceptStatus.NO;
}
}
}
@ -367,14 +372,6 @@ public final class FuzzyTermsEnum extends TermsEnum {
final boolean matches(BytesRef term, int k) {
return k == 0 ? term.equals(termRef) : matchers[k].run(term.bytes, term.offset, term.length);
}
/** defers to superclass, except can start at an arbitrary location */
@Override
protected BytesRef nextSeekTerm(BytesRef term) throws IOException {
if (term == null)
term = lastTerm;
return super.nextSeekTerm(term);
}
}
/**
@ -408,7 +405,7 @@ public final class FuzzyTermsEnum extends TermsEnum {
* @throws IOException
*/
public LinearFuzzyTermsEnum() throws IOException {
super(tenum);
super(terms.iterator());
this.text = new int[termLength - realPrefixLength];
System.arraycopy(termText, realPrefixLength, text, 0, text.length);

View File

@ -882,6 +882,6 @@ public class IndexSearcher implements Closeable {
@Override
public String toString() {
return "IndexSearcher(" + reader + ")";
return "IndexSearcher(" + reader + "; executor=" + executor + ")";
}
}

View File

@ -51,7 +51,7 @@ public class PrefixQuery extends MultiTermQuery {
// no prefix -- match all terms for this field:
return tenum;
}
return new PrefixTermsEnum(tenum, prefix);
return new PrefixTermsEnum(tenum, prefix.bytes());
}
/** Prints a user-readable version of this query. */

View File

@ -19,7 +19,6 @@ package org.apache.lucene.search;
import java.io.IOException;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.BytesRef;
@ -34,9 +33,9 @@ public class PrefixTermsEnum extends FilteredTermsEnum {
private final BytesRef prefixRef;
public PrefixTermsEnum(TermsEnum tenum, Term prefix) throws IOException {
public PrefixTermsEnum(TermsEnum tenum, BytesRef prefixText) throws IOException {
super(tenum);
setInitialSeekTerm(prefixRef = prefix.bytes());
setInitialSeekTerm(this.prefixRef = prefixText);
}
@Override

View File

@ -19,7 +19,6 @@ package org.apache.lucene.search;
import java.io.IOException;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.BytesRef;
@ -39,10 +38,10 @@ public final class SingleTermsEnum extends FilteredTermsEnum {
* After calling the constructor the enumeration is already pointing to the term,
* if it exists.
*/
public SingleTermsEnum(TermsEnum tenum, Term singleTerm) throws IOException {
public SingleTermsEnum(TermsEnum tenum, BytesRef termText) throws IOException {
super(tenum);
singleRef = singleTerm.bytes();
setInitialSeekTerm(singleRef);
singleRef = termText;
setInitialSeekTerm(termText);
}
@Override

View File

@ -98,17 +98,18 @@ public class TermQuery extends Query {
TermsEnum getTermsEnum(AtomicReaderContext context) throws IOException {
final TermState state = termStates.get(context.ord);
if (state == null) { // term is not present in that reader
assert termNotInReader(context.reader, term.field(), term.bytes()) : "no termstate found but term exists in reader";
assert termNotInReader(context.reader, term.field(), term.bytes()) : "no termstate found but term exists in reader term=" + term;
return null;
}
final TermsEnum termsEnum = context.reader.terms(term.field())
.getThreadTermsEnum();
//System.out.println("LD=" + reader.getLiveDocs() + " set?=" + (reader.getLiveDocs() != null ? reader.getLiveDocs().get(0) : "null"));
final TermsEnum termsEnum = context.reader.terms(term.field()).getThreadTermsEnum();
termsEnum.seekExact(term.bytes(), state);
return termsEnum;
}
private boolean termNotInReader(IndexReader reader, String field, BytesRef bytes) throws IOException {
// only called from assert
//System.out.println("TQ.termNotInReader reader=" + reader + " term=" + field + ":" + bytes.utf8ToString());
final Terms terms = reader.terms(field);
return terms == null || terms.docFreq(bytes) == 0;
}

View File

@ -70,6 +70,7 @@ final class TermScorer extends Scorer {
public boolean score(Collector c, int end, int firstDocID) throws IOException {
c.setScorer(this);
while (doc < end) { // for docs in window
//System.out.println("TS: collect doc=" + doc);
c.collect(doc); // collect score
if (++pointer >= pointerMax) {
refillBuffer();

View File

@ -78,15 +78,38 @@ public abstract class TopTermsRewrite<Q extends Query> extends TermCollectingRew
public void setNextEnum(TermsEnum termsEnum) throws IOException {
this.termsEnum = termsEnum;
this.termComp = termsEnum.getComparator();
assert compareToLastTerm(null);
// lazy init the initial ScoreTerm because comparator is not known on ctor:
if (st == null)
st = new ScoreTerm(this.termComp, new TermContext(topReaderContext));
boostAtt = termsEnum.attributes().addAttribute(BoostAttribute.class);
}
// for assert:
private BytesRef lastTerm;
private boolean compareToLastTerm(BytesRef t) throws IOException {
if (lastTerm == null && t != null) {
lastTerm = new BytesRef(t);
} else if (t == null) {
lastTerm = null;
} else {
assert termsEnum.getComparator().compare(lastTerm, t) < 0: "lastTerm=" + lastTerm + " t=" + t;
lastTerm.copy(t);
}
return true;
}
@Override
public boolean collect(BytesRef bytes) throws IOException {
final float boost = boostAtt.getBoost();
// make sure within a single seg we always collect
// terms in order
assert compareToLastTerm(bytes);
//System.out.println("TTR.collect term=" + bytes.utf8ToString() + " boost=" + boost + " ord=" + readerContext.ord);
// ignore uncompetitive hits
if (stQueue.size() == maxSize) {
final ScoreTerm t = stQueue.peek();
@ -134,9 +157,10 @@ public abstract class TopTermsRewrite<Q extends Query> extends TermCollectingRew
final Q q = getTopLevelQuery();
final ScoreTerm[] scoreTerms = stQueue.toArray(new ScoreTerm[stQueue.size()]);
ArrayUtil.mergeSort(scoreTerms, scoreTermSortByTermComp);
for (final ScoreTerm st : scoreTerms) {
final Term term = new Term(query.field, st.bytes);
assert reader.docFreq(term) == st.termState.docFreq() : "reader DF is " + reader.docFreq(term) + " vs " + st.termState.docFreq();
assert reader.docFreq(term) == st.termState.docFreq() : "reader DF is " + reader.docFreq(term) + " vs " + st.termState.docFreq() + " term=" + term;
addClause(q, term, st.termState.docFreq(), query.getBoost() * st.boost, st.termState); // add to query
}
query.incTotalNumberOfTerms(scoreTerms.length);

View File

@ -43,16 +43,30 @@ public final class ByteArrayDataInput extends DataInput {
reset(bytes, 0, bytes.length);
}
// NOTE: sets pos to 0, which is not right if you had
// called reset w/ non-zero offset!!
public void rewind() {
pos = 0;
}
public int getPosition() {
return pos;
}
public void setPosition(int pos) {
this.pos = pos;
}
public void reset(byte[] bytes, int offset, int len) {
this.bytes = bytes;
pos = offset;
limit = offset + len;
}
public int length() {
return limit;
}
public boolean eof() {
return pos == limit;
}

View File

@ -19,10 +19,6 @@ package org.apache.lucene.store;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.Lock;
import org.apache.lucene.util.IOUtils;
import java.util.Collection;
@ -189,14 +185,14 @@ public abstract class CompoundFileDirectory extends Directory {
}
@Override
public synchronized IndexInput openInput(String id, IOContext context) throws IOException {
public synchronized IndexInput openInput(String fileName, IOContext context) throws IOException {
ensureOpen();
assert !openForWrite;
id = IndexFileNames.stripSegmentName(id);
final String id = IndexFileNames.stripSegmentName(fileName);
final FileEntry entry = entries.get(id);
if (entry == null)
throw new IOException("No sub-file with id " + id + " found (files: " + entries.keySet() + ")");
if (entry == null) {
throw new IOException("No sub-file with id " + id + " found (fileName=" + fileName + " files: " + entries.keySet() + ")");
}
return openInputSlice(id, entry.offset, entry.length, readBufferSize);
}

View File

@ -448,6 +448,7 @@ public abstract class FSDirectory extends Directory {
/** output methods: */
@Override
public void flushBuffer(byte[] b, int offset, int size) throws IOException {
assert isOpen;
if (rateLimiter != null) {
rateLimiter.pause(size);
}

View File

@ -65,6 +65,18 @@ public final class BytesRef implements Comparable<BytesRef> {
this.bytes = new byte[capacity];
}
/** Incoming IntsRef values must be Byte.MIN_VALUE -
* Byte.MAX_VALUE. */
public BytesRef(IntsRef intsRef) {
bytes = new byte[intsRef.length];
for(int idx=0;idx<intsRef.length;idx++) {
final int v = intsRef.ints[intsRef.offset + idx];
assert v >= Byte.MIN_VALUE && v <= Byte.MAX_VALUE;
bytes[idx] = (byte) v;
}
length = intsRef.length;
}
/**
* @param text Initialize the byte[] from the UTF8 bytes
* for the provided Sring. This must be well-formed

View File

@ -21,14 +21,13 @@ import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.IndexReader.ReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.IndexReader.ReaderContext;
import org.apache.lucene.index.TermsEnum.SeekStatus;
/**
* Maintains a {@link IndexReader} {@link TermState} view over
@ -45,6 +44,9 @@ public final class TermContext {
private int docFreq;
private long totalTermFreq;
//public static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
/**
* Creates an empty {@link TermContext} from a {@link ReaderContext}
*/
@ -85,7 +87,9 @@ public final class TermContext {
final BytesRef bytes = term.bytes();
final TermContext perReaderTermState = new TermContext(context);
final AtomicReaderContext[] leaves = ReaderUtil.leaves(context);
//if (DEBUG) System.out.println("prts.build term=" + term);
for (int i = 0; i < leaves.length; i++) {
//if (DEBUG) System.out.println(" r=" + leaves[i].reader);
final Fields fields = leaves[i].reader.fields();
if (fields != null) {
final Terms terms = fields.terms(field);
@ -93,6 +97,7 @@ public final class TermContext {
final TermsEnum termsEnum = terms.getThreadTermsEnum(); // thread-private don't share!
if (termsEnum.seekExact(bytes, cache)) {
final TermState termState = termsEnum.termState();
//if (DEBUG) System.out.println(" found");
perReaderTermState.register(termState, leaves[i].ord, termsEnum.docFreq(), termsEnum.totalTermFreq());
}
}

View File

@ -0,0 +1,313 @@
package org.apache.lucene.util.automaton;
/**
* 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.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.PrefixTermsEnum;
import org.apache.lucene.search.SingleTermsEnum;
import org.apache.lucene.util.BytesRef;
/**
* Immutable class holding compiled details for a given
* Automaton. The Automaton is deterministic, must not have
* dead states but may not be minimal.
*
* @lucene.experimental
*/
public class CompiledAutomaton {
public enum AUTOMATON_TYPE {NONE, ALL, SINGLE, PREFIX, NORMAL};
public final AUTOMATON_TYPE type;
// For PREFIX, this is the prefix term; for SINGLE this is
// the singleton term:
public final BytesRef term;
// NOTE: the next 4 members are only non-null if type ==
// NORMAL:
public final ByteRunAutomaton runAutomaton;
// TODO: would be nice if these sortedTransitions had "int
// to;" instead of "State to;" somehow:
public final Transition[][] sortedTransitions;
public final BytesRef commonSuffixRef;
public final Boolean finite;
public CompiledAutomaton(Automaton automaton) {
this(automaton, null, true);
}
public CompiledAutomaton(Automaton automaton, Boolean finite, boolean simplify) {
if (simplify) {
// Test whether the automaton is a "simple" form and
// if so, don't create a runAutomaton. Note that on a
// large automaton these tests could be costly:
if (BasicOperations.isEmpty(automaton)) {
// matches nothing
type = AUTOMATON_TYPE.NONE;
term = null;
commonSuffixRef = null;
runAutomaton = null;
sortedTransitions = null;
this.finite = null;
return;
} else if (BasicOperations.isTotal(automaton)) {
// matches all possible strings
type = AUTOMATON_TYPE.ALL;
term = null;
commonSuffixRef = null;
runAutomaton = null;
sortedTransitions = null;
this.finite = null;
return;
} else {
final String commonPrefix;
final String singleton;
if (automaton.getSingleton() == null) {
commonPrefix = SpecialOperations.getCommonPrefix(automaton);
if (commonPrefix.length() > 0 && BasicOperations.sameLanguage(automaton, BasicAutomata.makeString(commonPrefix))) {
singleton = commonPrefix;
} else {
singleton = null;
}
} else {
commonPrefix = null;
singleton = automaton.getSingleton();
}
if (singleton != null) {
// matches a fixed string in singleton or expanded
// representation
type = AUTOMATON_TYPE.SINGLE;
term = new BytesRef(singleton);
commonSuffixRef = null;
runAutomaton = null;
sortedTransitions = null;
this.finite = null;
return;
} else if (BasicOperations.sameLanguage(automaton, BasicOperations.concatenate(
BasicAutomata.makeString(commonPrefix), BasicAutomata.makeAnyString()))) {
// matches a constant prefix
type = AUTOMATON_TYPE.PREFIX;
term = new BytesRef(commonPrefix);
commonSuffixRef = null;
runAutomaton = null;
sortedTransitions = null;
this.finite = null;
return;
}
}
}
type = AUTOMATON_TYPE.NORMAL;
term = null;
if (finite == null) {
this.finite = SpecialOperations.isFinite(automaton);
} else {
this.finite = finite;
}
Automaton utf8 = new UTF32ToUTF8().convert(automaton);
if (this.finite) {
commonSuffixRef = null;
} else {
commonSuffixRef = SpecialOperations.getCommonSuffixBytesRef(utf8);
}
runAutomaton = new ByteRunAutomaton(utf8, true);
sortedTransitions = utf8.getSortedTransitions();
}
//private static final boolean DEBUG = BlockTreeTermsWriter.DEBUG;
private BytesRef addTail(int state, BytesRef term, int idx, int leadLabel) {
// Find biggest transition that's < label
// TODO: use binary search here
Transition maxTransition = null;
for (Transition transition : sortedTransitions[state]) {
if (transition.min < leadLabel) {
maxTransition = transition;
}
}
assert maxTransition != null;
// Append floorLabel
final int floorLabel;
if (maxTransition.max > leadLabel-1) {
floorLabel = leadLabel-1;
} else {
floorLabel = maxTransition.max;
}
if (idx >= term.bytes.length) {
term.grow(1+idx);
}
//if (DEBUG) System.out.println(" add floorLabel=" + (char) floorLabel + " idx=" + idx);
term.bytes[idx] = (byte) floorLabel;
state = maxTransition.to.getNumber();
idx++;
// Push down to last accept state
while (true) {
Transition[] transitions = sortedTransitions[state];
if (transitions.length == 0) {
assert runAutomaton.isAccept(state);
term.length = idx;
//if (DEBUG) System.out.println(" return " + term.utf8ToString());
return term;
} else {
// We are pushing "top" -- so get last label of
// last transition:
assert transitions.length != 0;
Transition lastTransition = transitions[transitions.length-1];
if (idx >= term.bytes.length) {
term.grow(1+idx);
}
//if (DEBUG) System.out.println(" push maxLabel=" + (char) lastTransition.max + " idx=" + idx);
term.bytes[idx] = (byte) lastTransition.max;
state = lastTransition.to.getNumber();
idx++;
}
}
}
// TODO: should this take startTerm too? This way
// Terms.intersect could forward to this method if type !=
// NORMAL:
public TermsEnum getTermsEnum(Terms terms) throws IOException {
switch(type) {
case NONE:
return TermsEnum.EMPTY;
case ALL:
return terms.iterator();
case SINGLE:
return new SingleTermsEnum(terms.iterator(), term);
case PREFIX:
// TODO: this is very likely faster than .intersect,
// but we should test and maybe cutover
return new PrefixTermsEnum(terms.iterator(), term);
case NORMAL:
return terms.intersect(this, null);
default:
// unreachable
throw new RuntimeException("unhandled case");
}
}
/** Finds largest term accepted by this Automaton, that's
* <= the provided input term. The result is placed in
* output; it's fine for output and input to point to
* the same BytesRef. The returned result is either the
* provided output, or null if there is no floor term
* (ie, the provided input term is before the first term
* accepted by this Automaton). */
public BytesRef floor(BytesRef input, BytesRef output) {
output.offset = 0;
//if (DEBUG) System.out.println("CA.floor input=" + input.utf8ToString());
int state = runAutomaton.getInitialState();
// Special case empty string:
if (input.length == 0) {
if (runAutomaton.isAccept(state)) {
output.length = 0;
return output;
} else {
return null;
}
}
final List<Integer> stack = new ArrayList<Integer>();
int idx = 0;
while (true) {
int label = input.bytes[input.offset + idx] & 0xff;
int nextState = runAutomaton.step(state, label);
//if (DEBUG) System.out.println(" cycle label=" + (char) label + " nextState=" + nextState);
if (idx == input.length-1) {
if (nextState != -1 && runAutomaton.isAccept(nextState)) {
// Input string is accepted
if (idx >= output.bytes.length) {
output.grow(1+idx);
}
output.bytes[idx] = (byte) label;
output.length = input.length;
//if (DEBUG) System.out.println(" input is accepted; return term=" + output.utf8ToString());
return output;
} else {
nextState = -1;
}
}
if (nextState == -1) {
// Pop back to a state that has a transition
// <= our label:
while (true) {
Transition[] transitions = sortedTransitions[state];
if (transitions.length == 0) {
assert runAutomaton.isAccept(state);
output.length = idx;
//if (DEBUG) System.out.println(" return " + output.utf8ToString());
return output;
} else if (label-1 < transitions[0].min) {
if (runAutomaton.isAccept(state)) {
output.length = idx;
//if (DEBUG) System.out.println(" return " + output.utf8ToString());
return output;
}
// pop
if (stack.size() == 0) {
//if (DEBUG) System.out.println(" pop ord=" + idx + " return null");
return null;
} else {
state = stack.remove(stack.size()-1);
idx--;
//if (DEBUG) System.out.println(" pop ord=" + (idx+1) + " label=" + (char) label + " first trans.min=" + (char) transitions[0].min);
label = input.bytes[input.offset + idx] & 0xff;
}
} else {
//if (DEBUG) System.out.println(" stop pop ord=" + idx + " first trans.min=" + (char) transitions[0].min);
break;
}
}
//if (DEBUG) System.out.println(" label=" + (char) label + " idx=" + idx);
return addTail(state, output, idx, label);
} else {
if (idx >= output.bytes.length) {
output.grow(1+idx);
}
output.bytes[idx] = (byte) label;
stack.add(state);
state = nextState;
idx++;
}
}
}
}

View File

@ -53,6 +53,8 @@ public class Builder<T> {
private final FST<T> fst;
private final T NO_OUTPUT;
// private static final boolean DEBUG = false;
// simplistic pruning: we prune node (and all following
// nodes) if less than this number of terms go through it:
private final int minSuffixCount1;
@ -73,13 +75,21 @@ public class Builder<T> {
// current "frontier"
private UnCompiledNode<T>[] frontier;
// Expert: you pass an instance of this if you want to do
// something "custom" as suffixes are "frozen":
public static abstract class FreezeTail<T> {
public abstract void freeze(final UnCompiledNode<T>[] frontier, int prefixLenPlus1, IntsRef prevInput) throws IOException;
}
private final FreezeTail<T> freezeTail;
/**
* Instantiates an FST/FSA builder without any pruning. A shortcut
* to {@link #Builder(FST.INPUT_TYPE, int, int, boolean, boolean, int, Outputs)} with
* pruning options turned off.
*/
public Builder(FST.INPUT_TYPE inputType, Outputs<T> outputs) {
this(inputType, 0, 0, true, true, Integer.MAX_VALUE, outputs);
this(inputType, 0, 0, true, true, Integer.MAX_VALUE, outputs, null);
}
/**
@ -120,9 +130,11 @@ public class Builder<T> {
* singleton output object.
*/
public Builder(FST.INPUT_TYPE inputType, int minSuffixCount1, int minSuffixCount2, boolean doShareSuffix,
boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs<T> outputs) {
boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs<T> outputs,
FreezeTail<T> freezeTail) {
this.minSuffixCount1 = minSuffixCount1;
this.minSuffixCount2 = minSuffixCount2;
this.freezeTail = freezeTail;
this.doShareNonSingletonNodes = doShareNonSingletonNodes;
this.shareMaxTailLength = shareMaxTailLength;
fst = new FST<T>(inputType, outputs);
@ -179,94 +191,100 @@ public class Builder<T> {
return fn;
}
private void compilePrevTail(int prefixLenPlus1) throws IOException {
assert prefixLenPlus1 >= 1;
//System.out.println(" compileTail " + prefixLenPlus1);
for(int idx=lastInput.length; idx >= prefixLenPlus1; idx--) {
boolean doPrune = false;
boolean doCompile = false;
private void freezeTail(int prefixLenPlus1) throws IOException {
if (freezeTail != null) {
// Custom plugin:
freezeTail.freeze(frontier, prefixLenPlus1, lastInput);
} else {
//System.out.println(" compileTail " + prefixLenPlus1);
final int downTo = Math.max(1, prefixLenPlus1);
for(int idx=lastInput.length; idx >= downTo; idx--) {
final UnCompiledNode<T> node = frontier[idx];
final UnCompiledNode<T> parent = frontier[idx-1];
boolean doPrune = false;
boolean doCompile = false;
if (node.inputCount < minSuffixCount1) {
doPrune = true;
doCompile = true;
} else if (idx > prefixLenPlus1) {
// prune if parent's inputCount is less than suffixMinCount2
if (parent.inputCount < minSuffixCount2 || minSuffixCount2 == 1 && parent.inputCount == 1) {
// my parent, about to be compiled, doesn't make the cut, so
// I'm definitely pruned
final UnCompiledNode<T> node = frontier[idx];
final UnCompiledNode<T> parent = frontier[idx-1];
// if pruneCount2 is 1, we keep only up
// until the 'distinguished edge', ie we keep only the
// 'divergent' part of the FST. if my parent, about to be
// compiled, has inputCount 1 then we are already past the
// distinguished edge. NOTE: this only works if
// the FST outputs are not "compressible" (simple
// ords ARE compressible).
if (node.inputCount < minSuffixCount1) {
doPrune = true;
doCompile = true;
} else if (idx > prefixLenPlus1) {
// prune if parent's inputCount is less than suffixMinCount2
if (parent.inputCount < minSuffixCount2 || (minSuffixCount2 == 1 && parent.inputCount == 1 && idx > 1)) {
// my parent, about to be compiled, doesn't make the cut, so
// I'm definitely pruned
// if minSuffixCount2 is 1, we keep only up
// until the 'distinguished edge', ie we keep only the
// 'divergent' part of the FST. if my parent, about to be
// compiled, has inputCount 1 then we are already past the
// distinguished edge. NOTE: this only works if
// the FST outputs are not "compressible" (simple
// ords ARE compressible).
doPrune = true;
} else {
// my parent, about to be compiled, does make the cut, so
// I'm definitely not pruned
doPrune = false;
}
doCompile = true;
} else {
// my parent, about to be compiled, does make the cut, so
// I'm definitely not pruned
doPrune = false;
// if pruning is disabled (count is 0) we can always
// compile current node
doCompile = minSuffixCount2 == 0;
}
doCompile = true;
} else {
// if pruning is disabled (count is 0) we can always
// compile current node
doCompile = minSuffixCount2 == 0;
}
//System.out.println(" label=" + ((char) lastInput.ints[lastInput.offset+idx-1]) + " idx=" + idx + " inputCount=" + frontier[idx].inputCount + " doCompile=" + doCompile + " doPrune=" + doPrune);
//System.out.println(" label=" + ((char) lastInput.ints[lastInput.offset+idx-1]) + " idx=" + idx + " inputCount=" + frontier[idx].inputCount + " doCompile=" + doCompile + " doPrune=" + doPrune);
if (node.inputCount < minSuffixCount2 || minSuffixCount2 == 1 && node.inputCount == 1) {
// drop all arcs
for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
@SuppressWarnings("unchecked") final UnCompiledNode<T> target = (UnCompiledNode<T>) node.arcs[arcIdx].target;
target.clear();
if (node.inputCount < minSuffixCount2 || (minSuffixCount2 == 1 && node.inputCount == 1 && idx > 1)) {
// drop all arcs
for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
@SuppressWarnings("unchecked") final UnCompiledNode<T> target = (UnCompiledNode<T>) node.arcs[arcIdx].target;
target.clear();
}
node.numArcs = 0;
}
node.numArcs = 0;
}
if (doPrune) {
// this node doesn't make it -- deref it
node.clear();
parent.deleteLast(lastInput.ints[lastInput.offset+idx-1], node);
} else {
if (minSuffixCount2 != 0) {
compileAllTargets(node, lastInput.length-idx);
}
final T nextFinalOutput = node.output;
// We "fake" the node as being final if it has no
// outgoing arcs; in theory we could leave it
// as non-final (the FST can represent this), but
// FSTEnum, Util, etc., have trouble w/ non-final
// dead-end states:
final boolean isFinal = node.isFinal || node.numArcs == 0;
if (doCompile) {
// this node makes it and we now compile it. first,
// compile any targets that were previously
// undecided:
parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
compileNode(node, 1+lastInput.length-idx),
nextFinalOutput,
isFinal);
if (doPrune) {
// this node doesn't make it -- deref it
node.clear();
parent.deleteLast(lastInput.ints[lastInput.offset+idx-1], node);
} else {
// replaceLast just to install
// nextFinalOutput/isFinal onto the arc
parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
node,
nextFinalOutput,
isFinal);
// this node will stay in play for now, since we are
// undecided on whether to prune it. later, it
// will be either compiled or pruned, so we must
// allocate a new node:
frontier[idx] = new UnCompiledNode<T>(this, idx);
if (minSuffixCount2 != 0) {
compileAllTargets(node, lastInput.length-idx);
}
final T nextFinalOutput = node.output;
// We "fake" the node as being final if it has no
// outgoing arcs; in theory we could leave it
// as non-final (the FST can represent this), but
// FSTEnum, Util, etc., have trouble w/ non-final
// dead-end states:
final boolean isFinal = node.isFinal || node.numArcs == 0;
if (doCompile) {
// this node makes it and we now compile it. first,
// compile any targets that were previously
// undecided:
parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
compileNode(node, 1+lastInput.length-idx),
nextFinalOutput,
isFinal);
} else {
// replaceLast just to install
// nextFinalOutput/isFinal onto the arc
parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
node,
nextFinalOutput,
isFinal);
// this node will stay in play for now, since we are
// undecided on whether to prune it. later, it
// will be either compiled or pruned, so we must
// allocate a new node:
frontier[idx] = new UnCompiledNode<T>(this, idx);
}
}
}
}
@ -320,11 +338,36 @@ public class Builder<T> {
add(scratchIntsRef, output);
}
// for debugging
/*
private String toString(BytesRef b) {
try {
return b.utf8ToString() + " " + b;
} catch (Throwable t) {
return b.toString();
}
}
*/
/** It's OK to add the same input twice in a row with
* different outputs, as long as outputs impls the merge
* method. */
public void add(IntsRef input, T output) throws IOException {
//System.out.println("\nFST ADD: input=" + input + " output=" + fst.outputs.outputToString(output));
/*
if (DEBUG) {
BytesRef b = new BytesRef(input.length);
for(int x=0;x<input.length;x++) {
b.bytes[x] = (byte) input.ints[x];
}
b.length = input.length;
if (output == NO_OUTPUT) {
System.out.println("\nFST ADD: input=" + toString(b) + " " + b);
} else {
System.out.println("\nFST ADD: input=" + toString(b) + " " + b + " output=" + fst.outputs.outputToString(output));
}
}
*/
assert lastInput.length == 0 || input.compareTo(lastInput) >= 0: "inputs are added out of order lastInput=" + lastInput + " vs input=" + input;
assert validOutput(output);
@ -346,8 +389,8 @@ public class Builder<T> {
int pos2 = input.offset;
final int pos1Stop = Math.min(lastInput.length, input.length);
while(true) {
//System.out.println(" incr " + pos1);
frontier[pos1].inputCount++;
//System.out.println(" incr " + pos1 + " ct=" + frontier[pos1].inputCount + " n=" + frontier[pos1]);
if (pos1 >= pos1Stop || lastInput.ints[pos1] != input.ints[pos2]) {
break;
}
@ -368,13 +411,12 @@ public class Builder<T> {
// minimize/compile states from previous input's
// orphan'd suffix
compilePrevTail(prefixLenPlus1);
freezeTail(prefixLenPlus1);
// init tail states for current input
for(int idx=prefixLenPlus1;idx<=input.length;idx++) {
frontier[idx-1].addArc(input.ints[input.offset + idx - 1],
frontier[idx]);
//System.out.println(" incr tail " + idx);
frontier[idx].inputCount++;
}
@ -433,33 +475,24 @@ public class Builder<T> {
* nothing is accepted by the FST. */
public FST<T> finish() throws IOException {
final UnCompiledNode<T> root = frontier[0];
// minimize nodes in the last word's suffix
compilePrevTail(1);
//System.out.println("finish: inputCount=" + frontier[0].inputCount);
if (frontier[0].inputCount < minSuffixCount1 || frontier[0].inputCount < minSuffixCount2 || frontier[0].numArcs == 0) {
freezeTail(0);
if (root.inputCount < minSuffixCount1 || root.inputCount < minSuffixCount2 || root.numArcs == 0) {
if (fst.emptyOutput == null) {
return null;
} else if (minSuffixCount1 > 0 || minSuffixCount2 > 0) {
// empty string got pruned
return null;
} else {
fst.finish(compileNode(frontier[0], lastInput.length).address);
//System.out.println("compile addr = " + fst.getStartNode());
return fst;
}
} else {
if (minSuffixCount2 != 0) {
compileAllTargets(frontier[0], lastInput.length);
compileAllTargets(root, lastInput.length);
}
//System.out.println("NOW: " + frontier[0].numArcs);
fst.finish(compileNode(frontier[0], lastInput.length).address);
}
/*
if (dedupHash != null) {
System.out.println("NH: " + dedupHash.count());
}
*/
//if (DEBUG) System.out.println(" builder.finish root.isFinal=" + root.isFinal + " root.output=" + root.output);
fst.finish(compileNode(root, lastInput.length).address);
return fst;
}
@ -479,7 +512,7 @@ public class Builder<T> {
}
}
static class Arc<T> {
public static class Arc<T> {
public int label; // really an "unsigned" byte
public Node target;
public boolean isFinal;
@ -502,16 +535,20 @@ public class Builder<T> {
}
}
static final class UnCompiledNode<T> implements Node {
public static final class UnCompiledNode<T> implements Node {
final Builder<T> owner;
int numArcs;
Arc<T>[] arcs;
T output;
boolean isFinal;
long inputCount;
public int numArcs;
public Arc<T>[] arcs;
// TODO: instead of recording isFinal/output on the
// node, maybe we should use -1 arc to mean "end" (like
// we do when reading the FST). Would simplify much
// code here...
public T output;
public boolean isFinal;
public long inputCount;
/** This node's depth, starting from the automaton root. */
final int depth;
public final int depth;
/**
* @param depth

View File

@ -133,6 +133,6 @@ public final class ByteSequenceOutputs extends Outputs<BytesRef> {
@Override
public String outputToString(BytesRef output) {
return output.utf8ToString();
return output.toString();
}
}

View File

@ -123,7 +123,7 @@ public class FST<T> {
public int label;
public T output;
int target;
public int target;
byte flags;
public T nextFinalOutput;
@ -274,6 +274,10 @@ public class FST<T> {
}
}
public T getEmptyOutput() {
return emptyOutput;
}
void setEmptyOutput(T v) throws IOException {
if (emptyOutput != null) {
emptyOutput = outputs.merge(emptyOutput, v);
@ -597,9 +601,9 @@ public class FST<T> {
arc.label = END_LABEL;
arc.output = follow.nextFinalOutput;
if (follow.target <= 0) {
arc.flags = BIT_LAST_ARC;
arc.flags = BIT_LAST_ARC | BIT_FINAL_ARC;
} else {
arc.flags = 0;
arc.flags = BIT_FINAL_ARC;
arc.nextArc = follow.target;
}
//System.out.println(" insert isFinal; nextArc=" + follow.target + " isLast=" + arc.isLast() + " output=" + outputs.outputToString(arc.output));
@ -609,8 +613,7 @@ public class FST<T> {
}
}
// Not private because NodeHash needs access:
Arc<T> readFirstRealArc(int address, Arc<T> arc) throws IOException {
public Arc<T> readFirstRealArc(int address, Arc<T> arc) throws IOException {
final BytesReader in = getBytesReader(address);
@ -693,7 +696,9 @@ public class FST<T> {
return readLabel(in);
}
Arc<T> readNextRealArc(Arc<T> arc, final BytesReader in) throws IOException {
/** Never returns null, but you should never call this if
* arc.isLast() is true. */
public Arc<T> readNextRealArc(Arc<T> arc, final BytesReader in) throws IOException {
// this is a continuing arc in a fixed array
if (arc.bytesPerArc != 0) {
// arcs are at fixed entries
@ -925,7 +930,7 @@ public class FST<T> {
}
}
final BytesReader getBytesReader(int pos) {
public final BytesReader getBytesReader(int pos) {
// TODO: maybe re-use via ThreadLocal?
return new BytesReader(pos);
}

View File

@ -39,8 +39,8 @@ import org.apache.lucene.store.DataOutput;
public final class UpToTwoPositiveIntOutputs extends Outputs<Object> {
public final static class TwoLongs {
final long first;
final long second;
public final long first;
public final long second;
public TwoLongs(long first, long second) {
this.first = first;

View File

@ -213,6 +213,7 @@ public final class Util {
// Shape for states.
final String stateShape = "circle";
final String finalStateShape = "doublecircle";
// Emit DOT prologue.
out.write("digraph FST {\n");
@ -223,12 +224,34 @@ public final class Util {
}
emitDotState(out, "initial", "point", "white", "");
emitDotState(out, Integer.toString(startArc.target), stateShape,
fst.isExpandedTarget(startArc) ? expandedNodeColor : null,
"");
out.write(" initial -> " + startArc.target + "\n");
final T NO_OUTPUT = fst.outputs.getNoOutput();
// final FST.Arc<T> scratchArc = new FST.Arc<T>();
{
final String stateColor;
if (fst.isExpandedTarget(startArc)) {
stateColor = expandedNodeColor;
} else {
stateColor = null;
}
final boolean isFinal;
final T finalOutput;
if (startArc.isFinal()) {
isFinal = true;
finalOutput = startArc.nextFinalOutput == NO_OUTPUT ? null : startArc.nextFinalOutput;
} else {
isFinal = false;
finalOutput = null;
}
emitDotState(out, Integer.toString(startArc.target), isFinal ? finalStateShape : stateShape, stateColor, finalOutput == null ? "" : fst.outputs.outputToString(finalOutput));
}
out.write(" initial -> " + startArc.target + "\n");
int level = 0;
while (!nextLevelQueue.isEmpty()) {
@ -240,19 +263,48 @@ public final class Util {
out.write("\n // Transitions and states at level: " + level + "\n");
while (!thisLevelQueue.isEmpty()) {
final FST.Arc<T> arc = thisLevelQueue.remove(thisLevelQueue.size() - 1);
if (fst.targetHasArcs(arc)) {
// scan all arcs
final int node = arc.target;
fst.readFirstTargetArc(arc, arc);
if (arc.label == FST.END_LABEL) {
// Skip it -- prior recursion took this into account already
assert !arc.isLast();
fst.readNextArc(arc);
}
while (true) {
// Emit the unseen state and add it to the queue for the next level.
if (arc.target >= 0 && !seen.get(arc.target)) {
final boolean isExpanded = fst.isExpandedTarget(arc);
emitDotState(out, Integer.toString(arc.target), stateShape,
isExpanded ? expandedNodeColor : null,
labelStates ? Integer.toString(arc.target) : "");
/*
boolean isFinal = false;
T finalOutput = null;
fst.readFirstTargetArc(arc, scratchArc);
if (scratchArc.isFinal() && fst.targetHasArcs(scratchArc)) {
// target is final
isFinal = true;
finalOutput = scratchArc.output == NO_OUTPUT ? null : scratchArc.output;
System.out.println("dot hit final label=" + (char) scratchArc.label);
}
*/
final String stateColor;
if (fst.isExpandedTarget(arc)) {
stateColor = expandedNodeColor;
} else {
stateColor = null;
}
final String finalOutput;
if (arc.nextFinalOutput != null && arc.nextFinalOutput != NO_OUTPUT) {
finalOutput = fst.outputs.outputToString(arc.nextFinalOutput);
} else {
finalOutput = "";
}
emitDotState(out, Integer.toString(arc.target), arc.isFinal() ? finalStateShape : stateShape, stateColor, finalOutput);
seen.set(arc.target);
nextLevelQueue.add(new FST.Arc<T>().copyFrom(arc));
sameLevelStates.add(arc.target);
@ -265,14 +317,18 @@ public final class Util {
outs = "";
}
final String cl;
if (arc.label == FST.END_LABEL) {
cl = "~";
} else {
cl = printableLabel(arc.label);
if (!fst.targetHasArcs(arc) && arc.isFinal() && arc.nextFinalOutput != NO_OUTPUT) {
// Tricky special case: sometimes, due to
// pruning, the builder can [sillily] produce
// an FST with an arc into the final end state
// (-1) but also with a next final output; in
// this case we pull that output up onto this
// arc
outs = outs + "/[" + fst.outputs.outputToString(arc.nextFinalOutput) + "]";
}
out.write(" " + node + " -> " + arc.target + " [label=\"" + cl + outs + "\"]\n");
assert arc.label != FST.END_LABEL;
out.write(" " + node + " -> " + arc.target + " [label=\"" + printableLabel(arc.label) + outs + "\"]\n");
// Break the loop if we're on the last arc of this state.
if (arc.isLast()) {
@ -295,7 +351,7 @@ public final class Util {
}
// Emit terminating state (always there anyway).
out.write(" -1 [style=filled, color=black, shape=circle, label=\"\"]\n\n");
out.write(" -1 [style=filled, color=black, shape=doublecircle, label=\"\"]\n\n");
out.write(" {rank=sink; -1 }\n");
out.write("}\n");

View File

@ -60,7 +60,7 @@ public class RandomIndexWriter implements Closeable {
private final Random r;
public MockIndexWriter(Random r,Directory dir, IndexWriterConfig conf) throws IOException {
public MockIndexWriter(Random r, Directory dir, IndexWriterConfig conf) throws IOException {
super(dir, conf);
// must make a private random since our methods are
// called from different threads; else test failures may

View File

@ -30,8 +30,8 @@ import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.index.codecs.sep.IntStreamFactory;
import org.apache.lucene.index.codecs.sep.IntIndexInput;
import org.apache.lucene.index.codecs.sep.IntIndexOutput;
import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
import org.apache.lucene.index.codecs.sep.SepPostingsReader;
import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
import org.apache.lucene.index.codecs.standard.StandardCodec;
import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexInput;
import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexOutput;
@ -127,7 +127,7 @@ public class MockFixedIntBlockCodec extends Codec {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory(blockSize));
PostingsWriterBase postingsWriter = new SepPostingsWriter(state, new MockIntFactory(blockSize));
boolean success = false;
TermsIndexWriterBase indexWriter;
@ -158,10 +158,10 @@ public class MockFixedIntBlockCodec extends Codec {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir,
state.segmentInfo,
state.context,
new MockIntFactory(blockSize), state.codecId);
PostingsReaderBase postingsReader = new SepPostingsReader(state.dir,
state.segmentInfo,
state.context,
new MockIntFactory(blockSize), state.codecId);
TermsIndexReaderBase indexReader;
boolean success = false;
@ -204,7 +204,7 @@ public class MockFixedIntBlockCodec extends Codec {
@Override
public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files) throws IOException {
SepPostingsReaderImpl.files(segmentInfo, codecId, files);
SepPostingsReader.files(segmentInfo, codecId, files);
BlockTermsReader.files(dir, segmentInfo, codecId, files);
FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files, getDocValuesUseCFS());
@ -212,7 +212,7 @@ public class MockFixedIntBlockCodec extends Codec {
@Override
public void getExtensions(Set<String> extensions) {
SepPostingsWriterImpl.getExtensions(extensions);
SepPostingsWriter.getExtensions(extensions);
BlockTermsReader.getExtensions(extensions);
FixedGapTermsIndexReader.getIndexExtensions(extensions);
DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());

View File

@ -30,8 +30,8 @@ import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.index.codecs.sep.IntStreamFactory;
import org.apache.lucene.index.codecs.sep.IntIndexInput;
import org.apache.lucene.index.codecs.sep.IntIndexOutput;
import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
import org.apache.lucene.index.codecs.sep.SepPostingsReader;
import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
import org.apache.lucene.index.codecs.standard.StandardCodec;
import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexInput;
import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexOutput;
@ -150,7 +150,7 @@ public class MockVariableIntBlockCodec extends Codec {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory(baseBlockSize));
PostingsWriterBase postingsWriter = new SepPostingsWriter(state, new MockIntFactory(baseBlockSize));
boolean success = false;
TermsIndexWriterBase indexWriter;
@ -181,10 +181,10 @@ public class MockVariableIntBlockCodec extends Codec {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir,
state.segmentInfo,
state.context,
new MockIntFactory(baseBlockSize), state.codecId);
PostingsReaderBase postingsReader = new SepPostingsReader(state.dir,
state.segmentInfo,
state.context,
new MockIntFactory(baseBlockSize), state.codecId);
TermsIndexReaderBase indexReader;
boolean success = false;
@ -227,7 +227,7 @@ public class MockVariableIntBlockCodec extends Codec {
@Override
public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files) throws IOException {
SepPostingsReaderImpl.files(segmentInfo, codecId, files);
SepPostingsReader.files(segmentInfo, codecId, files);
BlockTermsReader.files(dir, segmentInfo, codecId, files);
FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files, getDocValuesUseCFS());
@ -235,7 +235,7 @@ public class MockVariableIntBlockCodec extends Codec {
@Override
public void getExtensions(Set<String> extensions) {
SepPostingsWriterImpl.getExtensions(extensions);
SepPostingsWriter.getExtensions(extensions);
BlockTermsReader.getExtensions(extensions);
FixedGapTermsIndexReader.getIndexExtensions(extensions);
DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());

View File

@ -30,6 +30,8 @@ import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.codecs.BlockTreeTermsReader;
import org.apache.lucene.index.codecs.BlockTreeTermsWriter;
import org.apache.lucene.index.codecs.BlockTermsReader;
import org.apache.lucene.index.codecs.BlockTermsWriter;
import org.apache.lucene.index.codecs.Codec;
@ -51,13 +53,13 @@ import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockCodec;
import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockCodec;
import org.apache.lucene.index.codecs.mocksep.MockSingleIntFactory;
import org.apache.lucene.index.codecs.pulsing.PulsingPostingsReaderImpl;
import org.apache.lucene.index.codecs.pulsing.PulsingPostingsWriterImpl;
import org.apache.lucene.index.codecs.pulsing.PulsingPostingsReader;
import org.apache.lucene.index.codecs.pulsing.PulsingPostingsWriter;
import org.apache.lucene.index.codecs.sep.IntIndexInput;
import org.apache.lucene.index.codecs.sep.IntIndexOutput;
import org.apache.lucene.index.codecs.sep.IntStreamFactory;
import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
import org.apache.lucene.index.codecs.sep.SepPostingsReader;
import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
import org.apache.lucene.store.Directory;
@ -137,7 +139,7 @@ public class MockRandomCodec extends Codec {
final long seed = seedRandom.nextLong();
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: writing to seg=" + state.segmentName + " seed=" + seed);
System.out.println("MockRandomCodec: writing to seg=" + state.segmentName + " codecID=" + state.codecId + " seed=" + seed);
}
final String seedFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, SEED_EXT);
@ -153,9 +155,8 @@ public class MockRandomCodec extends Codec {
random.nextInt(); // consume a random for buffersize
PostingsWriterBase postingsWriter;
if (random.nextBoolean()) {
postingsWriter = new SepPostingsWriterImpl(state, new MockIntStreamFactory(random), skipInterval);
postingsWriter = new SepPostingsWriter(state, new MockIntStreamFactory(random), skipInterval);
} else {
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: writing Standard postings");
@ -166,76 +167,107 @@ public class MockRandomCodec extends Codec {
if (random.nextBoolean()) {
final int totTFCutoff = _TestUtil.nextInt(random, 1, 20);
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: pulsing postings with totTFCutoff=" + totTFCutoff);
System.out.println("MockRandomCodec: writing pulsing postings with totTFCutoff=" + totTFCutoff);
}
postingsWriter = new PulsingPostingsWriterImpl(totTFCutoff, postingsWriter);
postingsWriter = new PulsingPostingsWriter(totTFCutoff, postingsWriter);
}
final TermsIndexWriterBase indexWriter;
boolean success = false;
final FieldsConsumer fields;
try {
if (random.nextBoolean()) {
state.termIndexInterval = _TestUtil.nextInt(random, 1, 100);
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: fixed-gap terms index (tii=" + state.termIndexInterval + ")");
}
indexWriter = new FixedGapTermsIndexWriter(state);
} else {
final VariableGapTermsIndexWriter.IndexTermSelector selector;
final int n2 = random.nextInt(3);
if (n2 == 0) {
final int tii = _TestUtil.nextInt(random, 1, 100);
selector = new VariableGapTermsIndexWriter.EveryNTermSelector(tii);
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: variable-gap terms index (tii=" + tii + ")");
}
} else if (n2 == 1) {
final int docFreqThresh = _TestUtil.nextInt(random, 2, 100);
final int tii = _TestUtil.nextInt(random, 1, 100);
selector = new VariableGapTermsIndexWriter.EveryNOrDocFreqTermSelector(docFreqThresh, tii);
} else {
final long seed2 = random.nextLong();
final int gap = _TestUtil.nextInt(random, 2, 40);
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: random-gap terms index (max gap=" + gap + ")");
}
selector = new VariableGapTermsIndexWriter.IndexTermSelector() {
final Random rand = new Random(seed2);
if (random.nextBoolean()) {
// Use BlockTree terms dict
@Override
public boolean isIndexTerm(BytesRef term, TermStats stats) {
return rand.nextInt(gap) == gap/2;
}
@Override
public void newField(FieldInfo fieldInfo) {
}
};
}
indexWriter = new VariableGapTermsIndexWriter(state, selector);
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: writing BlockTree terms dict");
}
success = true;
} finally {
if (!success) {
postingsWriter.close();
}
}
success = false;
try {
FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
success = true;
return ret;
} finally {
if (!success) {
try {
// TODO: would be nice to allow 1 but this is very
// slow to write
final int minTermsInBlock = _TestUtil.nextInt(random, 2, 100);
final int maxTermsInBlock = Math.max(2, (minTermsInBlock-1)*2 + random.nextInt(100));
boolean success = false;
try {
fields = new BlockTreeTermsWriter(state, postingsWriter, minTermsInBlock, maxTermsInBlock);
success = true;
} finally {
if (!success) {
postingsWriter.close();
} finally {
indexWriter.close();
}
}
} else {
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: writing Block terms dict");
}
boolean success = false;
final TermsIndexWriterBase indexWriter;
try {
if (random.nextBoolean()) {
state.termIndexInterval = _TestUtil.nextInt(random, 1, 100);
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: fixed-gap terms index (tii=" + state.termIndexInterval + ")");
}
indexWriter = new FixedGapTermsIndexWriter(state);
} else {
final VariableGapTermsIndexWriter.IndexTermSelector selector;
final int n2 = random.nextInt(3);
if (n2 == 0) {
final int tii = _TestUtil.nextInt(random, 1, 100);
selector = new VariableGapTermsIndexWriter.EveryNTermSelector(tii);
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: variable-gap terms index (tii=" + tii + ")");
}
} else if (n2 == 1) {
final int docFreqThresh = _TestUtil.nextInt(random, 2, 100);
final int tii = _TestUtil.nextInt(random, 1, 100);
selector = new VariableGapTermsIndexWriter.EveryNOrDocFreqTermSelector(docFreqThresh, tii);
} else {
final long seed2 = random.nextLong();
final int gap = _TestUtil.nextInt(random, 2, 40);
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: random-gap terms index (max gap=" + gap + ")");
}
selector = new VariableGapTermsIndexWriter.IndexTermSelector() {
final Random rand = new Random(seed2);
@Override
public boolean isIndexTerm(BytesRef term, TermStats stats) {
return rand.nextInt(gap) == gap/2;
}
@Override
public void newField(FieldInfo fieldInfo) {
}
};
}
indexWriter = new VariableGapTermsIndexWriter(state, selector);
}
success = true;
} finally {
if (!success) {
postingsWriter.close();
}
}
success = false;
try {
fields = new BlockTermsWriter(indexWriter, state, postingsWriter);
success = true;
} finally {
if (!success) {
try {
postingsWriter.close();
} finally {
indexWriter.close();
}
}
}
}
return fields;
}
@Override
@ -245,7 +277,7 @@ public class MockRandomCodec extends Codec {
final IndexInput in = state.dir.openInput(seedFileName, state.context);
final long seed = in.readLong();
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: reading from seg=" + state.segmentInfo.name + " seed=" + seed);
System.out.println("MockRandomCodec: reading from seg=" + state.segmentInfo.name + " codecID=" + state.codecId + " seed=" + seed);
}
in.close();
@ -259,8 +291,11 @@ public class MockRandomCodec extends Codec {
PostingsReaderBase postingsReader;
if (random.nextBoolean()) {
postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo,
state.context, new MockIntStreamFactory(random), state.codecId);
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: reading Sep postings");
}
postingsReader = new SepPostingsReader(state.dir, state.segmentInfo,
state.context, new MockIntStreamFactory(random), state.codecId);
} else {
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: reading Standard postings");
@ -273,86 +308,119 @@ public class MockRandomCodec extends Codec {
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: reading pulsing postings with totTFCutoff=" + totTFCutoff);
}
postingsReader = new PulsingPostingsReaderImpl(postingsReader);
postingsReader = new PulsingPostingsReader(postingsReader);
}
final TermsIndexReaderBase indexReader;
boolean success = false;
final FieldsProducer fields;
try {
if (random.nextBoolean()) {
// if termsIndexDivisor is set to -1, we should not touch it. It means a
// test explicitly instructed not to load the terms index.
if (state.termsIndexDivisor != -1) {
state.termsIndexDivisor = _TestUtil.nextInt(random, 1, 10);
}
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: fixed-gap terms index (divisor=" + state.termsIndexDivisor + ")");
}
indexReader = new FixedGapTermsIndexReader(state.dir,
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
BytesRef.getUTF8SortedAsUnicodeComparator(),
state.codecId, state.context);
} else {
final int n2 = random.nextInt(3);
if (n2 == 1) {
random.nextInt();
} else if (n2 == 2) {
random.nextLong();
}
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: variable-gap terms index (divisor=" + state.termsIndexDivisor + ")");
}
if (state.termsIndexDivisor != -1) {
state.termsIndexDivisor = _TestUtil.nextInt(random, 1, 10);
}
indexReader = new VariableGapTermsIndexReader(state.dir,
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
state.codecId, state.context);
if (random.nextBoolean()) {
// Use BlockTree terms dict
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: reading BlockTree terms dict");
}
success = true;
} finally {
if (!success) {
postingsReader.close();
}
}
final int termsCacheSize = _TestUtil.nextInt(random, 1, 1024);
success = false;
try {
FieldsProducer ret = new BlockTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,
postingsReader,
state.context,
termsCacheSize,
state.codecId);
success = true;
return ret;
} finally {
if (!success) {
try {
boolean success = false;
try {
fields = new BlockTreeTermsReader(state.dir,
state.fieldInfos,
state.segmentInfo.name,
postingsReader,
state.context,
state.codecId,
state.termsIndexDivisor);
success = true;
} finally {
if (!success) {
postingsReader.close();
} finally {
indexReader.close();
}
}
} else {
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: reading Block terms dict");
}
final TermsIndexReaderBase indexReader;
boolean success = false;
try {
final boolean doFixedGap = random.nextBoolean();
// randomness diverges from writer, here:
if (state.termsIndexDivisor != -1) {
state.termsIndexDivisor = _TestUtil.nextInt(random, 1, 10);
}
if (doFixedGap) {
// if termsIndexDivisor is set to -1, we should not touch it. It means a
// test explicitly instructed not to load the terms index.
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: fixed-gap terms index (divisor=" + state.termsIndexDivisor + ")");
}
indexReader = new FixedGapTermsIndexReader(state.dir,
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
BytesRef.getUTF8SortedAsUnicodeComparator(),
state.codecId, state.context);
} else {
final int n2 = random.nextInt(3);
if (n2 == 1) {
random.nextInt();
} else if (n2 == 2) {
random.nextLong();
}
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: variable-gap terms index (divisor=" + state.termsIndexDivisor + ")");
}
indexReader = new VariableGapTermsIndexReader(state.dir,
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
state.codecId, state.context);
}
success = true;
} finally {
if (!success) {
postingsReader.close();
}
}
final int termsCacheSize = _TestUtil.nextInt(random, 1, 1024);
success = false;
try {
fields = new BlockTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,
postingsReader,
state.context,
termsCacheSize,
state.codecId);
success = true;
} finally {
if (!success) {
try {
postingsReader.close();
} finally {
indexReader.close();
}
}
}
}
return fields;
}
@Override
public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files) throws IOException {
final String seedFileName = IndexFileNames.segmentFileName(segmentInfo.name, codecId, SEED_EXT);
files.add(seedFileName);
SepPostingsReaderImpl.files(segmentInfo, codecId, files);
SepPostingsReader.files(segmentInfo, codecId, files);
StandardPostingsReader.files(dir, segmentInfo, codecId, files);
BlockTermsReader.files(dir, segmentInfo, codecId, files);
BlockTreeTermsReader.files(dir, segmentInfo, codecId, files);
FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
VariableGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files, getDocValuesUseCFS());
@ -369,8 +437,9 @@ public class MockRandomCodec extends Codec {
@Override
public void getExtensions(Set<String> extensions) {
SepPostingsWriterImpl.getExtensions(extensions);
SepPostingsWriter.getExtensions(extensions);
BlockTermsReader.getExtensions(extensions);
BlockTreeTermsReader.getExtensions(extensions);
FixedGapTermsIndexReader.getIndexExtensions(extensions);
VariableGapTermsIndexReader.getIndexExtensions(extensions);
DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());

View File

@ -40,8 +40,8 @@ import org.apache.lucene.index.codecs.BlockTermsWriter;
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
import org.apache.lucene.index.codecs.standard.StandardCodec;
import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl;
import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl;
import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
import org.apache.lucene.index.codecs.sep.SepPostingsReader;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
@ -60,7 +60,7 @@ public class MockSepCodec extends Codec {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new MockSingleIntFactory());
PostingsWriterBase postingsWriter = new SepPostingsWriter(state, new MockSingleIntFactory());
boolean success = false;
TermsIndexWriterBase indexWriter;
@ -92,7 +92,7 @@ public class MockSepCodec extends Codec {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo,
PostingsReaderBase postingsReader = new SepPostingsReader(state.dir, state.segmentInfo,
state.context, new MockSingleIntFactory(), state.codecId);
TermsIndexReaderBase indexReader;
@ -136,7 +136,7 @@ public class MockSepCodec extends Codec {
@Override
public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files) throws IOException {
SepPostingsReaderImpl.files(segmentInfo, codecId, files);
SepPostingsReader.files(segmentInfo, codecId, files);
BlockTermsReader.files(dir, segmentInfo, codecId, files);
FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files, getDocValuesUseCFS());
@ -149,7 +149,7 @@ public class MockSepCodec extends Codec {
}
public static void getSepExtensions(Set<String> extensions) {
SepPostingsWriterImpl.getExtensions(extensions);
SepPostingsWriter.getExtensions(extensions);
BlockTermsReader.getExtensions(extensions);
FixedGapTermsIndexReader.getIndexExtensions(extensions);
}

View File

@ -66,6 +66,11 @@ public class MockSingleIntIndexOutput extends IntIndexOutput {
out.close();
}
@Override
public String toString() {
return "MockSingleIntIndexOutput fp=" + out.getFilePointer();
}
private class Index extends IntIndexOutput.Index {
long fp;
long lastFP;
@ -74,8 +79,11 @@ public class MockSingleIntIndexOutput extends IntIndexOutput {
fp = out.getFilePointer();
}
@Override
public void set(IntIndexOutput.Index other) {
lastFP = fp = ((Index) other).fp;
public void copyFrom(IntIndexOutput.Index other, boolean copyLast) {
fp = ((Index) other).fp;
if (copyLast) {
lastFP = ((Index) other).fp;
}
}
@Override
public void write(IndexOutput indexOut, boolean absolute)

View File

@ -280,7 +280,11 @@ public abstract class LuceneTestCase extends Assert {
}
swapCodec(new MockSepCodec(), cp);
swapCodec(new PulsingCodec(codecHasParam && "Pulsing".equals(codec) ? codecParam : 1 + random.nextInt(20)), cp);
// TODO: make it possible to specify min/max iterms per
// block via CL:
int minItemsPerBlock = _TestUtil.nextInt(random, 2, 100);
int maxItemsPerBlock = 2*(Math.max(2, minItemsPerBlock-1)) + random.nextInt(100);
swapCodec(new PulsingCodec(codecHasParam && "Pulsing".equals(codec) ? codecParam : 1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock), cp);
swapCodec(new MockFixedIntBlockCodec(codecHasParam && "MockFixedIntBlock".equals(codec) ? codecParam : _TestUtil.nextInt(random, 1, 2000)), cp);
// baseBlockSize cannot be over 127:
swapCodec(new MockVariableIntBlockCodec(codecHasParam && "MockVariableIntBlock".equals(codec) ? codecParam : _TestUtil.nextInt(random, 1, 127)), cp);
@ -307,7 +311,7 @@ public abstract class LuceneTestCase extends Assert {
cp.unregister(cp.lookup("MockFixedIntBlock"));
cp.unregister(cp.lookup("MockVariableIntBlock"));
cp.unregister(cp.lookup("MockRandom"));
swapCodec(new PulsingCodec(1), cp);
swapCodec(new PulsingCodec(), cp);
cp.setDefaultFieldCodec(savedDefaultCodec);
}
@ -485,7 +489,7 @@ public abstract class LuceneTestCase extends Assert {
System.err.println("NOTE: test params are: codec=" + codecDescription +
", locale=" + locale +
", timezone=" + (timeZone == null ? "(null)" : timeZone.getID()));
if (testsFailed) {
if (VERBOSE || testsFailed) {
System.err.println("NOTE: all tests run in this JVM:");
System.err.println(Arrays.toString(testClassesRun.toArray()));
System.err.println("NOTE: " + System.getProperty("os.name") + " "
@ -1561,9 +1565,17 @@ public abstract class LuceneTestCase extends Assert {
RandomCodecProvider(Random random) {
this.perFieldSeed = random.nextInt();
register(randomizCodec(random, new StandardCodec()));
// TODO: make it possible to specify min/max iterms per
// block via CL:
int minItemsPerBlock = _TestUtil.nextInt(random, 2, 100);
int maxItemsPerBlock = 2*(Math.max(2, minItemsPerBlock-1)) + random.nextInt(100);
register(randomizCodec(random, new StandardCodec(minItemsPerBlock, maxItemsPerBlock)));
register(randomizCodec(random, new PreFlexCodec()));
register(randomizCodec(random, new PulsingCodec( 1 + random.nextInt(20))));
// TODO: make it possible to specify min/max iterms per
// block via CL:
minItemsPerBlock = _TestUtil.nextInt(random, 2, 100);
maxItemsPerBlock = 2*(Math.max(1, minItemsPerBlock-1)) + random.nextInt(100);
register(randomizCodec(random, new PulsingCodec( 1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock)));
register(randomizCodec(random, new SimpleTextCodec()));
register(randomizCodec(random, new MemoryCodec()));
Collections.shuffle(knownCodecs, random);

View File

@ -154,7 +154,7 @@ public class _TestUtil {
public static CheckIndex.Status checkIndex(Directory dir, CodecProvider codecs) throws IOException {
ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
CheckIndex checker = new CheckIndex(dir);
checker.setInfoStream(new PrintStream(bos));
checker.setInfoStream(new PrintStream(bos), false);
CheckIndex.Status indexStatus = checker.checkIndex(null, codecs);
if (indexStatus == null || indexStatus.clean == false) {
System.out.println("CheckIndex failed");

View File

@ -0,0 +1,361 @@
package org.apache.lucene.util.automaton;
/**
* 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.util.*;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.UnicodeUtil;
/**
* Builds a minimal deterministic automaton that accepts a set of strings. The
* algorithm requires sorted input data, but is very fast (nearly linear with
* the input size).
*/
public final class DaciukMihovAutomatonBuilder {
/**
* DFSA state with <code>char</code> labels on transitions.
*/
final static class State {
/** An empty set of labels. */
private final static int[] NO_LABELS = new int[0];
/** An empty set of states. */
private final static State[] NO_STATES = new State[0];
/**
* Labels of outgoing transitions. Indexed identically to {@link #states}.
* Labels must be sorted lexicographically.
*/
int[] labels = NO_LABELS;
/**
* States reachable from outgoing transitions. Indexed identically to
* {@link #labels}.
*/
State[] states = NO_STATES;
/**
* <code>true</code> if this state corresponds to the end of at least one
* input sequence.
*/
boolean is_final;
/**
* Returns the target state of a transition leaving this state and labeled
* with <code>label</code>. If no such transition exists, returns
* <code>null</code>.
*/
public State getState(int label) {
final int index = Arrays.binarySearch(labels, label);
return index >= 0 ? states[index] : null;
}
/**
* Returns an array of outgoing transition labels. The array is sorted in
* lexicographic order and indexes correspond to states returned from
* {@link #getStates()}.
*/
public int[] getTransitionLabels() {
return this.labels;
}
/**
* Returns an array of outgoing transitions from this state. The returned
* array must not be changed.
*/
public State[] getStates() {
return this.states;
}
/**
* Two states are equal if:
* <ul>
* <li>they have an identical number of outgoing transitions, labeled with
* the same labels</li>
* <li>corresponding outgoing transitions lead to the same states (to states
* with an identical right-language).
* </ul>
*/
@Override
public boolean equals(Object obj) {
final State other = (State) obj;
return is_final == other.is_final
&& Arrays.equals(this.labels, other.labels)
&& referenceEquals(this.states, other.states);
}
/**
* Return <code>true</code> if this state has any children (outgoing
* transitions).
*/
public boolean hasChildren() {
return labels.length > 0;
}
/**
* Is this state a final state in the automaton?
*/
public boolean isFinal() {
return is_final;
}
/**
* Compute the hash code of the <i>current</i> status of this state.
*/
@Override
public int hashCode() {
int hash = is_final ? 1 : 0;
hash ^= hash * 31 + this.labels.length;
for (int c : this.labels)
hash ^= hash * 31 + c;
/*
* Compare the right-language of this state using reference-identity of
* outgoing states. This is possible because states are interned (stored
* in registry) and traversed in post-order, so any outgoing transitions
* are already interned.
*/
for (State s : this.states) {
hash ^= System.identityHashCode(s);
}
return hash;
}
/**
* Create a new outgoing transition labeled <code>label</code> and return
* the newly created target state for this transition.
*/
State newState(int label) {
assert Arrays.binarySearch(labels, label) < 0 : "State already has transition labeled: "
+ label;
labels = copyOf(labels, labels.length + 1);
states = copyOf(states, states.length + 1);
labels[labels.length - 1] = label;
return states[states.length - 1] = new State();
}
/**
* Return the most recent transitions's target state.
*/
State lastChild() {
assert hasChildren() : "No outgoing transitions.";
return states[states.length - 1];
}
/**
* Return the associated state if the most recent transition is labeled with
* <code>label</code>.
*/
State lastChild(int label) {
final int index = labels.length - 1;
State s = null;
if (index >= 0 && labels[index] == label) {
s = states[index];
}
assert s == getState(label);
return s;
}
/**
* Replace the last added outgoing transition's target state with the given
* state.
*/
void replaceLastChild(State state) {
assert hasChildren() : "No outgoing transitions.";
states[states.length - 1] = state;
}
/**
* JDK1.5-replacement of {@link Arrays#copyOf(int[], int)}
*/
private static int[] copyOf(int[] original, int newLength) {
int[] copy = new int[newLength];
System.arraycopy(original, 0, copy, 0,
Math.min(original.length, newLength));
return copy;
}
/**
* JDK1.5-replacement of {@link Arrays#copyOf(char[], int)}
*/
public static State[] copyOf(State[] original, int newLength) {
State[] copy = new State[newLength];
System.arraycopy(original, 0, copy, 0,
Math.min(original.length, newLength));
return copy;
}
/**
* Compare two lists of objects for reference-equality.
*/
private static boolean referenceEquals(Object[] a1, Object[] a2) {
if (a1.length != a2.length) return false;
for (int i = 0; i < a1.length; i++)
if (a1[i] != a2[i]) return false;
return true;
}
}
/**
* "register" for state interning.
*/
private HashMap<State,State> register = new HashMap<State,State>();
/**
* Root automaton state.
*/
private State root = new State();
/**
* Previous sequence added to the automaton in {@link #add(CharSequence)}.
*/
private CharsRef previous;
private static final Comparator<CharsRef> comparator = CharsRef.getUTF16SortedAsUTF8Comparator();
/**
* Add another character sequence to this automaton. The sequence must be
* lexicographically larger or equal compared to any previous sequences added
* to this automaton (the input must be sorted).
*/
public void add(CharsRef current) {
assert register != null : "Automaton already built.";
assert previous == null
|| comparator.compare(previous, current) <= 0 : "Input must be sorted: "
+ previous + " >= " + current;
assert setPrevious(current);
// Descend in the automaton (find matching prefix).
int pos = 0, max = current.length();
State next, state = root;
while (pos < max && (next = state.lastChild(Character.codePointAt(current, pos))) != null) {
state = next;
// todo, optimize me
pos += Character.charCount(Character.codePointAt(current, pos));
}
if (state.hasChildren()) replaceOrRegister(state);
addSuffix(state, current, pos);
}
/**
* Finalize the automaton and return the root state. No more strings can be
* added to the builder after this call.
*
* @return Root automaton state.
*/
public State complete() {
if (this.register == null) throw new IllegalStateException();
if (root.hasChildren()) replaceOrRegister(root);
register = null;
return root;
}
/**
* Internal recursive traversal for conversion.
*/
private static org.apache.lucene.util.automaton.State convert(State s,
IdentityHashMap<State,org.apache.lucene.util.automaton.State> visited) {
org.apache.lucene.util.automaton.State converted = visited.get(s);
if (converted != null) return converted;
converted = new org.apache.lucene.util.automaton.State();
converted.setAccept(s.is_final);
visited.put(s, converted);
int i = 0;
int[] labels = s.labels;
for (DaciukMihovAutomatonBuilder.State target : s.states) {
converted.addTransition(new Transition(labels[i++], convert(target,
visited)));
}
return converted;
}
/**
* Build a minimal, deterministic automaton from a sorted list of strings.
*/
public static Automaton build(Collection<BytesRef> input) {
final DaciukMihovAutomatonBuilder builder = new DaciukMihovAutomatonBuilder();
CharsRef scratch = new CharsRef();
for (BytesRef b : input) {
UnicodeUtil.UTF8toUTF16(b, scratch);
builder.add(scratch);
}
Automaton a = new Automaton();
a.initial = convert(builder.complete(), new IdentityHashMap<State,org.apache.lucene.util.automaton.State>());
a.deterministic = true;
return a;
}
/**
* Copy <code>current</code> into an internal buffer.
*/
private boolean setPrevious(CharsRef current) {
// don't need to copy, once we fix https://issues.apache.org/jira/browse/LUCENE-3277
// still, called only from assert
previous = new CharsRef(current);
return true;
}
/**
* Replace last child of <code>state</code> with an already registered state
* or register the last child state.
*/
private void replaceOrRegister(State state) {
final State child = state.lastChild();
if (child.hasChildren()) replaceOrRegister(child);
final State registered = register.get(child);
if (registered != null) {
state.replaceLastChild(registered);
} else {
register.put(child, child);
}
}
/**
* Add a suffix of <code>current</code> starting at <code>fromIndex</code>
* (inclusive) to state <code>state</code>.
*/
private void addSuffix(State state, CharSequence current, int fromIndex) {
final int len = current.length();
while (fromIndex < len) {
int cp = Character.codePointAt(current, fromIndex);
state = state.newState(cp);
fromIndex += Character.charCount(cp);
}
state.is_final = true;
}
}

View File

@ -86,7 +86,7 @@ public class TestSearchForDuplicates extends LuceneTestCase {
}
IndexWriter writer = new IndexWriter(directory, conf);
if (VERBOSE) {
System.out.println("TEST: now build index");
System.out.println("TEST: now build index MAX_DOCS=" + MAX_DOCS);
writer.setInfoStream(System.out);
}

View File

@ -155,9 +155,9 @@ public class Test2BTerms extends LuceneTestCase {
List<BytesRef> savedTerms = null;
MockDirectoryWrapper dir = newFSDirectory(_TestUtil.getTempDir("2BTerms"));
//MockDirectoryWrapper dir = newFSDirectory(new File("/p/lucene/indices/2bindex"));
dir.setThrottling(MockDirectoryWrapper.Throttling.NEVER);
dir.setCheckIndexOnClose(false); // don't double-checkindex
//Directory dir = newFSDirectory(new File("/p/lucene/indices/2bindex"));
if (true) {
@ -169,6 +169,7 @@ public class Test2BTerms extends LuceneTestCase {
.setMergePolicy(newLogMergePolicy(false, 10))
.setOpenMode(IndexWriterConfig.OpenMode.CREATE));
w.setInfoStream(VERBOSE ? System.out : null);
MergePolicy mp = w.getConfig().getMergePolicy();
if (mp instanceof LogByteSizeMergePolicy) {
// 1 petabyte:

View File

@ -40,8 +40,6 @@ import org.apache.lucene.search.PhraseQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IOContext.Context;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.Version;
@ -504,6 +502,9 @@ public class TestCodecs extends LuceneTestCase {
}
// Test seek to non-existent terms:
if (VERBOSE) {
System.out.println("TEST: seek non-exist terms");
}
for(int i=0;i<100;i++) {
final String text2 = _TestUtil.randomUnicodeString(random) + ".";
status = termsEnum.seekCeil(new BytesRef(text2));
@ -512,6 +513,9 @@ public class TestCodecs extends LuceneTestCase {
}
// Seek to each term, backwards:
if (VERBOSE) {
System.out.println("TEST: seek terms backwards");
}
for(int i=field.terms.length-1;i>=0;i--) {
assertEquals(Thread.currentThread().getName() + ": field=" + field.fieldInfo.name + " term=" + field.terms[i].text2, TermsEnum.SeekStatus.FOUND, termsEnum.seekCeil(new BytesRef(field.terms[i].text2)));
assertEquals(field.terms[i].docs.length, termsEnum.docFreq());

View File

@ -462,7 +462,7 @@ public class TestDocTermOrds extends LuceneTestCase {
*/
if (VERBOSE) {
System.out.println("TEST: verify prefix=" + prefixRef.utf8ToString());
System.out.println("TEST: verify prefix=" + (prefixRef==null ? "null" : prefixRef.utf8ToString()));
System.out.println("TEST: all TERMS:");
TermsEnum allTE = MultiFields.getTerms(r, "field").iterator();
int ord = 0;

View File

@ -194,7 +194,8 @@ public class TestDocsAndPositions extends LuceneTestCase {
public void testRandomDocs() throws IOException {
Directory dir = newDirectory();
RandomIndexWriter writer = new RandomIndexWriter(random, dir,
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
writer.w.setInfoStream(VERBOSE ? System.out : null);
int numDocs = atLeast(49);
int max = 15678;
int term = random.nextInt(max);
@ -290,7 +291,7 @@ public class TestDocsAndPositions extends LuceneTestCase {
writer.addDocument(doc);
}
// now do seaches
// now do searches
IndexReader reader = writer.getReader();
writer.close();

View File

@ -19,7 +19,6 @@ package org.apache.lucene.index;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.search.DefaultSimilarity;
import org.apache.lucene.search.Similarity;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
@ -169,7 +168,7 @@ public class TestIndexFileDeleter extends LuceneTestCase {
Set<String> dif = difFiles(files, files2);
if (!Arrays.equals(files, files2)) {
fail("IndexFileDeleter failed to delete unreferenced extra files: should have deleted " + (filesPre.length-files.length) + " files but only deleted " + (filesPre.length - files2.length) + "; expected files:\n " + asString(files) + "\n actual files:\n " + asString(files2)+"\ndif: "+dif);
fail("IndexFileDeleter failed to delete unreferenced extra files: should have deleted " + (filesPre.length-files.length) + " files but only deleted " + (filesPre.length - files2.length) + "; expected files:\n " + asString(files) + "\n actual files:\n " + asString(files2)+"\ndiff: "+dif);
}
}

View File

@ -74,8 +74,15 @@ public class TestIndexWriterDelete extends LuceneTestCase {
Term term = new Term("city", "Amsterdam");
int hitCount = getHitCount(dir, term);
assertEquals(1, hitCount);
if (VERBOSE) {
System.out.println("\nTEST: now delete by term=" + term);
}
modifier.deleteDocuments(term);
modifier.commit();
if (VERBOSE) {
System.out.println("\nTEST: now getHitCount");
}
hitCount = getHitCount(dir, term);
assertEquals(0, hitCount);

View File

@ -221,6 +221,8 @@ public class TestIndexWriterMerging extends LuceneTestCase
setMergePolicy(newLogMergePolicy(50))
);
writer.setInfoStream(VERBOSE ? System.out : null);
Document document = new Document();
document = new Document();

View File

@ -355,7 +355,7 @@ public class TestLongPostings extends LuceneTestCase {
}
if (VERBOSE) {
System.out.println("\nTEST: iter=" + iter + " doS1=" + doS1);
System.out.println("\nTEST: iter=" + iter + " doS1=" + doS1 + " term=" + term);
}
final DocsEnum postings = MultiFields.getTermDocsEnum(r, null, "field", new BytesRef(term));
@ -426,7 +426,7 @@ public class TestLongPostings extends LuceneTestCase {
if (random.nextInt(6) == 3) {
final int freq = postings.freq();
assertTrue(freq >=1 && freq <= 4);
assertTrue("got invalid freq=" + freq, freq >=1 && freq <= 4);
}
}
}

View File

@ -29,10 +29,15 @@ public class TestMultiFields extends LuceneTestCase {
int num = atLeast(2);
for (int iter = 0; iter < num; iter++) {
if (VERBOSE) {
System.out.println("TEST: iter=" + iter);
}
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(NoMergePolicy.COMPOUND_FILES));
_TestUtil.keepFullyDeletedSegments(w);
w.setInfoStream(VERBOSE ? System.out : null);
Map<BytesRef,List<Integer>> docs = new HashMap<BytesRef,List<Integer>>();
Set<Integer> deleted = new HashSet<Integer>();
@ -46,6 +51,9 @@ public class TestMultiFields extends LuceneTestCase {
doc.add(id);
boolean onlyUniqueTerms = random.nextBoolean();
if (VERBOSE) {
System.out.println("TEST: onlyUniqueTerms=" + onlyUniqueTerms + " numDocs=" + numDocs);
}
Set<BytesRef> uniqueTerms = new HashSet<BytesRef>();
for(int i=0;i<numDocs;i++) {
@ -74,21 +82,33 @@ public class TestMultiFields extends LuceneTestCase {
int delID = random.nextInt(i);
deleted.add(delID);
w.deleteDocuments(new Term("id", ""+delID));
if (VERBOSE) {
System.out.println("TEST: delete " + delID);
}
}
}
if (VERBOSE) {
List<BytesRef> termsList = new ArrayList<BytesRef>(uniqueTerms);
Collections.sort(termsList, BytesRef.getUTF8SortedAsUTF16Comparator());
System.out.println("UTF16 order:");
System.out.println("TEST: terms in UTF16 order:");
for(BytesRef b : termsList) {
System.out.println(" " + UnicodeUtil.toHexString(b.utf8ToString()));
System.out.println(" " + UnicodeUtil.toHexString(b.utf8ToString()) + " " + b);
for(int docID : docs.get(b)) {
if (deleted.contains(docID)) {
System.out.println(" " + docID + " (deleted)");
} else {
System.out.println(" " + docID);
}
}
}
}
IndexReader reader = w.getReader();
w.close();
//System.out.println("TEST reader=" + reader);
if (VERBOSE) {
System.out.println("TEST: reader=" + reader);
}
Bits liveDocs = MultiFields.getLiveDocs(reader);
for(int delDoc : deleted) {
@ -99,7 +119,7 @@ public class TestMultiFields extends LuceneTestCase {
for(int i=0;i<100;i++) {
BytesRef term = terms.get(random.nextInt(terms.size()));
if (VERBOSE) {
System.out.println("TEST: seek to term= "+ UnicodeUtil.toHexString(term.utf8ToString()));
System.out.println("TEST: seek term="+ UnicodeUtil.toHexString(term.utf8ToString()) + " " + term);
}
DocsEnum docsEnum = terms2.docs(liveDocs, term, null);

View File

@ -17,15 +17,32 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericField;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LineFileDocs;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.BasicAutomata;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.DaciukMihovAutomatonBuilder;
public class TestTermsEnum extends LuceneTestCase {
@ -140,4 +157,563 @@ public class TestTermsEnum extends LuceneTestCase {
r.close();
d.close();
}
private String randomString() {
//return _TestUtil.randomSimpleString(random);
return _TestUtil.randomRealisticUnicodeString(random);
}
private void addDoc(RandomIndexWriter w, Collection<String> terms, Map<BytesRef,Integer> termToID, int id) throws IOException {
Document doc = new Document();
doc.add(new NumericField("id").setIntValue(id));
if (VERBOSE) {
System.out.println("TEST: addDoc id:" + id + " terms=" + terms);
}
for (String s2 : terms) {
doc.add(newField("f", s2, Field.Index.NOT_ANALYZED));
termToID.put(new BytesRef(s2), id);
}
w.addDocument(doc);
terms.clear();
}
private boolean accepts(CompiledAutomaton c, BytesRef b) {
int state = c.runAutomaton.getInitialState();
for(int idx=0;idx<b.length;idx++) {
assertTrue(state != -1);
state = c.runAutomaton.step(state, b.bytes[b.offset+idx] & 0xff);
}
return c.runAutomaton.isAccept(state);
}
// Tests Terms.intersect
public void testIntersectRandom() throws IOException {
final Directory dir = newDirectory();
final RandomIndexWriter w = new RandomIndexWriter(random, dir);
final int numTerms = atLeast(1000);
final Set<String> terms = new HashSet<String>();
final Collection<String> pendingTerms = new ArrayList<String>();
final Map<BytesRef,Integer> termToID = new HashMap<BytesRef,Integer>();
int id = 0;
while(terms.size() != numTerms) {
final String s = randomString();
if (!terms.contains(s)) {
terms.add(s);
pendingTerms.add(s);
if (random.nextInt(20) == 7) {
addDoc(w, pendingTerms, termToID, id++);
}
}
}
addDoc(w, pendingTerms, termToID, id++);
final BytesRef[] termsArray = new BytesRef[terms.size()];
final Set<BytesRef> termsSet = new HashSet<BytesRef>();
{
int upto = 0;
for(String s : terms) {
final BytesRef b = new BytesRef(s);
termsArray[upto++] = b;
termsSet.add(b);
}
Arrays.sort(termsArray);
}
if (VERBOSE) {
System.out.println("\nTEST: indexed terms (unicode order):");
for(BytesRef t : termsArray) {
System.out.println(" " + t.utf8ToString() + " -> id:" + termToID.get(t));
}
}
final IndexReader r = w.getReader();
w.close();
// NOTE: intentional insanity!!
final int[] docIDToID = FieldCache.DEFAULT.getInts(r, "id");
for(int iter=0;iter<10*RANDOM_MULTIPLIER;iter++) {
// TODO: can we also test infinite As here...?
// From the random terms, pick some ratio and compile an
// automaton:
final Set<String> acceptTerms = new HashSet<String>();
final TreeSet<BytesRef> sortedAcceptTerms = new TreeSet<BytesRef>();
final double keepPct = random.nextDouble();
Automaton a;
if (iter == 0) {
if (VERBOSE) {
System.out.println("\nTEST: empty automaton");
}
a = BasicAutomata.makeEmpty();
} else {
if (VERBOSE) {
System.out.println("\nTEST: keepPct=" + keepPct);
}
for (String s : terms) {
final String s2;
if (random.nextDouble() <= keepPct) {
s2 = s;
} else {
s2 = randomString();
}
acceptTerms.add(s2);
sortedAcceptTerms.add(new BytesRef(s2));
}
a = DaciukMihovAutomatonBuilder.build(sortedAcceptTerms);
}
final CompiledAutomaton c = new CompiledAutomaton(a, true, false);
final BytesRef[] acceptTermsArray = new BytesRef[acceptTerms.size()];
final Set<BytesRef> acceptTermsSet = new HashSet<BytesRef>();
int upto = 0;
for(String s : acceptTerms) {
final BytesRef b = new BytesRef(s);
acceptTermsArray[upto++] = b;
acceptTermsSet.add(b);
assertTrue(accepts(c, b));
}
Arrays.sort(acceptTermsArray);
if (VERBOSE) {
System.out.println("\nTEST: accept terms (unicode order):");
for(BytesRef t : acceptTermsArray) {
System.out.println(" " + t.utf8ToString() + (termsSet.contains(t) ? " (exists)" : ""));
}
System.out.println(a.toDot());
}
for(int iter2=0;iter2<100;iter2++) {
final BytesRef startTerm = acceptTermsArray.length == 0 || random.nextBoolean() ? null : acceptTermsArray[random.nextInt(acceptTermsArray.length)];
final TermsEnum te = MultiFields.getTerms(r, "f").intersect(c, startTerm);
if (VERBOSE) {
System.out.println("\nTEST: iter2=" + iter2 + " startTerm=" + (startTerm == null ? "<null>" : startTerm.utf8ToString()));
}
int loc;
if (startTerm == null) {
loc = 0;
} else {
loc = Arrays.binarySearch(termsArray, new BytesRef(startTerm));
if (loc < 0) {
loc = -(loc+1);
} else {
// startTerm exists in index
loc++;
}
}
while(loc < termsArray.length && !acceptTermsSet.contains(termsArray[loc])) {
loc++;
}
DocsEnum docsEnum = null;
while (loc < termsArray.length) {
final BytesRef expected = termsArray[loc];
final BytesRef actual = te.next();
if (VERBOSE) {
System.out.println("TEST: next() expected=" + expected.utf8ToString() + " actual=" + actual.utf8ToString());
}
assertEquals(expected, actual);
assertEquals(1, te.docFreq());
docsEnum = te.docs(null, docsEnum);
final int docID = docsEnum.nextDoc();
assertTrue(docID != DocsEnum.NO_MORE_DOCS);
assertEquals(docIDToID[docID], termToID.get(expected).intValue());
do {
loc++;
} while (loc < termsArray.length && !acceptTermsSet.contains(termsArray[loc]));
}
assertNull(te.next());
}
}
r.close();
dir.close();
}
private Directory d;
private IndexReader r;
private final String FIELD = "field";
private IndexReader makeIndex(String... terms) throws Exception {
d = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random));
/*
CoreCodecProvider cp = new CoreCodecProvider();
cp.unregister(cp.lookup("Standard"));
cp.register(new StandardCodec(minTermsInBlock, maxTermsInBlock));
cp.setDefaultFieldCodec("Standard");
iwc.setCodecProvider(cp);
*/
final RandomIndexWriter w = new RandomIndexWriter(random, d, iwc);
w.w.setInfoStream(VERBOSE ? System.out : null);
for(String term : terms) {
Document doc = new Document();
Field f = newField(FIELD, term, Field.Index.NOT_ANALYZED_NO_NORMS);
doc.add(f);
w.addDocument(doc);
}
if (r != null) {
close();
}
r = w.getReader();
w.close();
return r;
}
private void close() throws Exception {
final Directory d = ((SegmentReader) r.getSequentialSubReaders()[0]).directory();
r.close();
d.close();
}
private int docFreq(IndexReader r, String term) throws Exception {
return r.docFreq(new Term(FIELD, term));
}
public void testEasy() throws Exception {
// No floor arcs:
r = makeIndex("aa0", "aa1", "aa2", "aa3", "bb0", "bb1", "bb2", "bb3", "aa");
// First term in block:
assertEquals(1, docFreq(r, "aa0"));
// Scan forward to another term in same block
assertEquals(1, docFreq(r, "aa2"));
assertEquals(1, docFreq(r, "aa"));
// Reset same block then scan forwards
assertEquals(1, docFreq(r, "aa1"));
// Not found, in same block
assertEquals(0, docFreq(r, "aa5"));
// Found, in same block
assertEquals(1, docFreq(r, "aa2"));
// Not found in index:
assertEquals(0, docFreq(r, "b0"));
// Found:
assertEquals(1, docFreq(r, "aa2"));
// Found, rewind:
assertEquals(1, docFreq(r, "aa0"));
// First term in block:
assertEquals(1, docFreq(r, "bb0"));
// Scan forward to another term in same block
assertEquals(1, docFreq(r, "bb2"));
// Reset same block then scan forwards
assertEquals(1, docFreq(r, "bb1"));
// Not found, in same block
assertEquals(0, docFreq(r, "bb5"));
// Found, in same block
assertEquals(1, docFreq(r, "bb2"));
// Not found in index:
assertEquals(0, docFreq(r, "b0"));
// Found:
assertEquals(1, docFreq(r, "bb2"));
// Found, rewind:
assertEquals(1, docFreq(r, "bb0"));
close();
}
// tests:
// - test same prefix has non-floor block and floor block (ie, has 2 long outputs on same term prefix)
// - term that's entirely in the index
public void testFloorBlocks() throws Exception {
final String[] terms = new String[] {"aa0", "aa1", "aa2", "aa3", "aa4", "aa5", "aa6", "aa7", "aa8", "aa9", "aa", "xx"};
r = makeIndex(terms);
//r = makeIndex("aa0", "aa1", "aa2", "aa3", "aa4", "aa5", "aa6", "aa7", "aa8", "aa9");
// First term in first block:
assertEquals(1, docFreq(r, "aa0"));
assertEquals(1, docFreq(r, "aa4"));
// No block
assertEquals(0, docFreq(r, "bb0"));
// Second block
assertEquals(1, docFreq(r, "aa4"));
// Backwards to prior floor block:
assertEquals(1, docFreq(r, "aa0"));
// Forwards to last floor block:
assertEquals(1, docFreq(r, "aa9"));
assertEquals(0, docFreq(r, "a"));
assertEquals(1, docFreq(r, "aa"));
assertEquals(0, docFreq(r, "a"));
assertEquals(1, docFreq(r, "aa"));
// Forwards to last floor block:
assertEquals(1, docFreq(r, "xx"));
assertEquals(1, docFreq(r, "aa1"));
assertEquals(0, docFreq(r, "yy"));
assertEquals(1, docFreq(r, "xx"));
assertEquals(1, docFreq(r, "aa9"));
assertEquals(1, docFreq(r, "xx"));
assertEquals(1, docFreq(r, "aa4"));
final TermsEnum te = MultiFields.getTerms(r, FIELD).iterator();
while(te.next() != null) {
//System.out.println("TEST: next term=" + te.term().utf8ToString());
}
assertTrue(seekExact(te, "aa1"));
assertEquals("aa2", next(te));
assertTrue(seekExact(te, "aa8"));
assertEquals("aa9", next(te));
assertEquals("xx", next(te));
testRandomSeeks(r, terms);
close();
}
public void testZeroTerms() throws Exception {
d = newDirectory();
final RandomIndexWriter w = new RandomIndexWriter(random, d);
w.w.setInfoStream(VERBOSE ? System.out : null);
Document doc = new Document();
doc.add(newField("field", "one two three", Field.Index.ANALYZED));
doc = new Document();
doc.add(newField("field2", "one two three", Field.Index.ANALYZED));
w.addDocument(doc);
w.commit();
w.deleteDocuments(new Term("field", "one"));
w.optimize();
IndexReader r = w.getReader();
w.close();
assertEquals(1, r.numDocs());
assertEquals(1, r.maxDoc());
Terms terms = MultiFields.getTerms(r, "field");
if (terms != null) {
assertNull(terms.iterator().next());
}
r.close();
d.close();
}
private String getRandomString() {
//return _TestUtil.randomSimpleString(random);
return _TestUtil.randomRealisticUnicodeString(random);
}
public void testRandomTerms() throws Exception {
final String[] terms = new String[_TestUtil.nextInt(random, 1, atLeast(1000))];
final Set<String> seen = new HashSet<String>();
final boolean allowEmptyString = random.nextBoolean();
if (random.nextInt(10) == 7 && terms.length > 2) {
// Sometimes add a bunch of terms sharing a longish common prefix:
final int numTermsSamePrefix = random.nextInt(terms.length/2);
if (numTermsSamePrefix > 0) {
String prefix;
while(true) {
prefix = getRandomString();
if (prefix.length() < 5) {
continue;
} else {
break;
}
}
while(seen.size() < numTermsSamePrefix) {
final String t = prefix + getRandomString();
if (!seen.contains(t)) {
terms[seen.size()] = t;
seen.add(t);
}
}
}
}
while(seen.size() < terms.length) {
final String t = getRandomString();
if (!seen.contains(t) && (allowEmptyString || t.length() != 0)) {
terms[seen.size()] = t;
seen.add(t);
}
}
r = makeIndex(terms);
testRandomSeeks(r, terms);
close();
}
// sugar
private boolean seekExact(TermsEnum te, String term) throws IOException {
return te.seekExact(new BytesRef(term), random.nextBoolean());
}
// sugar
private String next(TermsEnum te) throws IOException {
final BytesRef br = te.next();
if (br == null) {
return null;
} else {
return br.utf8ToString();
}
}
private BytesRef getNonExistTerm(BytesRef[] terms) {
BytesRef t = null;
while(true) {
final String ts = getRandomString();
t = new BytesRef(ts);
if (Arrays.binarySearch(terms, t) < 0) {
return t;
}
}
}
private static class TermAndState {
public final BytesRef term;
public final TermState state;
public TermAndState(BytesRef term, TermState state) {
this.term = term;
this.state = state;
}
}
private void testRandomSeeks(IndexReader r, String... validTermStrings) throws IOException {
final BytesRef[] validTerms = new BytesRef[validTermStrings.length];
for(int termIDX=0;termIDX<validTermStrings.length;termIDX++) {
validTerms[termIDX] = new BytesRef(validTermStrings[termIDX]);
}
Arrays.sort(validTerms);
if (VERBOSE) {
System.out.println("TEST: " + validTerms.length + " terms:");
for(BytesRef t : validTerms) {
System.out.println(" " + t.utf8ToString() + " " + t);
}
}
final TermsEnum te = MultiFields.getTerms(r, FIELD).iterator();
final int END_LOC = -validTerms.length-1;
final List<TermAndState> termStates = new ArrayList<TermAndState>();
for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
final BytesRef t;
int loc;
final TermState termState;
if (random.nextInt(6) == 4) {
// pick term that doens't exist:
t = getNonExistTerm(validTerms);
termState = null;
if (VERBOSE) {
System.out.println("\nTEST: invalid term=" + t.utf8ToString());
}
loc = Arrays.binarySearch(validTerms, t);
} else if (termStates.size() != 0 && random.nextInt(4) == 1) {
final TermAndState ts = termStates.get(random.nextInt(termStates.size()));
t = ts.term;
loc = Arrays.binarySearch(validTerms, t);
assertTrue(loc >= 0);
termState = ts.state;
if (VERBOSE) {
System.out.println("\nTEST: valid termState term=" + t.utf8ToString());
}
} else {
// pick valid term
loc = random.nextInt(validTerms.length);
t = new BytesRef(validTerms[loc]);
termState = null;
if (VERBOSE) {
System.out.println("\nTEST: valid term=" + t.utf8ToString());
}
}
// seekCeil or seekExact:
final boolean doSeekExact = random.nextBoolean();
if (termState != null) {
if (VERBOSE) {
System.out.println(" seekExact termState");
}
te.seekExact(t, termState);
} else if (doSeekExact) {
if (VERBOSE) {
System.out.println(" seekExact");
}
assertEquals(loc >= 0, te.seekExact(t, random.nextBoolean()));
} else {
if (VERBOSE) {
System.out.println(" seekCeil");
}
final TermsEnum.SeekStatus result = te.seekCeil(t, random.nextBoolean());
if (VERBOSE) {
System.out.println(" got " + result);
}
if (loc >= 0) {
assertEquals(TermsEnum.SeekStatus.FOUND, result);
} else if (loc == END_LOC) {
assertEquals(TermsEnum.SeekStatus.END, result);
} else {
assert loc >= -validTerms.length;
assertEquals(TermsEnum.SeekStatus.NOT_FOUND, result);
}
}
if (loc >= 0) {
assertEquals(t, te.term());
} else if (doSeekExact) {
// TermsEnum is unpositioned if seekExact returns false
continue;
} else if (loc == END_LOC) {
continue;
} else {
loc = -loc-1;
assertEquals(validTerms[loc], te.term());
}
// Do a bunch of next's after the seek
final int numNext = random.nextInt(validTerms.length);
for(int nextCount=0;nextCount<numNext;nextCount++) {
if (VERBOSE) {
System.out.println("\nTEST: next loc=" + loc + " of " + validTerms.length);
}
final BytesRef t2 = te.next();
loc++;
if (loc == validTerms.length) {
assertNull(t2);
break;
} else {
assertEquals(validTerms[loc], t2);
if (random.nextInt(40) == 17 && termStates.size() < 100) {
termStates.add(new TermAndState(validTerms[loc], te.termState()));
}
}
}
}
}
}

View File

@ -0,0 +1,175 @@
package org.apache.lucene.index;
/**
* 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.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.SortedSet;
import java.util.TreeSet;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.TermsEnum.SeekStatus;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.search.AutomatonQuery;
import org.apache.lucene.search.CheckHits;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.AutomatonTestUtil;
import org.apache.lucene.util.automaton.BasicOperations;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.DaciukMihovAutomatonBuilder;
import org.apache.lucene.util.automaton.RegExp;
import org.apache.lucene.util.automaton.SpecialOperations;
public class TestTermsEnum2 extends LuceneTestCase {
private Directory dir;
private IndexReader reader;
private IndexSearcher searcher;
private SortedSet<BytesRef> terms; // the terms we put in the index
private Automaton termsAutomaton; // automata of the same
int numIterations;
public void setUp() throws Exception {
super.setUp();
// we generate aweful regexps: good for testing.
// but for preflex codec, the test can be very slow, so use less iterations.
numIterations = CodecProvider.getDefault().getFieldCodec("field").equals("PreFlex") ? 10 * RANDOM_MULTIPLIER : atLeast(50);
dir = newDirectory();
RandomIndexWriter writer = new RandomIndexWriter(random, dir,
newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random, MockTokenizer.KEYWORD, false))
.setMaxBufferedDocs(_TestUtil.nextInt(random, 50, 1000)));
Document doc = new Document();
Field field = newField("field", "", Field.Store.YES, Field.Index.NOT_ANALYZED);
doc.add(field);
terms = new TreeSet<BytesRef>();
int num = atLeast(200);
for (int i = 0; i < num; i++) {
String s = _TestUtil.randomUnicodeString(random);
field.setValue(s);
terms.add(new BytesRef(s));
writer.addDocument(doc);
}
termsAutomaton = DaciukMihovAutomatonBuilder.build(terms);
reader = writer.getReader();
searcher = newSearcher(reader);
writer.close();
}
public void tearDown() throws Exception {
searcher.close();
reader.close();
dir.close();
super.tearDown();
}
/** tests a pre-intersected automaton against the original */
public void testFiniteVersusInfinite() throws Exception {
for (int i = 0; i < numIterations; i++) {
String reg = AutomatonTestUtil.randomRegexp(random);
Automaton automaton = new RegExp(reg, RegExp.NONE).toAutomaton();
final List<BytesRef> matchedTerms = new ArrayList<BytesRef>();
for(BytesRef t : terms) {
if (BasicOperations.run(automaton, t.utf8ToString())) {
matchedTerms.add(t);
}
}
Automaton alternate = DaciukMihovAutomatonBuilder.build(matchedTerms);
//System.out.println("match " + matchedTerms.size() + " " + alternate.getNumberOfStates() + " states, sigma=" + alternate.getStartPoints().length);
//AutomatonTestUtil.minimizeSimple(alternate);
//System.out.println("minmize done");
AutomatonQuery a1 = new AutomatonQuery(new Term("field", ""), automaton);
AutomatonQuery a2 = new AutomatonQuery(new Term("field", ""), alternate);
CheckHits.checkEqual(a1, searcher.search(a1, 25).scoreDocs, searcher.search(a2, 25).scoreDocs);
}
}
/** seeks to every term accepted by some automata */
public void testSeeking() throws Exception {
for (int i = 0; i < numIterations; i++) {
String reg = AutomatonTestUtil.randomRegexp(random);
Automaton automaton = new RegExp(reg, RegExp.NONE).toAutomaton();
TermsEnum te = MultiFields.getTerms(reader, "field").iterator();
ArrayList<BytesRef> unsortedTerms = new ArrayList<BytesRef>(terms);
Collections.shuffle(unsortedTerms, random);
for (BytesRef term : unsortedTerms) {
if (BasicOperations.run(automaton, term.utf8ToString())) {
// term is accepted
if (random.nextBoolean()) {
// seek exact
assertTrue(te.seekExact(term, random.nextBoolean()));
} else {
// seek ceil
assertEquals(SeekStatus.FOUND, te.seekCeil(term, random.nextBoolean()));
assertEquals(term, te.term());
}
}
}
}
}
/** mixes up seek and next for all terms */
public void testSeekingAndNexting() throws Exception {
for (int i = 0; i < numIterations; i++) {
TermsEnum te = MultiFields.getTerms(reader, "field").iterator();
for (BytesRef term : terms) {
int c = random.nextInt(3);
if (c == 0) {
assertEquals(term, te.next());
} else if (c == 1) {
assertEquals(SeekStatus.FOUND, te.seekCeil(term, random.nextBoolean()));
assertEquals(term, te.term());
} else {
assertTrue(te.seekExact(term, random.nextBoolean()));
}
}
}
}
/** tests intersect: TODO start at a random term! */
public void testIntersect() throws Exception {
for (int i = 0; i < numIterations; i++) {
String reg = AutomatonTestUtil.randomRegexp(random);
Automaton automaton = new RegExp(reg, RegExp.NONE).toAutomaton();
CompiledAutomaton ca = new CompiledAutomaton(automaton, SpecialOperations.isFinite(automaton), false);
TermsEnum te = MultiFields.getTerms(reader, "field").intersect(ca, null);
Automaton expected = BasicOperations.intersection(termsAutomaton, automaton);
TreeSet<BytesRef> found = new TreeSet<BytesRef>();
while (te.next() != null) {
found.add(new BytesRef(te.term()));
}
Automaton actual = DaciukMihovAutomatonBuilder.build(found);
assertTrue(BasicOperations.sameLanguage(expected, actual));
}
}
}

View File

@ -223,6 +223,7 @@ public class TestDocValuesIndexing extends LuceneTestCase {
return cfg;
}
@SuppressWarnings("fallthrough")
public void runTestNumerics(IndexWriterConfig cfg, boolean withDeletions)
throws IOException {
Directory d = newDirectory();

View File

@ -155,8 +155,6 @@ public class TestAutomatonQuery extends LuceneTestCase {
assertEquals(a1, a3);
assertEquals(a1.toString(), a3.toString());
// different class
AutomatonQuery w1 = new WildcardQuery(newTerm("foobar"));
// different class

View File

@ -87,6 +87,7 @@ public class TestBooleanMinShouldMatch extends LuceneTestCase {
printHits(getName(), h, s);
}
assertEquals("result count", expected, h.length);
//System.out.println("TEST: now check");
QueryUtils.check(random, q,s);
}

View File

@ -75,6 +75,9 @@ public class TestFuzzyQuery2 extends LuceneTestCase {
}
public void assertFromTestData(int codePointTable[]) throws Exception {
if (VERBOSE) {
System.out.println("TEST: codePointTable=" + codePointTable);
}
InputStream stream = getClass().getResourceAsStream("fuzzyTestData.txt");
BufferedReader reader = new BufferedReader(new InputStreamReader(stream, "UTF-8"));
@ -84,6 +87,8 @@ public class TestFuzzyQuery2 extends LuceneTestCase {
Directory dir = newDirectory();
RandomIndexWriter writer = new RandomIndexWriter(random, dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random, MockTokenizer.KEYWORD, false)).setMergePolicy(newLogMergePolicy()));
writer.w.setInfoStream(VERBOSE ? System.out : null);
Document doc = new Document();
Field field = newField("field", "", Field.Store.NO, Field.Index.ANALYZED);
doc.add(field);
@ -95,6 +100,9 @@ public class TestFuzzyQuery2 extends LuceneTestCase {
IndexReader r = writer.getReader();
IndexSearcher searcher = newSearcher(r);
if (VERBOSE) {
System.out.println("TEST: searcher=" + searcher);
}
writer.close();
String line;
while ((line = reader.readLine()) != null) {

View File

@ -94,12 +94,18 @@ public class TestMultiTermConstantScore extends BaseTestRangeFilter {
public static Query csrq(String f, String l, String h, boolean il, boolean ih) {
TermRangeQuery query = TermRangeQuery.newStringRange(f, l, h, il, ih);
query.setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_FILTER_REWRITE);
if (VERBOSE) {
System.out.println("TEST: query=" + query);
}
return query;
}
public static Query csrq(String f, String l, String h, boolean il, boolean ih, MultiTermQuery.RewriteMethod method) {
TermRangeQuery query = TermRangeQuery.newStringRange(f, l, h, il, ih);
query.setRewriteMethod(method);
if (VERBOSE) {
System.out.println("TEST: query=" + query + " method=" + method);
}
return query;
}
@ -275,6 +281,10 @@ public class TestMultiTermConstantScore extends BaseTestRangeFilter {
IndexReader reader = signedIndexReader;
IndexSearcher search = newSearcher(reader);
if (VERBOSE) {
System.out.println("TEST: reader=" + reader);
}
int medId = ((maxId - minId) / 2);
String minIP = pad(minId);

View File

@ -18,28 +18,27 @@ package org.apache.lucene.search;
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.ArrayList;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.UnicodeUtil;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.AutomatonTestUtil;
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
@ -143,6 +142,9 @@ public class TestRegexpRandom2 extends LuceneTestCase {
int num = CodecProvider.getDefault().getFieldCodec("field").equals("PreFlex") ? 100 * RANDOM_MULTIPLIER : atLeast(1000);
for (int i = 0; i < num; i++) {
String reg = AutomatonTestUtil.randomRegexp(random);
if (VERBOSE) {
System.out.println("TEST: regexp=" + reg);
}
assertSame(reg);
}
}
@ -154,17 +156,6 @@ public class TestRegexpRandom2 extends LuceneTestCase {
RegexpQuery smart = new RegexpQuery(new Term("field", regexp), RegExp.NONE);
DumbRegexpQuery dumb = new DumbRegexpQuery(new Term("field", regexp), RegExp.NONE);
// we can't compare the two if automaton rewrites to a simpler enum.
// for example: "a\uda07\udcc7?.*?" gets rewritten to a simpler query:
// a\uda07* prefixquery. Prefixquery then does the "wrong" thing, which
// isn't really wrong as the query was undefined to begin with... but not
// automatically comparable.
// TODO: does this check even matter anymore?!
Terms terms = MultiFields.getTerms(searcher1.getIndexReader(), "field");
if (!(smart.getTermsEnum(terms) instanceof AutomatonTermsEnum))
return;
TopDocs smartDocs = searcher1.search(smart, 25);
TopDocs dumbDocs = searcher2.search(dumb, 25);

View File

@ -135,7 +135,7 @@ public class TestWildcard
wq = new WildcardQuery(new Term("field", "*"));
assertMatches(searcher, wq, 2);
assertFalse(wq.getTermsEnum(terms) instanceof PrefixTermsEnum);
assertFalse(wq.getTermsEnum(terms) instanceof AutomatonTermsEnum);
assertFalse(wq.getTermsEnum(terms).getClass().getSimpleName().contains("AutomatonTermsEnum"));
searcher.close();
indexStore.close();
}

View File

@ -63,6 +63,9 @@ public class TestWildcardRandom extends LuceneTestCase {
reader = writer.getReader();
searcher = newSearcher(reader);
writer.close();
if (VERBOSE) {
System.out.println("TEST: setUp searcher=" + searcher);
}
}
private char N() {
@ -85,7 +88,11 @@ public class TestWildcardRandom extends LuceneTestCase {
private void assertPatternHits(String pattern, int numHits) throws Exception {
// TODO: run with different rewrites
Query wq = new WildcardQuery(new Term("field", fillPattern(pattern)));
final String filledPattern = fillPattern(pattern);
if (VERBOSE) {
System.out.println("TEST: run wildcard pattern=" + pattern + " filled=" + filledPattern);
}
Query wq = new WildcardQuery(new Term("field", filledPattern));
TopDocs docs = searcher.search(wq, 25);
assertEquals("Incorrect hits for pattern: " + pattern, numHits, docs.totalHits);
}

View File

@ -152,11 +152,14 @@ public class TestPayloadNearQuery extends LuceneTestCase {
}
for (int i=1;i<10;i++) {
query = newPhraseQuery("field", English.intToEnglish(i)+" hundred", true, new AveragePayloadFunction());
if (VERBOSE) {
System.out.println("TEST: run query=" + query);
}
// all should have score = 3 because adjacent terms have payloads of 2,4
// and all the similarity factors are set to 1
hits = searcher.search(query, null, 100);
assertTrue("hits is null and it shouldn't be", hits != null);
assertTrue("should be 100 hits", hits.totalHits == 100);
assertEquals("should be 100 hits", 100, hits.totalHits);
for (int j = 0; j < hits.scoreDocs.length; j++) {
ScoreDoc doc = hits.scoreDocs[j];
// System.out.println("Doc: " + doc.toString());

View File

@ -0,0 +1,121 @@
package org.apache.lucene.util.automaton;
/**
* 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.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
public class TestCompiledAutomaton extends LuceneTestCase {
private CompiledAutomaton build(String... strings) {
final List<Automaton> as = new ArrayList<Automaton>();
for(String s : strings) {
as.add(BasicAutomata.makeString(s));
}
Automaton a = BasicOperations.union(as);
a.determinize();
return new CompiledAutomaton(a, true, false);
}
private void testFloor(CompiledAutomaton c, String input, String expected) {
final BytesRef b = new BytesRef(input);
final BytesRef result = c.floor(b, b);
if (expected == null) {
assertNull(result);
} else {
assertNotNull(result);
assertEquals("actual=" + result.utf8ToString() + " vs expected=" + expected + " (input=" + input + ")",
result, new BytesRef(expected));
}
}
private void testTerms(String[] terms) throws Exception {
final CompiledAutomaton c = build(terms);
final BytesRef[] termBytes = new BytesRef[terms.length];
for(int idx=0;idx<terms.length;idx++) {
termBytes[idx] = new BytesRef(terms[idx]);
}
Arrays.sort(termBytes);
if (VERBOSE) {
System.out.println("\nTEST: terms in unicode order");
for(BytesRef t : termBytes) {
System.out.println(" " + t.utf8ToString());
}
//System.out.println(c.utf8.toDot());
}
for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) {
final String s = random.nextInt(10) == 1 ? terms[random.nextInt(terms.length)] : randomString();
if (VERBOSE) {
System.out.println("\nTEST: floor(" + s + ")");
}
int loc = Arrays.binarySearch(termBytes, new BytesRef(s));
final String expected;
if (loc >= 0) {
expected = s;
} else {
// term doesn't exist
loc = -(loc+1);
if (loc == 0) {
expected = null;
} else {
expected = termBytes[loc-1].utf8ToString();
}
}
if (VERBOSE) {
System.out.println(" expected=" + expected);
}
testFloor(c, s, expected);
}
}
public void testRandom() throws Exception {
final int numTerms = atLeast(1000);
final Set<String> terms = new HashSet<String>();
while(terms.size() != numTerms) {
terms.add(randomString());
}
testTerms(terms.toArray(new String[terms.size()]));
}
private String randomString() {
// return _TestUtil.randomSimpleString(random);
return _TestUtil.randomRealisticUnicodeString(random);
}
public void testBasic() throws Exception {
CompiledAutomaton c = build("foo", "fob", "goo");
testFloor(c, "goo", "goo");
testFloor(c, "ga", "foo");
testFloor(c, "g", "foo");
testFloor(c, "foc", "fob");
testFloor(c, "foz", "foo");
testFloor(c, "f", null);
testFloor(c, "", null);
testFloor(c, "aa", null);
testFloor(c, "zzz", "goo");
}
}

View File

@ -24,19 +24,25 @@ import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.OutputStreamWriter;
import java.io.StringWriter;
import java.io.Writer;
import java.util.*;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.IndexInput;
@ -456,7 +462,8 @@ public class TestFSTs extends LuceneTestCase {
prune1==0 && prune2==0,
allowRandomSuffixSharing ? random.nextBoolean() : true,
allowRandomSuffixSharing ? _TestUtil.nextInt(random, 1, 10) : Integer.MAX_VALUE,
outputs);
outputs,
null);
for(InputOutput<T> pair : pairs) {
if (pair.output instanceof UpToTwoPositiveIntOutputs.TwoLongs) {
@ -872,15 +879,15 @@ public class TestFSTs extends LuceneTestCase {
}
}
//System.out.println("TEST: after prune");
/*
for(Map.Entry<BytesRef,CountMinOutput> ent : prefixes.entrySet()) {
System.out.println(" " + inputToString(inputMode, ent.getKey()) + ": isLeaf=" + ent.getValue().isLeaf + " isFinal=" + ent.getValue().isFinal);
if (ent.getValue().isFinal) {
System.out.println(" finalOutput=" + outputs.outputToString(ent.getValue().finalOutput));
if (VERBOSE) {
System.out.println("TEST: after prune");
for(Map.Entry<IntsRef,CountMinOutput<T>> ent : prefixes.entrySet()) {
System.out.println(" " + inputToString(inputMode, ent.getKey()) + ": isLeaf=" + ent.getValue().isLeaf + " isFinal=" + ent.getValue().isFinal);
if (ent.getValue().isFinal) {
System.out.println(" finalOutput=" + outputs.outputToString(ent.getValue().finalOutput));
}
}
}
*/
}
if (prefixes.size() <= 1) {
assertNull(fst);
@ -1081,7 +1088,7 @@ public class TestFSTs extends LuceneTestCase {
final BytesRef randomTerm = new BytesRef(getRandomString());
if (VERBOSE) {
System.out.println("TEST: seek " + randomTerm.utf8ToString() + " " + randomTerm);
System.out.println("TEST: seek non-exist " + randomTerm.utf8ToString() + " " + randomTerm);
}
final TermsEnum.SeekStatus seekResult = termsEnum.seekCeil(randomTerm);
@ -1133,10 +1140,10 @@ public class TestFSTs extends LuceneTestCase {
assertEquals(termsEnum.term().utf8ToString() + " != " + fstEnum.current().input.utf8ToString(), termsEnum.term(), fstEnum.current().input);
if (storeOrd) {
// fst stored the ord
assertEquals(termsEnum.ord(), ((Long) fstEnum.current().output).longValue());
assertEquals("term=" + termsEnum.term().utf8ToString() + " " + termsEnum.term(), termsEnum.ord(), ((Long) fstEnum.current().output).longValue());
} else {
// fst stored the docFreq
assertEquals(termsEnum.docFreq(), (int) (((Long) fstEnum.current().output).longValue()));
assertEquals("term=" + termsEnum.term().utf8ToString() + " " + termsEnum.term(), termsEnum.docFreq(), (int) (((Long) fstEnum.current().output).longValue()));
}
}
}
@ -1154,7 +1161,7 @@ public class TestFSTs extends LuceneTestCase {
this.inputMode = inputMode;
this.outputs = outputs;
builder = new Builder<T>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs);
builder = new Builder<T>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, null);
}
protected abstract T getOutput(IntsRef input, int ord) throws IOException;
@ -1254,7 +1261,7 @@ public class TestFSTs extends LuceneTestCase {
}
}
// java -cp build/classes/test:build/classes/java:build/classes/test-framework:lib/junit-4.7.jar org.apache.lucene.util.fst.TestFSTs /x/tmp/allTerms3.txt out
// java -cp build/classes/test:build/classes/test-framework:build/classes/java:lib/junit-4.7.jar org.apache.lucene.util.automaton.fst.TestFSTs /x/tmp/allTerms3.txt out
public static void main(String[] args) throws IOException {
int prune = 0;
int limit = Integer.MAX_VALUE;
@ -1411,6 +1418,198 @@ public class TestFSTs extends LuceneTestCase {
assertEquals(42, (long) seekResult.output);
}
public void testPrimaryKeys() throws Exception {
Directory dir = newDirectory();
for(int cycle=0;cycle<2;cycle++) {
if (VERBOSE) {
System.out.println("TEST: cycle=" + cycle);
}
RandomIndexWriter w = new RandomIndexWriter(random, dir,
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(IndexWriterConfig.OpenMode.CREATE));
Document doc = new Document();
Field idField = newField("id", "", Field.Index.NOT_ANALYZED);
doc.add(idField);
final int NUM_IDS = (int) (1000*RANDOM_MULTIPLIER*(1.0+random.nextDouble()));
//final int NUM_IDS = (int) (377 * (1.0+random.nextDouble()));
if (VERBOSE) {
System.out.println("TEST: NUM_IDS=" + NUM_IDS);
}
final Set<String> allIDs = new HashSet<String>();
for(int id=0;id<NUM_IDS;id++) {
String idString;
if (cycle == 0) {
// PKs are assigned sequentially
idString = String.format("%07d", id);
} else {
while(true) {
final String s = Long.toString(random.nextLong());
if (!allIDs.contains(s)) {
idString = s;
break;
}
}
}
allIDs.add(idString);
idField.setValue(idString);
w.addDocument(doc);
}
//w.optimize();
// turn writer into reader:
final IndexReader r = w.getReader();
final IndexSearcher s = new IndexSearcher(r);
w.close();
final List<String> allIDsList = new ArrayList<String>(allIDs);
final List<String> sortedAllIDsList = new ArrayList<String>(allIDsList);
Collections.sort(sortedAllIDsList);
// Sprinkle in some non-existent PKs:
Set<String> outOfBounds = new HashSet<String>();
for(int idx=0;idx<NUM_IDS/10;idx++) {
String idString;
if (cycle == 0) {
idString = String.format("%07d", (NUM_IDS + idx));
} else {
while(true) {
idString = Long.toString(random.nextLong());
if (!allIDs.contains(idString)) {
break;
}
}
}
outOfBounds.add(idString);
allIDsList.add(idString);
}
// Verify w/ TermQuery
for(int iter=0;iter<2*NUM_IDS;iter++) {
final String id = allIDsList.get(random.nextInt(allIDsList.size()));
final boolean exists = !outOfBounds.contains(id);
if (VERBOSE) {
System.out.println("TEST: TermQuery " + (exists ? "" : "non-exist ") + " id=" + id);
}
assertEquals((exists ? "" : "non-exist ") + "id=" + id, exists ? 1 : 0, s.search(new TermQuery(new Term("id", id)), 1).totalHits);
}
// Verify w/ MultiTermsEnum
final TermsEnum termsEnum = MultiFields.getTerms(r, "id").iterator();
for(int iter=0;iter<2*NUM_IDS;iter++) {
final String id;
final String nextID;
final boolean exists;
if (random.nextBoolean()) {
id = allIDsList.get(random.nextInt(allIDsList.size()));
exists = !outOfBounds.contains(id);
nextID = null;
if (VERBOSE) {
System.out.println("TEST: exactOnly " + (exists ? "" : "non-exist ") + "id=" + id);
}
} else {
// Pick ID between two IDs:
exists = false;
final int idv = random.nextInt(NUM_IDS-1);
if (cycle == 0) {
id = String.format("%07da", idv);
nextID = String.format("%07d", idv+1);
} else {
id = sortedAllIDsList.get(idv) + "a";
nextID = sortedAllIDsList.get(idv+1);
}
if (VERBOSE) {
System.out.println("TEST: not exactOnly id=" + id + " nextID=" + nextID);
}
}
final boolean useCache = random.nextBoolean();
if (VERBOSE) {
System.out.println(" useCache=" + useCache);
}
final TermsEnum.SeekStatus status;
if (nextID == null) {
if (termsEnum.seekExact(new BytesRef(id), useCache)) {
status = TermsEnum.SeekStatus.FOUND;
} else {
status = TermsEnum.SeekStatus.NOT_FOUND;
}
} else {
status = termsEnum.seekCeil(new BytesRef(id), useCache);
}
if (nextID != null) {
assertEquals(TermsEnum.SeekStatus.NOT_FOUND, status);
assertEquals("expected=" + nextID + " actual=" + termsEnum.term().utf8ToString(), new BytesRef(nextID), termsEnum.term());
} else if (!exists) {
assertTrue(status == TermsEnum.SeekStatus.NOT_FOUND ||
status == TermsEnum.SeekStatus.END);
} else {
assertEquals(TermsEnum.SeekStatus.FOUND, status);
}
}
r.close();
}
dir.close();
}
public void testRandomTermLookup() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random, dir,
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(IndexWriterConfig.OpenMode.CREATE));
w.w.setInfoStream(VERBOSE ? System.out : null);
Document doc = new Document();
Field f = newField("field", "", Field.Index.NOT_ANALYZED);
doc.add(f);
final int NUM_TERMS = (int) (1000*RANDOM_MULTIPLIER * (1+random.nextDouble()));
if (VERBOSE) {
System.out.println("TEST: NUM_TERMS=" + NUM_TERMS);
}
final Set<String> allTerms = new HashSet<String>();
while(allTerms.size() < NUM_TERMS) {
allTerms.add(simpleRandomString(random));
}
for(String term : allTerms) {
f.setValue(term);
w.addDocument(doc);
}
// turn writer into reader:
if (VERBOSE) {
System.out.println("TEST: get reader");
}
IndexReader r = w.getReader();
if (VERBOSE) {
System.out.println("TEST: got reader=" + r);
}
IndexSearcher s = new IndexSearcher(r);
w.close();
final List<String> allTermsList = new ArrayList<String>(allTerms);
Collections.shuffle(allTermsList, random);
// verify exact lookup
for(String term : allTermsList) {
if (VERBOSE) {
System.out.println("TEST: term=" + term);
}
assertEquals("term=" + term, 1, s.search(new TermQuery(new Term("field", term)), 1).totalHits);
}
r.close();
dir.close();
}
/**
* Test state expansion (array format) on close-to-root states. Creates
* synthetic input that has one expanded state on each level.
@ -1492,6 +1691,36 @@ public class TestFSTs extends LuceneTestCase {
s.verifyStateAndBelow(fst, arc, 1);
}
public void testFinalOutputOnEndState() throws Exception {
final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, null);
builder.add("stat", outputs.get(17));
builder.add("station", outputs.get(10));
final FST<Long> fst = builder.finish();
//Writer w = new OutputStreamWriter(new FileOutputStream("/x/tmp/out.dot"));
StringWriter w = new StringWriter();
Util.toDot(fst, w, false, false);
w.close();
//System.out.println(w.toString());
assertTrue(w.toString().indexOf("label=\"t/[7]\"") != -1);
}
public void testInternalFinalState() throws Exception {
final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null);
builder.add(new BytesRef("stat"), outputs.getNoOutput());
builder.add(new BytesRef("station"), outputs.getNoOutput());
final FST<Long> fst = builder.finish();
StringWriter w = new StringWriter();
//Writer w = new OutputStreamWriter(new FileOutputStream("/x/tmp/out.dot"));
Util.toDot(fst, w, false, false);
w.close();
//System.out.println(w.toString());
assertTrue(w.toString().indexOf("6 [shape=doublecircle") != -1);
}
// Make sure raw FST can differentiate between final vs
// non-final end nodes
public void testNonFinalStopNodes() throws Exception {

View File

@ -400,7 +400,7 @@ public class DirectSpellChecker {
if (terms == null) {
return Collections.emptyList();
}
FuzzyTermsEnum e = new FuzzyTermsEnum(terms.iterator(), atts, term, editDistance, Math.max(minPrefix, editDistance-1));
FuzzyTermsEnum e = new FuzzyTermsEnum(terms, atts, term, editDistance, Math.max(minPrefix, editDistance-1));
final PriorityQueue<ScoreTerm> stQueue = new PriorityQueue<ScoreTerm>();
BytesRef queryTerm = new BytesRef(term.text());