LUCENE-5752: current state

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5752@1601967 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2014-06-11 17:35:26 +00:00
parent 670b76492e
commit 7339f52d92
54 changed files with 5083 additions and 371 deletions

View File

@ -447,7 +447,7 @@ public class Dictionary {
throw new UnsupportedOperationException("Too many patterns, please report this to dev@lucene.apache.org");
}
seenPatterns.put(regex, patternIndex);
CharacterRunAutomaton pattern = new CharacterRunAutomaton(new RegExp(regex, RegExp.NONE).toAutomaton());
CharacterRunAutomaton pattern = new CharacterRunAutomaton(new RegExp(regex, RegExp.NONE).toLightAutomaton());
patterns.add(pattern);
}

View File

@ -105,8 +105,9 @@ public class BlockTermsReader extends FieldsProducer {
return field.hashCode() * 31 + term.hashCode();
}
}
// private String segment;
// nocommit
private String segment;
public BlockTermsReader(TermsIndexReaderBase indexReader, Directory dir, FieldInfos fieldInfos, SegmentInfo info, PostingsReaderBase postingsReader, IOContext context,
String segmentSuffix)
@ -114,7 +115,7 @@ public class BlockTermsReader extends FieldsProducer {
this.postingsReader = postingsReader;
// this.segment = segment;
this.segment = segment;
in = dir.openInput(IndexFileNames.segmentFileName(info.name, segmentSuffix, BlockTermsWriter.TERMS_EXTENSION),
context);
@ -364,7 +365,7 @@ 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() + " 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() + " indexIsCurrent=" + indexIsCurrent + " didIndexNext=" + didIndexNext + " seekPending=" + seekPending + " this=" + this);
if (didIndexNext) {
if (nextIndexTerm == null) {
//System.out.println(" nextIndexTerm=null");
@ -450,6 +451,7 @@ public class BlockTermsReader extends FieldsProducer {
// do we then copy the bytes into the term.
while(true) {
//System.out.println("cycle common=" + common + " termBlockPrefix=" + termBlockPrefix + " term=" + term + " target=" + target);
// First, see if target term matches common prefix
// in this block:

View File

@ -46,6 +46,7 @@ import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.LightAutomaton;
import org.apache.lucene.util.automaton.RunAutomaton;
import org.apache.lucene.util.automaton.Transition;
@ -927,10 +928,11 @@ public final class DirectPostingsFormat extends PostingsFormat {
private final class State {
int changeOrd;
int state;
Transition[] transitions;
int transitionUpto;
int transitionCount;
int transitionMax;
int transitionMin;
final LightAutomaton.Transition transition = new LightAutomaton.Transition();
}
private State[] states;
@ -944,7 +946,8 @@ public final class DirectPostingsFormat extends PostingsFormat {
states[0] = new State();
states[0].changeOrd = terms.length;
states[0].state = runAutomaton.getInitialState();
states[0].transitions = compiledAutomaton.sortedTransitions[states[0].state];
states[0].transitionCount = compiledAutomaton.lightAutomaton.getNumTransitions(states[0].state);
compiledAutomaton.lightAutomaton.initTransition(states[0].state, states[0].transition);
states[0].transitionUpto = -1;
states[0].transitionMax = -1;
@ -965,9 +968,10 @@ public final class DirectPostingsFormat extends PostingsFormat {
while (label > states[i].transitionMax) {
states[i].transitionUpto++;
assert states[i].transitionUpto < states[i].transitions.length;
states[i].transitionMin = states[i].transitions[states[i].transitionUpto].getMin();
states[i].transitionMax = states[i].transitions[states[i].transitionUpto].getMax();
assert states[i].transitionUpto < states[i].transitionCount;
compiledAutomaton.lightAutomaton.getNextTransition(states[i].transition);
states[i].transitionMin = states[i].transition.min;
states[i].transitionMax = states[i].transition.max;
assert states[i].transitionMin >= 0;
assert states[i].transitionMin <= 255;
assert states[i].transitionMax >= 0;
@ -1024,7 +1028,8 @@ public final class DirectPostingsFormat extends PostingsFormat {
stateUpto++;
states[stateUpto].changeOrd = skips[skipOffset + skipUpto++];
states[stateUpto].state = nextState;
states[stateUpto].transitions = compiledAutomaton.sortedTransitions[nextState];
states[stateUpto].transitionCount = compiledAutomaton.lightAutomaton.getNumTransitions(nextState);
compiledAutomaton.lightAutomaton.initTransition(states[stateUpto].state, states[stateUpto].transition);
states[stateUpto].transitionUpto = -1;
states[stateUpto].transitionMax = -1;
//System.out.println(" push " + states[stateUpto].transitions.length + " trans");
@ -1178,7 +1183,7 @@ public final class DirectPostingsFormat extends PostingsFormat {
while (label > state.transitionMax) {
//System.out.println(" label=" + label + " vs max=" + state.transitionMax + " transUpto=" + state.transitionUpto + " vs " + state.transitions.length);
state.transitionUpto++;
if (state.transitionUpto == state.transitions.length) {
if (state.transitionUpto == state.transitionCount) {
// We've exhausted transitions leaving this
// state; force pop+next/skip now:
//System.out.println("forcepop: stateUpto=" + stateUpto);
@ -1197,9 +1202,10 @@ public final class DirectPostingsFormat extends PostingsFormat {
}
continue nextTerm;
}
assert state.transitionUpto < state.transitions.length: " state.transitionUpto=" + state.transitionUpto + " vs " + state.transitions.length;
state.transitionMin = state.transitions[state.transitionUpto].getMin();
state.transitionMax = state.transitions[state.transitionUpto].getMax();
compiledAutomaton.lightAutomaton.getNextTransition(state.transition);
assert state.transitionUpto < state.transitionCount: " state.transitionUpto=" + state.transitionUpto + " vs " + state.transitionCount;
state.transitionMin = state.transition.min;
state.transitionMax = state.transition.max;
assert state.transitionMin >= 0;
assert state.transitionMin <= 255;
assert state.transitionMax >= 0;
@ -1297,7 +1303,8 @@ public final class DirectPostingsFormat extends PostingsFormat {
stateUpto++;
states[stateUpto].state = nextState;
states[stateUpto].changeOrd = skips[skipOffset + skipUpto++];
states[stateUpto].transitions = compiledAutomaton.sortedTransitions[nextState];
states[stateUpto].transitionCount = compiledAutomaton.lightAutomaton.getNumTransitions(nextState);
compiledAutomaton.lightAutomaton.initTransition(nextState, states[stateUpto].transition);
states[stateUpto].transitionUpto = -1;
states[stateUpto].transitionMax = -1;

View File

@ -347,7 +347,7 @@ final class IntersectTermsEnum extends TermsEnum {
if (currentFrame.suffix != 0) {
final int label = currentFrame.suffixBytes[currentFrame.startBytePos] & 0xff;
while (label > currentFrame.curTransitionMax) {
if (currentFrame.transitionIndex >= currentFrame.transitions.length-1) {
if (currentFrame.transitionIndex >= currentFrame.transitionCount-1) {
// Stop processing this frame -- no further
// matches are possible because we've moved
// beyond what the max transition will allow
@ -359,7 +359,8 @@ final class IntersectTermsEnum extends TermsEnum {
continue nextTerm;
}
currentFrame.transitionIndex++;
currentFrame.curTransitionMax = currentFrame.transitions[currentFrame.transitionIndex].getMax();
compiledAutomaton.lightAutomaton.getNextTransition(currentFrame.transition);
currentFrame.curTransitionMax = currentFrame.transition.max;
//if (DEBUG) System.out.println(" next trans=" + currentFrame.transitions[currentFrame.transitionIndex]);
}
}

View File

@ -24,6 +24,7 @@ import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.automaton.LightAutomaton;
import org.apache.lucene.util.automaton.Transition;
import org.apache.lucene.util.fst.FST;
@ -68,9 +69,10 @@ final class IntersectTermsEnumFrame {
int numFollowFloorBlocks;
int nextFloorLabel;
Transition[] transitions;
LightAutomaton.Transition transition = new LightAutomaton.Transition();
int curTransitionMax;
int transitionIndex;
int transitionCount;
FST.Arc<BytesRef> arc;
@ -112,7 +114,7 @@ final class IntersectTermsEnumFrame {
nextFloorLabel = 256;
}
// if (DEBUG) System.out.println(" nextFloorLabel=" + (char) nextFloorLabel);
} while (numFollowFloorBlocks != 0 && nextFloorLabel <= transitions[transitionIndex].getMin());
} while (numFollowFloorBlocks != 0 && nextFloorLabel <= transition.min);
load(null);
}
@ -120,9 +122,11 @@ final class IntersectTermsEnumFrame {
public void setState(int state) {
this.state = state;
transitionIndex = 0;
transitions = ite.compiledAutomaton.sortedTransitions[state];
if (transitions.length != 0) {
curTransitionMax = transitions[0].getMax();
transitionCount = ite.compiledAutomaton.lightAutomaton.getNumTransitions(state);
if (transitionCount != 0) {
ite.compiledAutomaton.lightAutomaton.initTransition(state, transition);
ite.compiledAutomaton.lightAutomaton.getNextTransition(transition);
curTransitionMax = transition.max;
} else {
curTransitionMax = -1;
}
@ -132,7 +136,7 @@ final class IntersectTermsEnumFrame {
// if (DEBUG) System.out.println(" load fp=" + fp + " fpOrig=" + fpOrig + " frameIndexData=" + frameIndexData + " trans=" + (transitions.length != 0 ? transitions[0] : "n/a" + " state=" + state));
if (frameIndexData != null && transitions.length != 0) {
if (frameIndexData != null && transitionCount != 0) {
// Floor frame
if (floorData.length < frameIndexData.length) {
this.floorData = new byte[ArrayUtil.oversize(frameIndexData.length, 1)];
@ -151,7 +155,8 @@ final class IntersectTermsEnumFrame {
// first block in case it has empty suffix:
if (!ite.runAutomaton.isAccept(state)) {
// Maybe skip floor blocks:
while (numFollowFloorBlocks != 0 && nextFloorLabel <= transitions[0].getMin()) {
assert transitionIndex == 0: "transitionIndex=" + transitionIndex;
while (numFollowFloorBlocks != 0 && nextFloorLabel <= transition.min) {
fp = fpOrig + (floorDataReader.readVLong() >>> 1);
numFollowFloorBlocks--;
// if (DEBUG) System.out.println(" skip floor block! nextFloorLabel=" + (char) nextFloorLabel + " vs target=" + (char) transitions[0].getMin() + " newFP=" + fp + " numFollowFloorBlocks=" + numFollowFloorBlocks);

View File

@ -24,6 +24,7 @@ import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.automaton.ByteRunAutomaton;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.LightAutomaton;
import org.apache.lucene.util.automaton.Transition;
/**
@ -51,7 +52,7 @@ class AutomatonTermsEnum extends FilteredTermsEnum {
// true if the automaton accepts a finite language
private final boolean finite;
// array of sorted transitions for each state, indexed by state number
private final Transition[][] allTransitions;
private final LightAutomaton lightAutomaton;
// for path tracking: each long records gen when we last
// visited the state; we use gens to avoid having to clear
private final long[] visited;
@ -79,7 +80,7 @@ class AutomatonTermsEnum extends FilteredTermsEnum {
this.runAutomaton = compiled.runAutomaton;
assert this.runAutomaton != null;
this.commonSuffixRef = compiled.commonSuffixRef;
this.allTransitions = compiled.sortedTransitions;
this.lightAutomaton = compiled.lightAutomaton;
// used for path tracking, where each bit is a numbered state.
visited = new long[runAutomaton.getSize()];
@ -124,6 +125,8 @@ class AutomatonTermsEnum extends FilteredTermsEnum {
}
}
private LightAutomaton.Transition transition = new LightAutomaton.Transition();
/**
* Sets the enum to operate in linear fashion, as we have found
* a looping transition at position: we set an upper bound and
@ -133,16 +136,20 @@ class AutomatonTermsEnum extends FilteredTermsEnum {
assert linear == false;
int state = runAutomaton.getInitialState();
assert state == 0;
int maxInterval = 0xff;
//System.out.println("setLinear pos=" + position + " seekbytesRef=" + seekBytesRef);
for (int i = 0; i < position; i++) {
state = runAutomaton.step(state, seekBytesRef.bytes[i] & 0xff);
assert state >= 0: "state=" + state;
}
for (int i = 0; i < allTransitions[state].length; i++) {
Transition t = allTransitions[state][i];
if (t.getMin() <= (seekBytesRef.bytes[position] & 0xff) &&
(seekBytesRef.bytes[position] & 0xff) <= t.getMax()) {
maxInterval = t.getMax();
final int numTransitions = lightAutomaton.getNumTransitions(state);
lightAutomaton.initTransition(state, transition);
for (int i = 0; i < numTransitions; i++) {
lightAutomaton.getNextTransition(transition);
if (transition.min <= (seekBytesRef.bytes[position] & 0xff) &&
(seekBytesRef.bytes[position] & 0xff) <= transition.max) {
maxInterval = transition.max;
break;
}
}
@ -250,19 +257,19 @@ class AutomatonTermsEnum extends FilteredTermsEnum {
seekBytesRef.length = position;
visited[state] = curGen;
Transition transitions[] = allTransitions[state];
final int numTransitions = lightAutomaton.getNumTransitions(state);
lightAutomaton.initTransition(state, transition);
// find the minimal path (lexicographic order) that is >= c
for (int i = 0; i < transitions.length; i++) {
Transition transition = transitions[i];
if (transition.getMax() >= c) {
int nextChar = Math.max(c, transition.getMin());
for (int i = 0; i < numTransitions; i++) {
lightAutomaton.getNextTransition(transition);
if (transition.max >= c) {
int nextChar = Math.max(c, transition.min);
// append either the next sequential char, or the minimum transition
seekBytesRef.grow(seekBytesRef.length + 1);
seekBytesRef.length++;
seekBytesRef.bytes[seekBytesRef.length - 1] = (byte) nextChar;
state = transition.getDest().getNumber();
state = transition.dest;
/*
* as long as is possible, continue down the minimal path in
* lexicographic order. if a loop or accept state is encountered, stop.
@ -274,13 +281,14 @@ class AutomatonTermsEnum extends FilteredTermsEnum {
* so the below is ok, if it is not an accept state,
* then there MUST be at least one transition.
*/
transition = allTransitions[state][0];
state = transition.getDest().getNumber();
lightAutomaton.initTransition(state, transition);
lightAutomaton.getNextTransition(transition);
state = transition.dest;
// append the minimum transition
seekBytesRef.grow(seekBytesRef.length + 1);
seekBytesRef.length++;
seekBytesRef.bytes[seekBytesRef.length - 1] = (byte) transition.getMin();
seekBytesRef.bytes[seekBytesRef.length - 1] = (byte) transition.min;
// we found a loop, record it for faster enumeration
if (!finite && !linear && visited[state] == curGen) {

View File

@ -26,6 +26,7 @@ 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.CompiledAutomaton;
import org.apache.lucene.util.automaton.LightAutomaton;
/**
* A {@link Query} that will match terms against a finite-state machine.
@ -48,6 +49,7 @@ import org.apache.lucene.util.automaton.CompiledAutomaton;
public class AutomatonQuery extends MultiTermQuery {
/** the automaton to match index terms against */
protected final Automaton automaton;
protected final LightAutomaton lightAutomaton;
protected final CompiledAutomaton compiled;
/** term containing the field, and possibly some pattern structure */
protected final Term term;
@ -64,6 +66,15 @@ public class AutomatonQuery extends MultiTermQuery {
super(term.field());
this.term = term;
this.automaton = automaton;
this.lightAutomaton = null;
this.compiled = new CompiledAutomaton(automaton);
}
public AutomatonQuery(final Term term, LightAutomaton automaton) {
super(term.field());
this.term = term;
this.automaton = null;
this.lightAutomaton = automaton;
this.compiled = new CompiledAutomaton(automaton);
}
@ -110,7 +121,11 @@ public class AutomatonQuery extends MultiTermQuery {
buffer.append(getClass().getSimpleName());
buffer.append(" {");
buffer.append('\n');
buffer.append(automaton.toString());
if (automaton == null) {
buffer.append(lightAutomaton.toString());
} else {
buffer.append(automaton.toString());
}
buffer.append("}");
buffer.append(ToStringUtils.boost(getBoost()));
return buffer.toString();
@ -120,4 +135,9 @@ public class AutomatonQuery extends MultiTermQuery {
public Automaton getAutomaton() {
return automaton;
}
/** Returns the light automaton used to create this query */
public LightAutomaton getLightAutomaton() {
return lightAutomaton;
}
}

View File

@ -1,10 +1,10 @@
package org.apache.lucene.search;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.ToStringUtils;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.AutomatonProvider;
import org.apache.lucene.util.automaton.LightAutomaton;
import org.apache.lucene.util.automaton.LightAutomatonProvider;
import org.apache.lucene.util.automaton.RegExp;
/*
@ -50,9 +50,9 @@ public class RegexpQuery extends AutomatonQuery {
/**
* A provider that provides no named automata
*/
private static AutomatonProvider defaultProvider = new AutomatonProvider() {
private static LightAutomatonProvider defaultProvider = new LightAutomatonProvider() {
@Override
public Automaton getAutomaton(String name) {
public LightAutomaton getAutomaton(String name) {
return null;
}
};
@ -86,8 +86,8 @@ public class RegexpQuery extends AutomatonQuery {
* @param flags optional RegExp features from {@link RegExp}
* @param provider custom AutomatonProvider for named automata
*/
public RegexpQuery(Term term, int flags, AutomatonProvider provider) {
super(term, new RegExp(term.text(), flags).toAutomaton(provider));
public RegexpQuery(Term term, int flags, LightAutomatonProvider provider) {
super(term, new RegExp(term.text(), flags).toLightAutomaton(provider));
}
/** Prints a user-readable version of this query. */

View File

@ -17,14 +17,15 @@ package org.apache.lucene.search;
* limitations under the License.
*/
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.index.Term;
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 java.util.ArrayList;
import java.util.List;
import org.apache.lucene.util.automaton.LightAutomaton;
/** Implements the wildcard search query. Supported wildcards are <code>*</code>, which
* matches any character sequence (including the empty one), and <code>?</code>,
@ -54,7 +55,7 @@ public class WildcardQuery extends AutomatonQuery {
* Constructs a query for terms matching <code>term</code>.
*/
public WildcardQuery(Term term) {
super(term, toAutomaton(term));
super(term, toLightAutomaton(term));
}
/**
@ -93,6 +94,43 @@ public class WildcardQuery extends AutomatonQuery {
return BasicOperations.concatenate(automata);
}
/**
* Convert Lucene wildcard syntax into an automaton.
* @lucene.internal
*/
@SuppressWarnings("fallthrough")
public static LightAutomaton toLightAutomaton(Term wildcardquery) {
List<LightAutomaton> automata = new ArrayList<>();
String wildcardText = wildcardquery.text();
for (int i = 0; i < wildcardText.length();) {
final int c = wildcardText.codePointAt(i);
int length = Character.charCount(c);
switch(c) {
case WILDCARD_STRING:
automata.add(BasicAutomata.makeAnyStringLight());
break;
case WILDCARD_CHAR:
automata.add(BasicAutomata.makeAnyCharLight());
break;
case WILDCARD_ESCAPE:
// add the next codepoint instead, if it exists
if (i + length < wildcardText.length()) {
final int nextChar = wildcardText.codePointAt(i + length);
length += Character.charCount(nextChar);
automata.add(BasicAutomata.makeCharLight(nextChar));
break;
} // else fallthru, lenient parsing with a trailing \
default:
automata.add(BasicAutomata.makeCharLight(c));
}
i += length;
}
return BasicOperations.concatenateLight(automata);
}
/**
* Returns the pattern term.

View File

@ -29,6 +29,8 @@
package org.apache.lucene.util.automaton;
import java.io.IOException;
import java.io.PrintWriter;
import java.util.Arrays;
import java.util.BitSet;
import java.util.Collection;
@ -667,7 +669,7 @@ public class Automaton implements Cloneable {
if (allow_mutation) return this;
else return clone();
}
/**
* See {@link BasicOperations#concatenate(Automaton, Automaton)}.
*/
@ -774,4 +776,52 @@ public class Automaton implements Cloneable {
MinimizationOperations.minimize(a);
return a;
}
public LightAutomaton toLightAutomaton() {
State[] states = getNumberedStates();
LightAutomaton a = new LightAutomaton();
//System.out.println("INITIAL: " + initial);
// State numbers are nearly the same, except we must remap initial to state 0
int[] oldToNew = new int[states.length];
for(int i=0;i<states.length;i++) {
oldToNew[i] = i;
}
oldToNew[initial.number] = 0;
oldToNew[0] = initial.number;
//System.out.println("initial.number=" + initial.number);
// First pass creates all states
for(int i=0;i<states.length;i++) {
a.createState();
a.setAccept(oldToNew[i], states[i].isAccept());
}
// Second pass carries over all transitions
for(State state : states) {
for(Transition t : state.getTransitions()) {
assert t.getMax() >= t.getMin();
a.addTransition(oldToNew[state.number], oldToNew[t.getDest().getNumber()], t.getMin(), t.getMax());
//System.out.println(" add transtion " + oldToNew[state.number] + " -> " + oldToNew[t.getDest().getNumber()] + " min=" + (char) t.getMin() + " max=" + (char) t.getMax());
}
}
// a.writeDot("/l/la/lucene/core/afterconvert.dot");
a.finish();
return a;
}
public void writeDot(String fileName) {
if (fileName.indexOf('/') == -1) {
fileName = "/l/la/lucene/core/" + fileName + ".dot";
}
try {
PrintWriter pw = new PrintWriter(fileName);
pw.println(toDot());
pw.close();
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
}

View File

@ -52,6 +52,12 @@ final public class BasicAutomata {
a.deterministic = true;
return a;
}
public static LightAutomaton makeEmptyLight() {
LightAutomaton a = new LightAutomaton();
a.finish();
return a;
}
/**
* Returns a new (deterministic) automaton that accepts only the empty string.
@ -62,6 +68,16 @@ final public class BasicAutomata {
a.deterministic = true;
return a;
}
/**
* Returns a new (deterministic) automaton that accepts only the empty string.
*/
public static LightAutomaton makeEmptyStringLight() {
LightAutomaton a = new LightAutomaton();
a.createState();
a.setAccept(0, true);
return a;
}
/**
* Returns a new (deterministic) automaton that accepts all strings.
@ -76,6 +92,18 @@ final public class BasicAutomata {
a.deterministic = true;
return a;
}
/**
* Returns a new (deterministic) automaton that accepts all strings.
*/
public static LightAutomaton makeAnyStringLight() {
LightAutomaton a = new LightAutomaton();
int s = a.createState();
a.setAccept(s, true);
a.addTransition(s, s, Character.MIN_CODE_POINT, Character.MAX_CODE_POINT);
a.finish();
return a;
}
/**
* Returns a new (deterministic) automaton that accepts any single codepoint.
@ -83,6 +111,13 @@ final public class BasicAutomata {
public static Automaton makeAnyChar() {
return makeCharRange(Character.MIN_CODE_POINT, Character.MAX_CODE_POINT);
}
/**
* Returns a new (deterministic) automaton that accepts any single codepoint.
*/
public static LightAutomaton makeAnyCharLight() {
return makeCharRangeLight(Character.MIN_CODE_POINT, Character.MAX_CODE_POINT);
}
/**
* Returns a new (deterministic) automaton that accepts a single codepoint of
@ -94,6 +129,14 @@ final public class BasicAutomata {
a.deterministic = true;
return a;
}
/**
* Returns a new (deterministic) automaton that accepts a single codepoint of
* the given value.
*/
public static LightAutomaton makeCharLight(int c) {
return makeCharRangeLight(c, c);
}
/**
* Returns a new (deterministic) automaton that accepts a single codepoint whose
@ -111,6 +154,22 @@ final public class BasicAutomata {
return a;
}
/**
* Returns a new (deterministic) automaton that accepts a single codepoint whose
* value is in the given interval (including both end points).
*/
public static LightAutomaton makeCharRangeLight(int min, int max) {
LightAutomaton a = new LightAutomaton();
int s1 = a.createState();
int s2 = a.createState();
a.setAccept(s2, true);
if (min <= max) {
a.addTransition(s1, s2, min, max);
}
a.finish();
return a;
}
/**
* Constructs sub-automaton corresponding to decimal numbers of length
* x.substring(n).length().
@ -121,6 +180,20 @@ final public class BasicAutomata {
else s.addTransition(new Transition('0', '9', anyOfRightLength(x, n + 1)));
return s;
}
/**
* Constructs sub-automaton corresponding to decimal numbers of length
* x.substring(n).length().
*/
private static int anyOfRightLengthLight(LightAutomaton.Builder builder, String x, int n) {
int s = builder.createState();
if (x.length() == n) {
builder.setAccept(s, true);
} else {
builder.addTransition(s, anyOfRightLengthLight(builder, x, n + 1), '0', '9');
}
return s;
}
/**
* Constructs sub-automaton corresponding to decimal numbers of value at least
@ -140,6 +213,28 @@ final public class BasicAutomata {
}
return s;
}
/**
* Constructs sub-automaton corresponding to decimal numbers of value at least
* x.substring(n) and length x.substring(n).length().
*/
private static int atLeastLight(LightAutomaton.Builder builder, String x, int n, Collection<Integer> initials,
boolean zeros) {
int s = builder.createState();
if (x.length() == n) {
builder.setAccept(s, true);
} else {
if (zeros) {
initials.add(s);
}
char c = x.charAt(n);
builder.addTransition(s, atLeastLight(builder, x, n + 1, initials, zeros && c == '0'), c);
if (c < '9') {
builder.addTransition(s, anyOfRightLengthLight(builder, x, n + 1), (char) (c + 1), '9');
}
}
return s;
}
/**
* Constructs sub-automaton corresponding to decimal numbers of value at most
@ -156,6 +251,24 @@ final public class BasicAutomata {
}
return s;
}
/**
* Constructs sub-automaton corresponding to decimal numbers of value at most
* x.substring(n) and length x.substring(n).length().
*/
private static int atMostLight(LightAutomaton.Builder builder, String x, int n) {
int s = builder.createState();
if (x.length() == n) {
builder.setAccept(s, true);
} else {
char c = x.charAt(n);
builder.addTransition(s, atMostLight(builder, x, (char) n + 1), c);
if (c > '0') {
builder.addTransition(s, anyOfRightLengthLight(builder, x, n + 1), '0', (char) (c - 1));
}
}
return s;
}
/**
* Constructs sub-automaton corresponding to decimal numbers of value between
@ -182,6 +295,37 @@ final public class BasicAutomata {
}
return s;
}
/**
* Constructs sub-automaton corresponding to decimal numbers of value between
* x.substring(n) and y.substring(n) and of length x.substring(n).length()
* (which must be equal to y.substring(n).length()).
*/
private static int betweenLight(LightAutomaton.Builder builder,
String x, String y, int n,
Collection<Integer> initials, boolean zeros) {
int s = builder.createState();
if (x.length() == n) {
builder.setAccept(s, true);
} else {
if (zeros) {
initials.add(s);
}
char cx = x.charAt(n);
char cy = y.charAt(n);
if (cx == cy) {
builder.addTransition(s, betweenLight(builder, x, y, n + 1, initials, zeros && cx == '0'), cx);
} else { // cx<cy
builder.addTransition(s, atLeastLight(builder, x, n + 1, initials, zeros && cx == '0'), cx);
builder.addTransition(s, atMostLight(builder, y, n + 1), cy);
if (cx + 1 < cy) {
builder.addTransition(s, anyOfRightLengthLight(builder, x, n+1), (char) (cx + 1), (char) (cy - 1));
}
}
}
return s;
}
/**
* Returns a new automaton that accepts strings representing decimal
@ -229,6 +373,70 @@ final public class BasicAutomata {
a.checkMinimizeAlways();
return a;
}
/**
* Returns a new automaton that accepts strings representing decimal
* non-negative integers in the given interval.
*
* @param min minimal value of interval
* @param max maximal value of interval (both end points are included in the
* interval)
* @param digits if >0, use fixed number of digits (strings must be prefixed
* by 0's to obtain the right length) - otherwise, the number of
* digits is not fixed
* @exception IllegalArgumentException if min>max or if numbers in the
* interval cannot be expressed with the given fixed number of
* digits
*/
public static LightAutomaton makeIntervalLight(int min, int max, int digits)
throws IllegalArgumentException {
String x = Integer.toString(min);
String y = Integer.toString(max);
if (min > max || (digits > 0 && y.length() > digits)) {
throw new IllegalArgumentException();
}
int d;
if (digits > 0) d = digits;
else d = y.length();
StringBuilder bx = new StringBuilder();
for (int i = x.length(); i < d; i++) {
bx.append('0');
}
bx.append(x);
x = bx.toString();
StringBuilder by = new StringBuilder();
for (int i = y.length(); i < d; i++) {
by.append('0');
}
by.append(y);
y = by.toString();
LightAutomaton.Builder builder = new LightAutomaton.Builder();
Collection<Integer> initials = new ArrayList<>();
betweenLight(builder, x, y, 0, initials, digits <= 0);
LightAutomaton a1 = builder.finish();
if (digits <= 0) {
LightAutomaton a2 = new LightAutomaton();
a2.createState();
// TODO: can we somehow do this w/o a full copy here?
a2.copy(a1);
for (int p : initials) {
if (p != 0) {
a2.addEpsilon(0, p+1);
}
}
a2.finish();
return a2;
} else {
return a1;
}
}
/**
* Returns a new (deterministic) automaton that accepts the single given
@ -240,6 +448,26 @@ final public class BasicAutomata {
a.deterministic = true;
return a;
}
/**
* Returns a new (deterministic) automaton that accepts the single given
* string.
*/
public static LightAutomaton makeStringLight(String s) {
LightAutomaton a = new LightAutomaton();
int lastState = a.createState();
for (int i = 0, cp = 0; i < s.length(); i += Character.charCount(cp)) {
int state = a.createState();
cp = s.codePointAt(i);
a.addTransition(lastState, state, cp, cp);
lastState = state;
}
a.setAccept(lastState, true);
a.finish();
return a;
}
public static Automaton makeString(int[] word, int offset, int length) {
Automaton a = new Automaton();
@ -275,4 +503,25 @@ final public class BasicAutomata {
return DaciukMihovAutomatonBuilder.build(utf8Strings);
}
}
/**
* Returns a new (deterministic and minimal) automaton that accepts the union
* of the given collection of {@link BytesRef}s representing UTF-8 encoded
* strings.
*
* @param utf8Strings
* The input strings, UTF-8 encoded. The collection must be in sorted
* order.
*
* @return An {@link Automaton} accepting all input strings. The resulting
* automaton is codepoint based (full unicode codepoints on
* transitions).
*/
public static LightAutomaton makeStringUnionLight(Collection<BytesRef> utf8Strings) {
if (utf8Strings.isEmpty()) {
return makeEmptyLight();
} else {
return DaciukMihovAutomatonBuilderLight.build(utf8Strings);
}
}
}

View File

@ -26,11 +26,20 @@ public class ByteRunAutomaton extends RunAutomaton {
this(a, false);
}
public ByteRunAutomaton(LightAutomaton a) {
this(a, false);
}
/** expert: if utf8 is true, the input is already byte-based */
public ByteRunAutomaton(Automaton a, boolean utf8) {
super(utf8 ? a : new UTF32ToUTF8().convert(a), 256, true);
}
/** expert: if utf8 is true, the input is already byte-based */
public ByteRunAutomaton(LightAutomaton a, boolean utf8) {
super(utf8 ? a : new UTF32ToUTF8Light().convert(a), 256, true);
}
/**
* Returns true if the given byte array is accepted by this automaton
*/

View File

@ -26,6 +26,10 @@ public class CharacterRunAutomaton extends RunAutomaton {
super(a, Character.MAX_CODE_POINT, false);
}
public CharacterRunAutomaton(LightAutomaton a) {
super(a, Character.MAX_CODE_POINT, false);
}
/**
* Returns true if the given string is accepted by this automaton.
*/

View File

@ -65,15 +65,14 @@ public class CompiledAutomaton {
* only valid for {@link AUTOMATON_TYPE#NORMAL}.
*/
public final ByteRunAutomaton runAutomaton;
// TODO: would be nice if these sortedTransitions had "int
// to;" instead of "State to;" somehow:
/**
* Two dimensional array of transitions, indexed by state
* number for traversal. The state numbering is consistent with
* {@link #runAutomaton}.
* Only valid for {@link AUTOMATON_TYPE#NORMAL}.
*/
public final Transition[][] sortedTransitions;
public final LightAutomaton lightAutomaton;
/**
* Shared common suffix accepted by the automaton. Only valid
* for {@link AUTOMATON_TYPE#NORMAL}, and only when the
@ -92,7 +91,7 @@ public class CompiledAutomaton {
}
public CompiledAutomaton(Automaton automaton, Boolean finite, boolean simplify) {
//automaton.writeDot("/l/la/lucene/core/ca.dot");
if (simplify) {
// Test whether the automaton is a "simple" form and
// if so, don't create a runAutomaton. Note that on a
@ -103,7 +102,7 @@ public class CompiledAutomaton {
term = null;
commonSuffixRef = null;
runAutomaton = null;
sortedTransitions = null;
lightAutomaton = null;
this.finite = null;
return;
} else if (BasicOperations.isTotal(automaton)) {
@ -112,7 +111,7 @@ public class CompiledAutomaton {
term = null;
commonSuffixRef = null;
runAutomaton = null;
sortedTransitions = null;
lightAutomaton = null;
this.finite = null;
return;
} else {
@ -137,7 +136,7 @@ public class CompiledAutomaton {
term = new BytesRef(singleton);
commonSuffixRef = null;
runAutomaton = null;
sortedTransitions = null;
lightAutomaton = null;
this.finite = null;
return;
} else if (BasicOperations.sameLanguage(automaton, BasicOperations.concatenate(
@ -147,7 +146,7 @@ public class CompiledAutomaton {
term = new BytesRef(commonPrefix);
commonSuffixRef = null;
runAutomaton = null;
sortedTransitions = null;
lightAutomaton = null;
this.finite = null;
return;
}
@ -156,56 +155,175 @@ public class CompiledAutomaton {
type = AUTOMATON_TYPE.NORMAL;
term = null;
LightAutomaton la = automaton.toLightAutomaton();
if (finite == null) {
this.finite = SpecialOperations.isFinite(la);
} else {
this.finite = finite;
}
//System.out.println("finite=" + this.finite);
//System.out.println("\nPRE");
//automaton.writeDot("/l/la/lucene/core/ain.dot");
//System.out.println("\nNOW BUILD");
//la.writeDot("/l/la/lucene/core/la.dot");
LightAutomaton utf8 = new UTF32ToUTF8Light().convert(la);
if (this.finite) {
commonSuffixRef = null;
} else {
// nocommit fixme
// commonSuffixRef = SpecialOperations.getCommonSuffixBytesRef(utf8);
commonSuffixRef = null;
}
runAutomaton = new ByteRunAutomaton(utf8, true);
lightAutomaton = runAutomaton.a;
}
public CompiledAutomaton(LightAutomaton automaton) {
this(automaton, null, true);
}
public CompiledAutomaton(LightAutomaton automaton, Boolean finite, boolean simplify) {
//System.out.println("CA simplify=" + simplify);
//automaton.writeDot("ca");
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;
lightAutomaton = null;
this.finite = null;
return;
} else if (BasicOperations.isTotal(automaton)) {
// matches all possible strings
type = AUTOMATON_TYPE.ALL;
term = null;
commonSuffixRef = null;
runAutomaton = null;
lightAutomaton = null;
this.finite = null;
return;
} else {
automaton = BasicOperations.determinize(automaton);
final String commonPrefix = SpecialOperations.getCommonPrefix(automaton);
final String singleton;
if (commonPrefix.length() > 0 && BasicOperations.sameLanguage(automaton, BasicAutomata.makeStringLight(commonPrefix))) {
singleton = commonPrefix;
} else {
singleton = null;
}
//System.out.println("CHECK PREFIX: commonPrefix=" + commonPrefix);
if (singleton != null) {
// matches a fixed string
type = AUTOMATON_TYPE.SINGLE;
term = new BytesRef(singleton);
commonSuffixRef = null;
runAutomaton = null;
lightAutomaton = null;
this.finite = null;
return;
} else if (commonPrefix.length() > 0) {
LightAutomaton other = BasicOperations.concatenateLight(BasicAutomata.makeStringLight(commonPrefix), BasicAutomata.makeAnyStringLight());
other = BasicOperations.determinize(other);
if (BasicOperations.sameLanguage(automaton, other)) {
// matches a constant prefix
type = AUTOMATON_TYPE.PREFIX;
term = new BytesRef(commonPrefix);
commonSuffixRef = null;
runAutomaton = null;
lightAutomaton = 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);
LightAutomaton utf8 = new UTF32ToUTF8Light().convert(automaton);
if (this.finite) {
commonSuffixRef = null;
} else {
commonSuffixRef = SpecialOperations.getCommonSuffixBytesRef(utf8);
}
runAutomaton = new ByteRunAutomaton(utf8, true);
sortedTransitions = utf8.getSortedTransitions();
//utf8.writeDot("utf8");
lightAutomaton = runAutomaton.a;
}
private LightAutomaton.Transition scratch = new LightAutomaton.Transition();
//private static final boolean DEBUG = BlockTreeTermsWriter.DEBUG;
private BytesRef addTail(int state, BytesRef term, int idx, int leadLabel) {
//System.out.println("addTail state=" + state + " term=" + term.utf8ToString() + " idx=" + idx + " leadLabel=" + (char) leadLabel);
//System.out.println(lightAutomaton.toDot());
// 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;
lightAutomaton.initTransition(state, scratch);
int numTransitions = lightAutomaton.getNumTransitions(state);
int maxIndex = -1;
int lastMin = 0;
for(int i=0;i<numTransitions;i++) {
lightAutomaton.getNextTransition(scratch);
if (scratch.min < leadLabel) {
maxIndex = i;
}
assert scratch.min >= lastMin;
lastMin = scratch.min;
// nocommit else break?
}
assert maxTransition != null;
//System.out.println(" maxIndex=" + maxIndex);
assert maxIndex != -1;
lightAutomaton.getTransition(state, maxIndex, scratch);
// Append floorLabel
final int floorLabel;
if (maxTransition.max > leadLabel-1) {
if (scratch.max > leadLabel-1) {
floorLabel = leadLabel-1;
} else {
floorLabel = maxTransition.max;
floorLabel = scratch.max;
}
//System.out.println(" floorLabel=" + (char) floorLabel);
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();
state = scratch.dest;
//System.out.println(" dest: " + state);
idx++;
// Push down to last accept state
while (true) {
Transition[] transitions = sortedTransitions[state];
if (transitions.length == 0) {
numTransitions = lightAutomaton.getNumTransitions(state);
if (numTransitions == 0) {
//System.out.println("state=" + state + " 0 trans");
assert runAutomaton.isAccept(state);
term.length = idx;
//if (DEBUG) System.out.println(" return " + term.utf8ToString());
@ -213,14 +331,15 @@ public class CompiledAutomaton {
} else {
// We are pushing "top" -- so get last label of
// last transition:
assert transitions.length != 0;
Transition lastTransition = transitions[transitions.length-1];
//System.out.println("get state=" + state + " numTrans=" + numTransitions);
lightAutomaton.getTransition(state, numTransitions-1, scratch);
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();
//System.out.println(" add trans dest=" + scratch.dest + " label=" + (char) scratch.max);
term.bytes[idx] = (byte) scratch.max;
state = scratch.dest;
idx++;
}
}
@ -301,33 +420,36 @@ public class CompiledAutomaton {
// Pop back to a state that has a transition
// <= our label:
while (true) {
Transition[] transitions = sortedTransitions[state];
if (transitions.length == 0) {
int numTransitions = lightAutomaton.getNumTransitions(state);
if (numTransitions == 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;
lightAutomaton.getTransition(state, 0, scratch);
if (label-1 < scratch.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;
}
}
}
@ -350,18 +472,27 @@ public class CompiledAutomaton {
public String toDot() {
StringBuilder b = new StringBuilder("digraph CompiledAutomaton {\n");
b.append(" rankdir = LR;\n");
int initial = runAutomaton.getInitialState();
for (int i = 0; i < sortedTransitions.length; i++) {
int initial = 0;
for (int i = 0; i < lightAutomaton.getNumStates(); i++) {
b.append(" ").append(i);
if (runAutomaton.isAccept(i)) b.append(" [shape=doublecircle,label=\"\"];\n");
if (lightAutomaton.isAccept(i)) b.append(" [shape=doublecircle,label=\"\"];\n");
else b.append(" [shape=circle,label=\"\"];\n");
if (i == initial) {
if (i == 0) {
b.append(" initial [shape=plaintext,label=\"\"];\n");
b.append(" initial -> ").append(i).append("\n");
}
for (int j = 0; j < sortedTransitions[i].length; j++) {
lightAutomaton.initTransition(i, scratch);
int numTransitions = lightAutomaton.getNumTransitions(i);
for (int j = 0; j < numTransitions; j++) {
b.append(" ").append(i);
sortedTransitions[i][j].appendDot(b);
b.append(" -> ");
b.append(scratch.dest);
b.append(scratch.min);
if (scratch.min != scratch.max) {
b.append("-");
b.append(scratch.max);
}
lightAutomaton.getNextTransition(scratch);
}
}
return b.append("}\n").toString();

View File

@ -0,0 +1,334 @@
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 {@link Automaton} that accepts a set of
* strings. The algorithm requires sorted input data, but is very fast
* (nearly linear with the input size).
*
* @see #build(Collection)
* @see BasicAutomata#makeStringUnion(Collection)
*/
final class DaciukMihovAutomatonBuilderLight {
/**
* DFSA state with <code>char</code> labels on transitions.
*/
private 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>.
*/
State getState(int label) {
final int index = Arrays.binarySearch(labels, label);
return index >= 0 ? states[index] : null;
}
/**
* 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);
}
/**
* 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;
}
/**
* Return <code>true</code> if this state has any children (outgoing
* transitions).
*/
boolean hasChildren() {
return labels.length > 0;
}
/**
* 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 = Arrays.copyOf(labels, labels.length + 1);
states = Arrays.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;
}
/**
* 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;
}
}
/**
* A "registry" for state interning.
*/
private HashMap<State,State> stateRegistry = new HashMap<>();
/**
* Root automaton state.
*/
private State root = new State();
/**
* Previous sequence added to the automaton in {@link #add(CharsRef)}.
*/
private CharsRef previous;
/**
* A comparator used for enforcing sorted UTF8 order, used in assertions only.
*/
@SuppressWarnings("deprecation")
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 stateRegistry != null : "Automaton already built.";
assert previous == null
|| comparator.compare(previous, current) <= 0 : "Input must be in sorted UTF-8 order: "
+ 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.stateRegistry == null) throw new IllegalStateException();
if (root.hasChildren()) replaceOrRegister(root);
stateRegistry = null;
return root;
}
/**
* Internal recursive traversal for conversion.
*/
private static int convert(LightAutomaton.Builder a, State s,
IdentityHashMap<State,Integer> visited) {
Integer converted = visited.get(s);
if (converted != null) {
return converted;
}
converted = a.createState();
a.setAccept(converted, s.is_final);
visited.put(s, converted);
int i = 0;
int[] labels = s.labels;
for (DaciukMihovAutomatonBuilderLight.State target : s.states) {
a.addTransition(converted, convert(a, target, visited), labels[i++]);
}
return converted;
}
/**
* Build a minimal, deterministic automaton from a sorted list of {@link BytesRef} representing
* strings in UTF-8. These strings must be binary-sorted.
*/
public static LightAutomaton build(Collection<BytesRef> input) {
final DaciukMihovAutomatonBuilderLight builder = new DaciukMihovAutomatonBuilderLight();
CharsRef scratch = new CharsRef();
for (BytesRef b : input) {
UnicodeUtil.UTF8toUTF16(b, scratch);
builder.add(scratch);
}
LightAutomaton.Builder a = new LightAutomaton.Builder();
convert(a,
builder.complete(),
new IdentityHashMap<State,Integer>());
return a.finish();
}
/**
* 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 = CharsRef.deepCopyOf(current);
return true;
}
/**
* Replace last child of <code>state</code> with an already registered state
* or stateRegistry the last child state.
*/
private void replaceOrRegister(State state) {
final State child = state.lastChild();
if (child.hasChildren()) replaceOrRegister(child);
final State registered = stateRegistry.get(child);
if (registered != null) {
state.replaceLastChild(registered);
} else {
stateRegistry.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

@ -178,6 +178,69 @@ public class LevenshteinAutomata {
//a.restoreInvariant();
return a;
}
/**
* Compute a DFA that accepts all strings within an edit distance of <code>n</code>.
* <p>
* All automata have the following properties:
* <ul>
* <li>They are deterministic (DFA).
* <li>There are no transitions to dead states.
* <li>They are not minimal (some transitions could be combined).
* </ul>
* </p>
*/
public LightAutomaton toLightAutomaton(int n) {
if (n == 0) {
return BasicAutomata.makeString(word, 0, word.length).toLightAutomaton();
}
if (n >= descriptions.length)
return null;
final int range = 2*n+1;
ParametricDescription description = descriptions[n];
// the number of states is based on the length of the word and n
int numStates = description.size();
LightAutomaton a = new LightAutomaton();
// create all states, and mark as accept states if appropriate
for (int i = 0; i < numStates; i++) {
a.createState();
a.setAccept(i, description.isAccept(i));
}
// create transitions from state to state
for (int k = 0; k < numStates; k++) {
final int xpos = description.getPosition(k);
if (xpos < 0)
continue;
final int end = xpos + Math.min(word.length - xpos, range);
for (int x = 0; x < alphabet.length; x++) {
final int ch = alphabet[x];
// get the characteristic vector at this position wrt ch
final int cvec = getVector(ch, xpos, end);
int dest = description.transition(k, xpos, cvec);
if (dest >= 0) {
a.addTransition(k, dest, ch);
}
}
// add transitions for all other chars in unicode
// by definition, their characteristic vectors are always 0,
// because they do not exist in the input string.
int dest = description.transition(k, xpos, 0); // by definition
if (dest >= 0) {
for (int r = 0; r < numRanges; r++) {
a.addTransition(k, dest, rangeLower[r], rangeUpper[r]);
}
}
}
a.finish();
return a;
}
/**
* Get the characteristic vector <code>X(x, V)</code>

View File

@ -0,0 +1,793 @@
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.io.PrintWriter;
import java.util.Arrays;
import java.util.BitSet;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.Set;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.InPlaceMergeSorter;
import org.apache.lucene.util.Sorter;
// nocommit make tests that do the same ops w/ old and new and assertSameLang
// TODO
// - could use packed int arrays instead
// - could encode dest w/ delta from to?
// nocommit should we keep determinized bit?
/** Uses only int[]s to represent the automaton, but requires that all
* transitions for each state are added at once. If this is too restrictive,
* use {@link #Builder} instead. State 0 is always the
* initial state.
*
* @lucene.experimental */
// nocommit rename to Automaton once everything is cutover
public class LightAutomaton {
private int nextState;
/** Where we next write to in int[] transitions; this
* increments by 3 for each added transition because we
* pack min, max, dest in sequence. */
private int nextTransition;
/** Current state we are adding transitions to; the caller
* must add all transitions for this state before moving
* onto another state. */
private int curState = -1;
/** Index in the transitions array, where this states
* leaving transitions are stored, or -1 if this state
* has not added any transitions yet, followed by number
* of transitions. */
private int[] states = new int[4];
/** Holds toState, min, max for each transition: */
// nocommit inefficient when labels are really bytes (max 256)
private int[] transitions = new int[6];
private final Set<Integer> finalStates = new HashSet<Integer>();
public int createState() {
growStates();
int state = nextState/2;
states[nextState] = -1;
nextState += 2;
return state;
}
/** Mark this state as an accept state. */
public void setAccept(int state, boolean isAccept) {
if (isAccept) {
finalStates.add(state);
} else {
finalStates.remove(state);
}
}
public boolean isEmpty() {
return finalStates.isEmpty();
}
/** Sugar, but object-heavy; it's better to iterate instead. */
public Transition[][] getSortedTransitions() {
int numStates = getNumStates();
Transition[][] transitions = new Transition[numStates][];
for(int s=0;s<numStates;s++) {
int numTransitions = getNumTransitions(s);
transitions[s] = new Transition[numTransitions];
for(int t=0;t<numTransitions;t++) {
Transition transition = new Transition();
getTransition(s, t, transition);
transitions[s][t] = transition;
}
}
return transitions;
}
public Set<Integer> getAcceptStates() {
return finalStates;
}
/** Returns true if this state is an accept state. */
public boolean isAccept(int state) {
return finalStates.contains(state);
}
public void addTransition(int source, int dest, int label) {
addTransition(source, dest, label, label);
}
public void addTransition(int source, int dest, int min, int max) {
assert nextTransition%3 == 0;
if (source >= nextState/2) {
throw new IllegalArgumentException("source is out of bounds");
}
if (dest >= nextState/2) {
throw new IllegalArgumentException("dest is out of bounds");
}
//System.out.println(" addTransition nextTransition=" + nextTransition + " source=" + source + " dest=" + dest + " min=" + min + " max=" + max);
growTransitions();
if (curState != source) {
//System.out.println(" newstate");
if (curState != -1) {
finishCurrentState();
}
// Move to next source:
curState = source;
if (states[2*curState] != -1) {
throw new IllegalStateException("from state (" + source + ") already had transitions added");
}
assert states[2*curState+1] == 0;
states[2*curState] = nextTransition;
}
transitions[nextTransition++] = dest;
transitions[nextTransition++] = min;
transitions[nextTransition++] = max;
// Increment transition count for this state
states[2*curState+1]++;
}
public void addEpsilon(int source, int dest) {
Transition t = new Transition();
int count = initTransition(dest, t);
for(int i=0;i<count;i++) {
getNextTransition(t);
addTransition(source, t.dest, t.min, t.max);
}
if (isAccept(dest)) {
setAccept(source, true);
}
}
/** Copies over all states/transitions from other. */
public void copy(LightAutomaton other) {
int offset = getNumStates();
int otherNumStates = other.getNumStates();
for(int s=0;s<otherNumStates;s++) {
createState();
setAccept(offset+s, other.isAccept(s));
}
Transition t = new Transition();
for(int s=0;s<otherNumStates;s++) {
int count = other.initTransition(s, t);
for(int i=0;i<count;i++) {
other.getNextTransition(t);
addTransition(offset + s, offset + t.dest, t.min, t.max);
}
}
}
/** Freezes the last state, reducing and sorting its transitions. */
private void finishCurrentState() {
int numTransitions = states[2*curState+1];
assert numTransitions > 0;
// System.out.println("finish curState=" + curState + " numTransitions=" + numTransitions);
int offset = states[2*curState];
int start = offset/3;
destMinMaxSorter.sort(start, start+numTransitions);
/*
for(int i=0;i<numTransitions;i++) {
System.out.println(" " + i + ": dest=" + transitions[offset+3*i] + " (accept?=" + isAccept(transitions[offset+3*i]) + ") min=" + transitions[offset+3*i+1] + " max=" + transitions[offset+3*i+2]);
}
*/
// Reduce any "adjacent" transitions:
int upto = 0;
int min = -1;
int max = -1;
int dest = -1;
for(int i=0;i<numTransitions;i++) {
int tDest = transitions[offset+3*i];
int tMin = transitions[offset+3*i+1];
int tMax = transitions[offset+3*i+2];
if (dest == tDest) {
if (tMin <= max+1) {
if (tMax > max) {
max = tMax;
}
} else {
if (dest != -1) {
transitions[offset+3*upto] = dest;
transitions[offset+3*upto+1] = min;
transitions[offset+3*upto+2] = max;
upto++;
}
min = tMin;
max = tMax;
}
} else {
if (dest != -1) {
transitions[offset+3*upto] = dest;
transitions[offset+3*upto+1] = min;
transitions[offset+3*upto+2] = max;
upto++;
}
dest = tDest;
min = tMin;
max = tMax;
}
}
if (dest != -1) {
// Last transition
transitions[offset+3*upto] = dest;
transitions[offset+3*upto+1] = min;
transitions[offset+3*upto+2] = max;
upto++;
}
nextTransition -= (numTransitions-upto)*3;
states[2*curState+1] = upto;
// Sort transitions by min/max/dest:
minMaxDestSorter.sort(start, start+upto);
/*
System.out.println("after finish: reduce collapsed " + (numTransitions-upto) + " transitions");
for(int i=0;i<upto;i++) {
System.out.println(" " + i + ": dest=" + transitions[offset+3*i] + " (accept?=" + isAccept(transitions[offset+3*i]) + ") min=" + transitions[offset+3*i+1] + " max=" + transitions[offset+3*i+2]);
}
*/
}
public void finish() {
if (curState != -1) {
//System.out.println("finish: finish current state " + curState);
finishCurrentState();
curState = -1;
}
// nocommit downsize the arrays?
//assert getNumStates() > 0;
}
public int getNumStates() {
return nextState/2;
}
public int getNumTransitions(int state) {
//assert curState == -1: "not finished";
int count = states[2*state+1];
if (count == -1) {
return 0;
} else {
return count;
}
}
public int getDest(int state, int transitionIndex) {
return transitions[states[2*state]];
}
public int getMin(int state, int transitionIndex) {
return transitions[states[2*state]+1];
}
public int getMax(int state, int transitionIndex) {
return transitions[states[2*state]+2];
}
private void growStates() {
if (nextState+2 >= states.length) {
states = ArrayUtil.grow(states, nextState+2);
}
}
private void growTransitions() {
if (nextTransition+3 >= transitions.length) {
transitions = ArrayUtil.grow(transitions, nextTransition+3);
}
}
/** Sorts transitions by dest, ascending, then min label ascending, then max label ascending */
private final Sorter destMinMaxSorter = new InPlaceMergeSorter() {
private void swapOne(int i, int j) {
int x = transitions[i];
transitions[i] = transitions[j];
transitions[j] = x;
}
@Override
protected void swap(int i, int j) {
int iStart = 3*i;
int jStart = 3*j;
swapOne(iStart, jStart);
swapOne(iStart+1, jStart+1);
swapOne(iStart+2, jStart+2);
};
@Override
protected int compare(int i, int j) {
int iStart = 3*i;
int jStart = 3*j;
// First dest:
int iDest = transitions[iStart];
int jDest = transitions[jStart];
if (iDest < jDest) {
return -1;
} else if (iDest > jDest) {
return 1;
}
// Then min:
int iMin = transitions[iStart+1];
int jMin = transitions[jStart+1];
if (iMin < jMin) {
return -1;
} else if (iMin > jMin) {
return 1;
}
// Then max:
int iMax = transitions[iStart+2];
int jMax = transitions[jStart+2];
if (iMax < jMax) {
return -1;
} else if (iMax > jMax) {
return 1;
}
return 0;
}
};
/** Sorts transitions by min label, ascending, then max label ascending, then dest ascending */
private final Sorter minMaxDestSorter = new InPlaceMergeSorter() {
private void swapOne(int i, int j) {
int x = transitions[i];
transitions[i] = transitions[j];
transitions[j] = x;
}
@Override
protected void swap(int i, int j) {
int iStart = 3*i;
int jStart = 3*j;
swapOne(iStart, jStart);
swapOne(iStart+1, jStart+1);
swapOne(iStart+2, jStart+2);
};
@Override
protected int compare(int i, int j) {
int iStart = 3*i;
int jStart = 3*j;
// First min:
int iMin = transitions[iStart+1];
int jMin = transitions[jStart+1];
if (iMin < jMin) {
return -1;
} else if (iMin > jMin) {
return 1;
}
// Then max:
int iMax = transitions[iStart+2];
int jMax = transitions[jStart+2];
if (iMax < jMax) {
return -1;
} else if (iMax > jMax) {
return 1;
}
// Then dest:
int iDest = transitions[iStart];
int jDest = transitions[jStart];
if (iDest < jDest) {
return -1;
} else if (iDest > jDest) {
return 1;
}
return 0;
}
};
/** Just used temporarily to return the transition from
* {@link getTransition} and {@link #getNextTransition}. */
public static class Transition {
// used only for assert:
public int source;
public int dest;
public int min;
public int max;
/** Remembers where we are in the iteration; init to -1 to provoke
* exception if nextTransition is called without first initTransition. */
private int transitionUpto = -1;
@Override
public String toString() {
return source + " --> " + dest + " " + (char) min + "-" + (char) max;
}
// nocommit equals? hashCode? don't want to encourage putting these into a Map...?
}
// nocommit createStates(int count)?
// nocommit kinda awkward iterator api...
/** Initialize the provided Transition for iteration; you
* must call {@link #getNextTransition} to get the first
* transition for the state. Returns the number of transitions
* leaving this state. */
public int initTransition(int state, Transition t) {
// assert curState == -1: "not finished";
t.source = state;
//System.out.println("initTrans source=" + state + " numTrans=" + getNumTransitions(state));
t.transitionUpto = states[2*state];
return getNumTransitions(state);
}
/** Iterate to the next transition after the provided one */
public void getNextTransition(Transition t) {
//assert curState == -1: "not finished";
// Make sure there is still a transition left:
//System.out.println("getNextTrans transUpto=" + t.transitionUpto);
//System.out.println(" states[2*t.source]=" + states[2*t.source] + " numTrans=" + states[2*t.source+1] + " transitionUpto+3=" + (t.transitionUpto+3) + " t=" + t);
assert (t.transitionUpto+3 - states[2*t.source]) <= 3*states[2*t.source+1];
t.dest = transitions[t.transitionUpto++];
t.min = transitions[t.transitionUpto++];
t.max = transitions[t.transitionUpto++];
}
/** Fill the provided {@link Transition} with the index'th
* transition leaving the specified state. */
public void getTransition(int state, int index, Transition t) {
assert curState == -1: "not finished";
int i = states[2*state] + 3*index;
t.source = state;
t.dest = transitions[i++];
t.min = transitions[i++];
t.max = transitions[i++];
}
private static void appendCharString(int c, StringBuilder b) {
if (c >= 0x21 && c <= 0x7e && c != '\\' && c != '"') b.appendCodePoint(c);
else {
b.append("\\\\U");
String s = Integer.toHexString(c);
if (c < 0x10) b.append("0000000").append(s);
else if (c < 0x100) b.append("000000").append(s);
else if (c < 0x1000) b.append("00000").append(s);
else if (c < 0x10000) b.append("0000").append(s);
else if (c < 0x100000) b.append("000").append(s);
else if (c < 0x1000000) b.append("00").append(s);
else if (c < 0x10000000) b.append("0").append(s);
else b.append(s);
}
}
public LightAutomaton totalize() {
LightAutomaton result = new LightAutomaton();
int numStates = getNumStates();
for(int i=0;i<numStates;i++) {
result.createState();
result.setAccept(i, isAccept(i));
}
int deadState = result.createState();
result.addTransition(deadState, deadState, Character.MIN_CODE_POINT, Character.MAX_CODE_POINT);
Transition t = new Transition();
for(int i=0;i<numStates;i++) {
int maxi = Character.MIN_CODE_POINT;
int count = initTransition(i, t);
for(int j=0;j<count;j++) {
getNextTransition(t);
result.addTransition(i, t.dest, t.min, t.max);
if (t.min > maxi) {
result.addTransition(i, deadState, maxi, t.min-1);
}
if (t.max + 1 > maxi) {
maxi = t.max + 1;
}
}
if (maxi <= Character.MAX_CODE_POINT) {
result.addTransition(i, deadState, maxi, Character.MAX_CODE_POINT);
}
}
result.finish();
return result;
}
public void writeDot(String fileName) {
if (fileName.indexOf('/') == -1) {
fileName = "/l/la/lucene/core/" + fileName + ".dot";
}
try {
PrintWriter pw = new PrintWriter(fileName);
pw.println(toDot());
pw.close();
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
public String toDot() {
// TODO: breadth first search so we can see get layered output...
StringBuilder b = new StringBuilder();
b.append("digraph Automaton {\n");
b.append(" rankdir = LR\n");
final int numStates = getNumStates();
if (numStates > 0) {
b.append(" initial [shape=plaintext,label=\"0\"]\n");
b.append(" initial -> 0\n");
}
Transition t = new Transition();
for(int state=0;state<numStates;state++) {
b.append(" ");
b.append(state);
if (isAccept(state)) {
b.append(" [shape=doublecircle,label=\"" + state + "\"]\n");
} else {
b.append(" [shape=circle,label=\"" + state + "\"]\n");
}
int numTransitions = getNumTransitions(state);
initTransition(state, t);
//System.out.println("toDot: state " + state + " has " + numTransitions + " transitions; t.nextTrans=" + t.transitionUpto);
for(int i=0;i<numTransitions;i++) {
getNextTransition(t);
//System.out.println(" t.nextTrans=" + t.transitionUpto);
assert t.max >= t.min;
b.append(" ");
b.append(state);
b.append(" -> ");
b.append(t.dest);
b.append(" [label=\"");
appendCharString(t.min, b);
if (t.max != t.min) {
b.append('-');
appendCharString(t.max, b);
}
b.append("\"]\n");
//System.out.println(" t=" + t);
}
}
b.append('}');
return b.toString();
}
/**
* Returns sorted array of all interval start points.
*/
int[] getStartPoints() {
Set<Integer> pointset = new HashSet<>();
pointset.add(Character.MIN_CODE_POINT);
//System.out.println("getStartPoints");
for (int s=0;s<nextState;s+=2) {
int trans = states[s];
int limit = trans+3*states[s+1];
//System.out.println(" state=" + (s/2) + " trans=" + trans + " limit=" + limit);
while (trans < limit) {
int min = transitions[trans+1];
int max = transitions[trans+2];
//System.out.println(" min=" + min);
pointset.add(min);
if (max < Character.MAX_CODE_POINT) {
pointset.add(max + 1);
}
trans += 3;
}
}
int[] points = new int[pointset.size()];
int n = 0;
for (Integer m : pointset) {
points[n++] = m;
}
Arrays.sort(points);
return points;
}
/**
* Performs lookup in transitions, assuming determinism.
*
* @param c codepoint to look up
* @return destination state, -1 if no matching outgoing transition
* @see #step(int, Collection)
*/
public int step(int state, int label) {
assert state >= 0;
assert label >= 0;
int trans = states[2*state];
int limit = trans + 3*states[2*state+1];
// nocommit we could do bin search; transitions are sorted
// System.out.println("la.step state=" + state + " label=" + label + " trans=" + trans + " limit=" + limit);
while (trans < limit) {
int dest = transitions[trans];
int min = transitions[trans+1];
int max = transitions[trans+2];
if (min <= label && label <= max) {
//System.out.println(" ret dest=" + dest);
return dest;
}
trans += 3;
}
return -1;
}
/** Records new states and transitions and then {@link
* #finish} creates the {@link LightAutomaton}. Use this
* when it's too restrictive to have to add all transitions
* leaving each state at once. */
public static class Builder {
private int[] transitions = new int[4];
private int nextTransition;
private final LightAutomaton a = new LightAutomaton();
public void addTransition(int from, int to, int label) {
addTransition(from, to, label, label);
}
public void addTransition(int from, int to, int min, int max) {
if (transitions.length < nextTransition+4) {
transitions = ArrayUtil.grow(transitions, nextTransition+4);
}
transitions[nextTransition++] = from;
transitions[nextTransition++] = to;
transitions[nextTransition++] = min;
transitions[nextTransition++] = max;
}
/** Sorts transitions first then min label ascending, then
* max label ascending, then dest ascending */
private final Sorter sorter = new InPlaceMergeSorter() {
private void swapOne(int i, int j) {
int x = transitions[i];
transitions[i] = transitions[j];
transitions[j] = x;
}
@Override
protected void swap(int i, int j) {
int iStart = 4*i;
int jStart = 4*j;
swapOne(iStart, jStart);
swapOne(iStart+1, jStart+1);
swapOne(iStart+2, jStart+2);
swapOne(iStart+3, jStart+3);
};
@Override
protected int compare(int i, int j) {
int iStart = 4*i;
int jStart = 4*j;
// First src:
int iSrc = transitions[iStart];
int jSrc = transitions[jStart];
if (iSrc < jSrc) {
return -1;
} else if (iSrc > jSrc) {
return 1;
}
// Then min:
int iMin = transitions[iStart+2];
int jMin = transitions[jStart+2];
if (iMin < jMin) {
return -1;
} else if (iMin > jMin) {
return 1;
}
// Then max:
int iMax = transitions[iStart+3];
int jMax = transitions[jStart+3];
if (iMax < jMax) {
return -1;
} else if (iMax > jMax) {
return 1;
}
// First dest:
int iDest = transitions[iStart+1];
int jDest = transitions[jStart+1];
if (iDest < jDest) {
return -1;
} else if (iDest > jDest) {
return 1;
}
return 0;
}
};
public LightAutomaton finish() {
//System.out.println("LA.Builder.finish: count=" + (nextTransition/4));
// nocommit: we could make this more efficient,
// e.g. somehow xfer the int[] to the automaton, or
// alloc exactly the right size from the automaton
//System.out.println("finish pending");
sorter.sort(0, nextTransition/4);
int upto = 0;
while (upto < nextTransition) {
a.addTransition(transitions[upto],
transitions[upto+1],
transitions[upto+2],
transitions[upto+3]);
upto += 4;
}
a.finish();
return a;
}
public int createState() {
return a.createState();
}
public void setAccept(int state, boolean accept) {
a.setAccept(state, accept);
}
public boolean isAccept(int state) {
return a.isAccept(state);
}
public int getNumStates() {
return a.getNumStates();
}
/** Copies over all states/transitions from other. */
public void copy(LightAutomaton other) {
int offset = getNumStates();
int otherNumStates = other.getNumStates();
for(int s=0;s<otherNumStates;s++) {
int newState = createState();
setAccept(newState, other.isAccept(s));
}
Transition t = new Transition();
for(int s=0;s<otherNumStates;s++) {
int count = other.initTransition(s, t);
for(int i=0;i<count;i++) {
other.getNextTransition(t);
addTransition(offset + s, offset + t.dest, t.min, t.max);
}
}
}
}
}

View File

@ -0,0 +1,50 @@
/*
* dk.brics.automaton
*
* Copyright (c) 2001-2009 Anders Moeller
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* 1. Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* 3. The name of the author may not be used to endorse or promote products
* derived from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
* IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
* OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
* IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
* NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
package org.apache.lucene.util.automaton;
import java.io.IOException;
/**
* Automaton provider for <code>RegExp.</code>
* {@link RegExp#toAutomaton(AutomatonProvider)}
*
* @lucene.experimental
*/
public interface LightAutomatonProvider {
/**
* Returns automaton of the given name.
*
* @param name automaton name
* @return automaton
* @throws IOException if errors occur
*/
public LightAutomaton getAutomaton(String name) throws IOException;
}

View File

@ -0,0 +1,89 @@
/*
* dk.brics.automaton
*
* Copyright (c) 2001-2009 Anders Moeller
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* 1. Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* 3. The name of the author may not be used to endorse or promote products
* derived from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
* IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
* OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
* IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
* NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
package org.apache.lucene.util.automaton;
/**
* Pair of states.
*
* @lucene.experimental
*/
public class LightStatePair {
int s;
int s1;
int s2;
LightStatePair(int s, int s1, int s2) {
this.s = s;
this.s1 = s1;
this.s2 = s2;
}
/**
* Constructs a new state pair.
*
* @param s1 first state
* @param s2 second state
*/
public LightStatePair(int s1, int s2) {
this.s1 = s1;
this.s2 = s2;
this.s = -1;
}
/**
* Checks for equality.
*
* @param obj object to compare with
* @return true if <tt>obj</tt> represents the same pair of states as this
* pair
*/
@Override
public boolean equals(Object obj) {
if (obj instanceof LightStatePair) {
LightStatePair p = (LightStatePair) obj;
return p.s1 == s1 && p.s2 == s2;
} else return false;
}
/**
* Returns hash code.
*
* @return hash code
*/
@Override
public int hashCode() {
return s1 ^ s2;
}
@Override
public String toString() {
return "LightStatePair(s1=" + s1 + " s2=" + s2 + ")";
}
}

View File

@ -0,0 +1,308 @@
/*
* dk.brics.automaton
*
* Copyright (c) 2001-2009 Anders Moeller
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* 1. Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* 3. The name of the author may not be used to endorse or promote products
* derived from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
* IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
* OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
* IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
* NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
package org.apache.lucene.util.automaton;
import java.util.BitSet;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.LinkedList;
/**
* Operations for minimizing automata.
*
* @lucene.experimental
*/
final public class MinimizationOperationsLight {
private MinimizationOperationsLight() {}
/**
* Minimizes (and determinizes if not already deterministic) the given
* automaton.
*
* @see Automaton#setMinimization(int)
*/
public static LightAutomaton minimize(LightAutomaton a) {
return minimizeHopcroft(a);
}
/**
* Minimizes the given automaton using Hopcroft's algorithm.
*/
public static LightAutomaton minimizeHopcroft(LightAutomaton a) {
if (a.isEmpty()) {
return new LightAutomaton();
}
a = BasicOperations.determinize(a);
//a.writeDot("adet");
if (a.getNumTransitions(0) == 1) {
LightAutomaton.Transition t = new LightAutomaton.Transition();
a.getTransition(0, 0, t);
if (t.dest == 0 && t.min == Character.MIN_CODE_POINT
&& t.max == Character.MAX_CODE_POINT) {
// Accepts all strings
return a;
}
}
a = a.totalize();
//a.writeDot("atot");
// initialize data structures
final int[] sigma = a.getStartPoints();
final int sigmaLen = sigma.length, statesLen = a.getNumStates();
@SuppressWarnings({"rawtypes","unchecked"}) final ArrayList<Integer>[][] reverse =
(ArrayList<Integer>[][]) new ArrayList[statesLen][sigmaLen];
@SuppressWarnings({"rawtypes","unchecked"}) final HashSet<Integer>[] partition =
(HashSet<Integer>[]) new HashSet[statesLen];
@SuppressWarnings({"rawtypes","unchecked"}) final ArrayList<Integer>[] splitblock =
(ArrayList<Integer>[]) new ArrayList[statesLen];
final int[] block = new int[statesLen];
final StateList[][] active = new StateList[statesLen][sigmaLen];
final StateListNode[][] active2 = new StateListNode[statesLen][sigmaLen];
final LinkedList<IntPair> pending = new LinkedList<>();
final BitSet pending2 = new BitSet(sigmaLen*statesLen);
final BitSet split = new BitSet(statesLen),
refine = new BitSet(statesLen), refine2 = new BitSet(statesLen);
for (int q = 0; q < statesLen; q++) {
splitblock[q] = new ArrayList<>();
partition[q] = new HashSet<>();
for (int x = 0; x < sigmaLen; x++) {
active[q][x] = new StateList();
}
}
// find initial partition and reverse edges
for (int q = 0; q < statesLen; q++) {
final int j = a.isAccept(q) ? 0 : 1;
partition[j].add(q);
block[q] = j;
for (int x = 0; x < sigmaLen; x++) {
final ArrayList<Integer>[] r = reverse[a.step(q, sigma[x])];
if (r[x] == null) {
r[x] = new ArrayList<>();
}
r[x].add(q);
}
}
// initialize active sets
for (int j = 0; j <= 1; j++) {
for (int x = 0; x < sigmaLen; x++) {
for (int q : partition[j]) {
if (reverse[q][x] != null) {
active2[q][x] = active[j][x].add(q);
}
}
}
}
// initialize pending
for (int x = 0; x < sigmaLen; x++) {
final int j = (active[0][x].size <= active[1][x].size) ? 0 : 1;
pending.add(new IntPair(j, x));
pending2.set(x*statesLen + j);
}
// process pending until fixed point
int k = 2;
//System.out.println("start min");
while (!pending.isEmpty()) {
//System.out.println(" cycle pending");
final IntPair ip = pending.removeFirst();
final int p = ip.n1;
final int x = ip.n2;
//System.out.println(" pop n1=" + ip.n1 + " n2=" + ip.n2);
pending2.clear(x*statesLen + p);
// find states that need to be split off their blocks
for (StateListNode m = active[p][x].first; m != null; m = m.next) {
final ArrayList<Integer> r = reverse[m.q][x];
if (r != null) {
for (int i : r) {
if (!split.get(i)) {
split.set(i);
final int j = block[i];
splitblock[j].add(i);
if (!refine2.get(j)) {
refine2.set(j);
refine.set(j);
}
}
}
}
}
// refine blocks
for (int j = refine.nextSetBit(0); j >= 0; j = refine.nextSetBit(j+1)) {
final ArrayList<Integer> sb = splitblock[j];
if (sb.size() < partition[j].size()) {
final HashSet<Integer> b1 = partition[j];
final HashSet<Integer> b2 = partition[k];
for (int s : sb) {
b1.remove(s);
b2.add(s);
block[s] = k;
for (int c = 0; c < sigmaLen; c++) {
final StateListNode sn = active2[s][c];
if (sn != null && sn.sl == active[j][c]) {
sn.remove();
active2[s][c] = active[k][c].add(s);
}
}
}
// update pending
for (int c = 0; c < sigmaLen; c++) {
final int aj = active[j][c].size,
ak = active[k][c].size,
ofs = c*statesLen;
if (!pending2.get(ofs + j) && 0 < aj && aj <= ak) {
pending2.set(ofs + j);
pending.add(new IntPair(j, c));
} else {
pending2.set(ofs + k);
pending.add(new IntPair(k, c));
}
}
k++;
}
refine2.clear(j);
for (int s : sb) {
split.clear(s);
}
sb.clear();
}
refine.clear();
}
LightAutomaton result = new LightAutomaton();
LightAutomaton.Transition t = new LightAutomaton.Transition();
//System.out.println(" k=" + k);
// make a new state for each equivalence class, set initial state
int[] stateMap = new int[statesLen];
int[] stateRep = new int[k];
// nocommit maybe LA should be born already with the initial state?
result.createState();
//System.out.println("min: k=" + k);
for (int n = 0; n < k; n++) {
//System.out.println(" n=" + n);
boolean isInitial = false;
for (int q : partition[n]) {
if (q == 0) {
isInitial = true;
//System.out.println(" isInitial!");
break;
}
}
int newState;
if (isInitial) {
newState = 0;
} else {
newState = result.createState();
}
//System.out.println(" newState=" + newState);
for (int q : partition[n]) {
stateMap[q] = newState;
//System.out.println(" q=" + q + " isAccept?=" + a.isAccept(q));
result.setAccept(newState, a.isAccept(q));
stateRep[newState] = q; // select representative
}
}
// build transitions and set acceptance
for (int n = 0; n < k; n++) {
int numTransitions = a.initTransition(stateRep[n], t);
for(int i=0;i<numTransitions;i++) {
a.getNextTransition(t);
//System.out.println(" add trans");
result.addTransition(n, stateMap[t.dest], t.min, t.max);
}
}
result.finish();
//System.out.println(result.getNumStates() + " states");
return BasicOperations.removeDeadTransitions(result);
}
static final class IntPair {
final int n1, n2;
IntPair(int n1, int n2) {
this.n1 = n1;
this.n2 = n2;
}
}
static final class StateList {
int size;
StateListNode first, last;
StateListNode add(int q) {
return new StateListNode(q, this);
}
}
static final class StateListNode {
final int q;
StateListNode next, prev;
final StateList sl;
StateListNode(int q, StateList sl) {
this.q = q;
this.sl = sl;
if (sl.size++ == 0) sl.first = sl.last = this;
else {
sl.last.next = this;
prev = sl.last;
sl.last = this;
}
}
void remove() {
sl.size--;
if (sl.first == this) sl.first = next;
else prev.next = next;
if (sl.last == this) sl.last = prev;
else next.prev = prev;
}
}
}

View File

@ -419,17 +419,17 @@ public class RegExp {
to = e.to;
b = null;
}
/**
* Constructs new <code>Automaton</code> from this <code>RegExp</code>. Same
* as <code>toAutomaton(null)</code> (empty automaton map).
*/
public Automaton toAutomaton() {
return toAutomatonAllowMutate(null, null);
public LightAutomaton toLightAutomaton() {
return toLightAutomaton(null, null);
}
/**
* Constructs new <code>Automaton</code> from this <code>RegExp</code>. The
* Constructs new <code>LightAutomaton</code> from this <code>RegExp</code>. The
* constructed automaton is minimal and deterministic and has no transitions
* to dead states.
*
@ -437,9 +437,9 @@ public class RegExp {
* @exception IllegalArgumentException if this regular expression uses a named
* identifier that is not available from the automaton provider
*/
public Automaton toAutomaton(AutomatonProvider automaton_provider)
public LightAutomaton toLightAutomaton(LightAutomatonProvider automaton_provider)
throws IllegalArgumentException {
return toAutomatonAllowMutate(null, automaton_provider);
return toLightAutomaton(null, automaton_provider);
}
/**
@ -448,49 +448,26 @@ public class RegExp {
* to dead states.
*
* @param automata a map from automaton identifiers to automata (of type
* <code>Automaton</code>).
* <code>LightAutomaton</code>).
* @exception IllegalArgumentException if this regular expression uses a named
* identifier that does not occur in the automaton map
*/
public Automaton toAutomaton(Map<String,Automaton> automata)
public LightAutomaton toLightAutomaton(Map<String,LightAutomaton> automata)
throws IllegalArgumentException {
return toAutomatonAllowMutate(automata, null);
return toLightAutomaton(automata, null);
}
/**
* Sets or resets allow mutate flag. If this flag is set, then automata
* construction uses mutable automata, which is slightly faster but not thread
* safe. By default, the flag is not set.
*
* @param flag if true, the flag is set
* @return previous value of the flag
*/
public boolean setAllowMutate(boolean flag) {
boolean b = allow_mutation;
allow_mutation = flag;
return b;
}
private Automaton toAutomatonAllowMutate(Map<String,Automaton> automata,
AutomatonProvider automaton_provider) throws IllegalArgumentException {
boolean b = false;
if (allow_mutation) b = Automaton.setAllowMutate(true); // thread unsafe
Automaton a = toAutomaton(automata, automaton_provider);
if (allow_mutation) Automaton.setAllowMutate(b);
return a;
}
private Automaton toAutomaton(Map<String,Automaton> automata,
AutomatonProvider automaton_provider) throws IllegalArgumentException {
List<Automaton> list;
Automaton a = null;
private LightAutomaton toLightAutomaton(Map<String,LightAutomaton> automata,
LightAutomatonProvider automaton_provider) throws IllegalArgumentException {
List<LightAutomaton> list;
LightAutomaton a = null;
switch (kind) {
case REGEXP_UNION:
list = new ArrayList<>();
findLeaves(exp1, Kind.REGEXP_UNION, list, automata, automaton_provider);
findLeaves(exp2, Kind.REGEXP_UNION, list, automata, automaton_provider);
a = BasicOperations.union(list);
MinimizationOperations.minimize(a);
a = BasicOperations.unionLight(list);
MinimizationOperationsLight.minimize(a);
break;
case REGEXP_CONCATENATION:
list = new ArrayList<>();
@ -498,77 +475,85 @@ public class RegExp {
automaton_provider);
findLeaves(exp2, Kind.REGEXP_CONCATENATION, list, automata,
automaton_provider);
a = BasicOperations.concatenate(list);
MinimizationOperations.minimize(a);
a = BasicOperations.concatenateLight(list);
MinimizationOperationsLight.minimize(a);
break;
case REGEXP_INTERSECTION:
a = exp1.toAutomaton(automata, automaton_provider).intersection(
exp2.toAutomaton(automata, automaton_provider));
MinimizationOperations.minimize(a);
a = BasicOperations.intersectionLight(
exp1.toLightAutomaton(automata, automaton_provider),
exp2.toLightAutomaton(automata, automaton_provider));
a = MinimizationOperationsLight.minimize(a);
break;
case REGEXP_OPTIONAL:
a = exp1.toAutomaton(automata, automaton_provider).optional();
MinimizationOperations.minimize(a);
a = BasicOperations.optionalLight(exp1.toLightAutomaton(automata, automaton_provider));
a = MinimizationOperationsLight.minimize(a);
break;
case REGEXP_REPEAT:
a = exp1.toAutomaton(automata, automaton_provider).repeat();
MinimizationOperations.minimize(a);
a = BasicOperations.repeatLight(exp1.toLightAutomaton(automata, automaton_provider));
a = MinimizationOperationsLight.minimize(a);
break;
case REGEXP_REPEAT_MIN:
a = exp1.toAutomaton(automata, automaton_provider).repeat(min);
MinimizationOperations.minimize(a);
a = BasicOperations.repeatLight(exp1.toLightAutomaton(automata, automaton_provider), min);
a = MinimizationOperationsLight.minimize(a);
break;
case REGEXP_REPEAT_MINMAX:
a = exp1.toAutomaton(automata, automaton_provider).repeat(min, max);
MinimizationOperations.minimize(a);
a = BasicOperations.repeatLight(exp1.toLightAutomaton(automata, automaton_provider), min, max);
a = MinimizationOperationsLight.minimize(a);
break;
case REGEXP_COMPLEMENT:
a = exp1.toAutomaton(automata, automaton_provider).complement();
MinimizationOperations.minimize(a);
a = BasicOperations.complementLight(exp1.toLightAutomaton(automata, automaton_provider));
a = MinimizationOperationsLight.minimize(a);
break;
case REGEXP_CHAR:
a = BasicAutomata.makeChar(c);
a = BasicAutomata.makeCharLight(c);
break;
case REGEXP_CHAR_RANGE:
a = BasicAutomata.makeCharRange(from, to);
a = BasicAutomata.makeCharRangeLight(from, to);
break;
case REGEXP_ANYCHAR:
a = BasicAutomata.makeAnyChar();
a = BasicAutomata.makeAnyCharLight();
break;
case REGEXP_EMPTY:
a = BasicAutomata.makeEmpty();
a = BasicAutomata.makeEmptyLight();
break;
case REGEXP_STRING:
a = BasicAutomata.makeString(s);
a = BasicAutomata.makeStringLight(s);
break;
case REGEXP_ANYSTRING:
a = BasicAutomata.makeAnyString();
a = BasicAutomata.makeAnyStringLight();
break;
case REGEXP_AUTOMATON:
Automaton aa = null;
if (automata != null) aa = automata.get(s);
if (aa == null && automaton_provider != null) try {
aa = automaton_provider.getAutomaton(s);
} catch (IOException e) {
throw new IllegalArgumentException(e);
LightAutomaton aa = null;
if (automata != null) {
aa = automata.get(s);
}
if (aa == null) throw new IllegalArgumentException("'" + s
+ "' not found");
a = aa.clone(); // always clone here (ignore allow_mutate)
if (aa == null && automaton_provider != null) {
try {
aa = automaton_provider.getAutomaton(s);
} catch (IOException e) {
throw new IllegalArgumentException(e);
}
}
if (aa == null) {
throw new IllegalArgumentException("'" + s + "' not found");
}
a = aa;
break;
case REGEXP_INTERVAL:
a = BasicAutomata.makeInterval(min, max, digits);
a = BasicAutomata.makeIntervalLight(min, max, digits);
break;
}
return a;
}
private void findLeaves(RegExp exp, Kind kind, List<Automaton> list,
Map<String,Automaton> automata, AutomatonProvider automaton_provider) {
private void findLeaves(RegExp exp, Kind kind, List<LightAutomaton> list,
Map<String,LightAutomaton> automata, LightAutomatonProvider automaton_provider) {
if (exp.kind == kind) {
findLeaves(exp.exp1, kind, list, automata, automaton_provider);
findLeaves(exp.exp2, kind, list, automata, automaton_provider);
} else list.add(exp.toAutomaton(automata, automaton_provider));
} else {
list.add(exp.toLightAutomaton(automata, automaton_provider));
}
}
/**

View File

@ -37,6 +37,8 @@ import java.util.Arrays;
* @lucene.experimental
*/
public abstract class RunAutomaton {
// nocommit
final LightAutomaton a;
final int maxInterval;
final int size;
final boolean[] accept;
@ -121,9 +123,13 @@ public abstract class RunAutomaton {
*/
public RunAutomaton(Automaton a, int maxInterval, boolean tableize) {
this.maxInterval = maxInterval;
this.a = null;
a.determinize();
//System.out.println("AFTER DET tableize=" + tableize + ": ");
//System.out.println(a.toDot());
points = a.getStartPoints();
final State[] states = a.getNumberedStates();
//System.out.println(" states=" + states.length);
initial = a.initial.number;
size = states.length;
accept = new boolean[size];
@ -153,6 +159,52 @@ public abstract class RunAutomaton {
classmap = null;
}
}
public RunAutomaton(LightAutomaton a, int maxInterval, boolean tableize) {
this.maxInterval = maxInterval;
//System.out.println("before det a=" + a.getNumStates());
a = BasicOperations.determinize(a);
this.a = a;
//System.out.println("AFTER DET tableize= " + tableize + ": ");
//System.out.println(a.toDot());
points = a.getStartPoints();
//System.out.println(" points=" + Arrays.toString(points));
initial = 0;
size = Math.max(1,a.getNumStates());
accept = new boolean[size];
transitions = new int[size * points.length];
Arrays.fill(transitions, -1);
//System.out.println("RA: size=" + size + " points.length=" + points.length + " total=" + (size * points.length));
for (int n=0;n<size;n++) {
accept[n] = a.isAccept(n);
//System.out.println("n=" + n + " acc=" + accept[n] + " size=" + size);
for (int c = 0; c < points.length; c++) {
int dest = a.step(n, points[c]);
//System.out.println(" step from point=" + c + " n=" + n + " label=" + (char) points[c] + " -> " + dest);
assert dest == -1 || dest < size;
transitions[n * points.length + c] = dest;
//System.out.println(" trans label=" + points[c] + " dest=" + transitions[n * points.length + c]);
}
}
/*
* Set alphabet table for optimal run performance.
*/
if (tableize) {
classmap = new int[maxInterval + 1];
int i = 0;
for (int j = 0; j <= maxInterval; j++) {
if (i + 1 < points.length && j == points[i + 1]) {
i++;
}
classmap[j] = i;
//System.out.println("classmap[" + (char) j + "]=" + i);
}
//System.out.println(" after classmap i=" + i + " maxInterval=" + maxInterval);
} else {
classmap = null;
}
}
/**
* Returns the state obtained by reading the given char from the given state.
@ -162,10 +214,13 @@ public abstract class RunAutomaton {
* transition function.)
*/
public final int step(int state, int c) {
if (classmap == null)
//System.out.println(" step state=" + state + " c=" + c + " points.length=" + points.length + " transitions.len=" + transitions.length);
if (classmap == null) {
return transitions[state * points.length + getCharClass(c)];
else
} else {
//System.out.println(" classmap[c]=" + classmap[c]);
return transitions[state * points.length + classmap[c]];
}
}
@Override

View File

@ -0,0 +1,278 @@
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.TreeMap;
import java.util.Map;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
// Just holds a set of int[] states, plus a corresponding
// int[] count per state. Used by
// BasicOperations.determinize
final class SortedIntSetLight {
int[] values;
int[] counts;
int upto;
private int hashCode;
// If we hold more than this many states, we switch from
// O(N^2) linear ops to O(N log(N)) TreeMap
private final static int TREE_MAP_CUTOVER = 30;
private final Map<Integer,Integer> map = new TreeMap<>();
private boolean useTreeMap;
int state;
public SortedIntSetLight(int capacity) {
values = new int[capacity];
counts = new int[capacity];
}
// Adds this state to the set
public void incr(int num) {
if (useTreeMap) {
final Integer key = num;
Integer val = map.get(key);
if (val == null) {
map.put(key, 1);
} else {
map.put(key, 1+val);
}
return;
}
if (upto == values.length) {
values = ArrayUtil.grow(values, 1+upto);
counts = ArrayUtil.grow(counts, 1+upto);
}
for(int i=0;i<upto;i++) {
if (values[i] == num) {
counts[i]++;
return;
} else if (num < values[i]) {
// insert here
int j = upto-1;
while (j >= i) {
values[1+j] = values[j];
counts[1+j] = counts[j];
j--;
}
values[i] = num;
counts[i] = 1;
upto++;
return;
}
}
// append
values[upto] = num;
counts[upto] = 1;
upto++;
if (upto == TREE_MAP_CUTOVER) {
useTreeMap = true;
for(int i=0;i<upto;i++) {
map.put(values[i], counts[i]);
}
}
}
// Removes this state from the set, if count decrs to 0
public void decr(int num) {
if (useTreeMap) {
final int count = map.get(num);
if (count == 1) {
map.remove(num);
} else {
map.put(num, count-1);
}
// Fall back to simple arrays once we touch zero again
if (map.size() == 0) {
useTreeMap = false;
upto = 0;
}
return;
}
for(int i=0;i<upto;i++) {
if (values[i] == num) {
counts[i]--;
if (counts[i] == 0) {
final int limit = upto-1;
while(i < limit) {
values[i] = values[i+1];
counts[i] = counts[i+1];
i++;
}
upto = limit;
}
return;
}
}
assert false;
}
public void computeHash() {
if (useTreeMap) {
if (map.size() > values.length) {
final int size = ArrayUtil.oversize(map.size(), RamUsageEstimator.NUM_BYTES_INT);
values = new int[size];
counts = new int[size];
}
hashCode = map.size();
upto = 0;
for(int state : map.keySet()) {
hashCode = 683*hashCode + state;
values[upto++] = state;
}
} else {
hashCode = upto;
for(int i=0;i<upto;i++) {
hashCode = 683*hashCode + values[i];
}
}
}
public FrozenIntSetLight freeze(int state) {
final int[] c = new int[upto];
System.arraycopy(values, 0, c, 0, upto);
return new FrozenIntSetLight(c, hashCode, state);
}
@Override
public int hashCode() {
return hashCode;
}
@Override
public boolean equals(Object _other) {
if (_other == null) {
return false;
}
if (!(_other instanceof FrozenIntSetLight)) {
return false;
}
FrozenIntSetLight other = (FrozenIntSetLight) _other;
if (hashCode != other.hashCode) {
return false;
}
if (other.values.length != upto) {
return false;
}
for(int i=0;i<upto;i++) {
if (other.values[i] != values[i]) {
return false;
}
}
return true;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder().append('[');
for(int i=0;i<upto;i++) {
if (i > 0) {
sb.append(' ');
}
sb.append(values[i]).append(':').append(counts[i]);
}
sb.append(']');
return sb.toString();
}
public final static class FrozenIntSetLight {
final int[] values;
final int hashCode;
final int state;
public FrozenIntSetLight(int[] values, int hashCode, int state) {
this.values = values;
this.hashCode = hashCode;
this.state = state;
}
public FrozenIntSetLight(int num, int state) {
this.values = new int[] {num};
this.state = state;
this.hashCode = 683+num;
}
@Override
public int hashCode() {
return hashCode;
}
@Override
public boolean equals(Object _other) {
if (_other == null) {
return false;
}
if (_other instanceof FrozenIntSetLight) {
FrozenIntSetLight other = (FrozenIntSetLight) _other;
if (hashCode != other.hashCode) {
return false;
}
if (other.values.length != values.length) {
return false;
}
for(int i=0;i<values.length;i++) {
if (other.values[i] != values[i]) {
return false;
}
}
return true;
} else if (_other instanceof SortedIntSetLight) {
SortedIntSetLight other = (SortedIntSetLight) _other;
if (hashCode != other.hashCode) {
return false;
}
if (other.values.length != values.length) {
return false;
}
for(int i=0;i<values.length;i++) {
if (other.values[i] != values[i]) {
return false;
}
}
return true;
}
return false;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder().append('[');
for(int i=0;i<values.length;i++) {
if (i > 0) {
sb.append(' ');
}
sb.append(values[i]);
}
sb.append(']');
return sb.toString();
}
}
}

View File

@ -89,6 +89,33 @@ final public class SpecialOperations {
visited.set(s.number);
return true;
}
/**
* Returns true if the language of this automaton is finite.
*/
public static boolean isFinite(LightAutomaton a) {
return isFinite(new LightAutomaton.Transition(), a, 0, new BitSet(a.getNumStates()), new BitSet(a.getNumStates()));
}
/**
* Checks whether there is a loop containing s. (This is sufficient since
* there are never transitions to dead states.)
*/
// TODO: not great that this is recursive... in theory a
// large automata could exceed java's stack
private static boolean isFinite(LightAutomaton.Transition scratch, LightAutomaton a, int state, BitSet path, BitSet visited) {
path.set(state);
int numTransitions = a.initTransition(state, scratch);
for(int t=0;t<numTransitions;t++) {
a.getTransition(state, t, scratch);
if (path.get(scratch.dest) || (!visited.get(scratch.dest) && !isFinite(scratch, a, scratch.dest, path, visited))) {
return false;
}
}
path.clear(state);
visited.set(state);
return true;
}
/**
* Returns the longest string that is a prefix of all accepted strings and
@ -96,6 +123,7 @@ final public class SpecialOperations {
*
* @return common prefix
*/
// nocommit a must be det? we should document if so?
public static String getCommonPrefix(Automaton a) {
if (a.isSingleton()) return a.singleton;
StringBuilder b = new StringBuilder();
@ -116,6 +144,36 @@ final public class SpecialOperations {
} while (!done);
return b.toString();
}
/**
* Returns the longest string that is a prefix of all accepted strings and
* visits each state at most once.
*
* @return common prefix
*/
// nocommit a must be det? we should document if so?
public static String getCommonPrefix(LightAutomaton a) {
//a.writeDot("cp");
StringBuilder b = new StringBuilder();
HashSet<Integer> visited = new HashSet<>();
int s = 0;
boolean done;
LightAutomaton.Transition t = new LightAutomaton.Transition();
do {
done = true;
visited.add(s);
if (a.isAccept(s) == false && a.getNumTransitions(s) == 1) {
a.getTransition(s, 0, t);
if (t.min == t.max && !visited.contains(t.dest)) {
b.appendCodePoint(t.min);
s = t.dest;
done = false;
}
}
} while (!done);
return b.toString();
}
// TODO: this currently requites a determinized machine,
// but it need not -- we can speed it up by walking the
@ -142,6 +200,29 @@ final public class SpecialOperations {
return ref;
}
public static BytesRef getCommonPrefixBytesRef(LightAutomaton a) {
BytesRef ref = new BytesRef(10);
HashSet<Integer> visited = new HashSet<>();
int s = 0;
boolean done;
LightAutomaton.Transition t = new LightAutomaton.Transition();
do {
done = true;
visited.add(s);
if (a.isAccept(s) == false && a.getNumTransitions(s) == 1) {
a.getTransition(s, 0, t);
if (t.min == t.max && !visited.contains(t.dest)) {
ref.grow(++ref.length);
ref.bytes[ref.length - 1] = (byte) t.min;
s = t.dest;
done = false;
}
}
} while (!done);
return ref;
}
/**
* Returns the longest string that is a suffix of all accepted strings and
* visits each state at most once.
@ -171,6 +252,14 @@ final public class SpecialOperations {
reverseBytes(ref);
return ref;
}
public static BytesRef getCommonSuffixBytesRef(LightAutomaton a) {
// reverse the language of the automaton, then reverse its common prefix.
LightAutomaton r = BasicOperations.determinize(reverse(a));
BytesRef ref = getCommonPrefixBytesRef(r);
reverseBytes(ref);
return ref;
}
private static void reverseBytes(BytesRef ref) {
if (ref.length <= 1) return;
@ -216,6 +305,57 @@ final public class SpecialOperations {
return accept;
}
// nocommit merge Special/Basic operations
public static LightAutomaton reverse(LightAutomaton a) {
return reverse(a, null);
}
public static LightAutomaton reverse(LightAutomaton a, Set<Integer> initialStates) {
if (a.isEmpty()) {
return a;
}
int numStates = a.getNumStates();
// Build a new automaton with all edges reversed
LightAutomaton.Builder builder = new LightAutomaton.Builder();
// Initial node; we'll add epsilon transitions in the end:
builder.createState();
for(int s=0;s<numStates;s++) {
builder.createState();
}
// Old initial state becomes new accept state:
builder.setAccept(1, true);
LightAutomaton.Transition t = new LightAutomaton.Transition();
for (int s=0;s<numStates;s++) {
int numTransitions = a.getNumTransitions(s);
a.initTransition(s, t);
for(int i=0;i<numTransitions;i++) {
a.getNextTransition(t);
builder.addTransition(t.dest+1, s+1, t.min, t.max);
}
}
LightAutomaton result = builder.finish();
for(int s : a.getAcceptStates()) {
result.addEpsilon(0, s+1);
if (initialStates != null) {
initialStates.add(s+1);
}
}
result.finish();
return result;
}
private static class PathNode {
/** Which state the path node ends on, whose
@ -371,4 +511,154 @@ final public class SpecialOperations {
return results;
}
private static class LightPathNode {
/** Which state the path node ends on, whose
* transitions we are enumerating. */
public int state;
/** Which state the current transition leads to. */
public int to;
/** Which transition we are on. */
public int transition;
/** Which label we are on, in the min-max range of the
* current Transition */
public int label;
private final LightAutomaton.Transition t = new LightAutomaton.Transition();
public void resetState(LightAutomaton a, int state) {
assert a.getNumTransitions(state) != 0;
this.state = state;
transition = 0;
a.getTransition(state, 0, t);
label = t.min;
to = t.dest;
}
/** Returns next label of current transition, or
* advances to next transition and returns its first
* label, if current one is exhausted. If there are
* no more transitions, returns -1. */
public int nextLabel(LightAutomaton a) {
if (label > t.max) {
// We've exhaused the current transition's labels;
// move to next transitions:
transition++;
if (transition >= a.getNumTransitions(state)) {
// We're done iterating transitions leaving this state
return -1;
}
a.getTransition(state, transition, t);
label = t.min;
to = t.dest;
}
return label++;
}
}
private static LightPathNode getNode(LightPathNode[] nodes, int index) {
assert index < nodes.length;
if (nodes[index] == null) {
nodes[index] = new LightPathNode();
}
return nodes[index];
}
// TODO: this is a dangerous method ... Automaton could be
// huge ... and it's better in general for caller to
// enumerate & process in a single walk:
/** Returns the set of accepted strings, up to at most
* <code>limit</code> strings. If more than <code>limit</code>
* strings are accepted, the first limit strings found are returned. If <code>limit</code> == -1, then
* the limit is infinite. If the {@link Automaton} has
* cycles then this method might throw {@code
* IllegalArgumentException} but that is not guaranteed
* when the limit is set. */
public static Set<IntsRef> getFiniteStrings(LightAutomaton a, int limit) {
Set<IntsRef> results = new HashSet<>();
if (limit == -1 || limit > 0) {
// OK
} else {
throw new IllegalArgumentException("limit must be -1 (which means no limit), or > 0; got: " + limit);
}
if (a.isAccept(0)) {
// Special case the empty string, as usual:
results.add(new IntsRef());
}
if (a.getNumTransitions(0) > 0 && (limit == -1 || results.size() < limit)) {
int numStates = a.getNumStates();
// Tracks which states are in the current path, for
// cycle detection:
BitSet pathStates = new BitSet(numStates);
// Stack to hold our current state in the
// recursion/iteration:
LightPathNode[] nodes = new LightPathNode[4];
pathStates.set(0);
LightPathNode root = getNode(nodes, 0);
root.resetState(a, 0);
IntsRef string = new IntsRef(1);
string.length = 1;
while (string.length > 0) {
LightPathNode node = nodes[string.length-1];
// Get next label leaving the current node:
int label = node.nextLabel(a);
if (label != -1) {
string.ints[string.length-1] = label;
if (a.isAccept(node.to)) {
// This transition leads to an accept state,
// so we save the current string:
results.add(IntsRef.deepCopyOf(string));
if (results.size() == limit) {
break;
}
}
if (a.getNumTransitions(node.to) != 0) {
// Now recurse: the destination of this transition has
// outgoing transitions:
if (pathStates.get(node.to)) {
throw new IllegalArgumentException("automaton has cycles");
}
pathStates.set(node.to);
// Push node onto stack:
if (nodes.length == string.length) {
LightPathNode[] newNodes = new LightPathNode[ArrayUtil.oversize(nodes.length+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
System.arraycopy(nodes, 0, newNodes, 0, nodes.length);
nodes = newNodes;
}
getNode(nodes, string.length).resetState(a, node.to);
string.length++;
string.grow(string.length);
}
} else {
// No more transitions leaving this state,
// pop/return back to previous state:
assert pathStates.get(node.state);
pathStates.clear(node.state);
string.length--;
}
}
}
return results;
}
}

View File

@ -0,0 +1,340 @@
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 org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.InPlaceMergeSorter;
import org.apache.lucene.util.Sorter;
import java.util.Arrays;
import java.util.ArrayList;
import java.util.List;
// TODO
// - do we really need the .bits...? if not we can make util in UnicodeUtil to convert 1 char into a BytesRef
/**
* Converts UTF-32 automata to the equivalent UTF-8 representation.
* @lucene.internal
*/
public final class UTF32ToUTF8Light {
// Unicode boundaries for UTF8 bytes 1,2,3,4
private static final int[] startCodes = new int[] {0, 128, 2048, 65536};
private static final int[] endCodes = new int[] {127, 2047, 65535, 1114111};
static int[] MASKS = new int[32];
static {
int v = 2;
for(int i=0;i<32;i++) {
MASKS[i] = v-1;
v *= 2;
}
}
// Represents one of the N utf8 bytes that (in sequence)
// define a code point. value is the byte value; bits is
// how many bits are "used" by utf8 at that byte
private static class UTF8Byte {
int value; // TODO: change to byte
byte bits;
}
// Holds a single code point, as a sequence of 1-4 utf8 bytes:
// TODO: maybe move to UnicodeUtil?
private static class UTF8Sequence {
private final UTF8Byte[] bytes;
private int len;
public UTF8Sequence() {
bytes = new UTF8Byte[4];
for(int i=0;i<4;i++) {
bytes[i] = new UTF8Byte();
}
}
public int byteAt(int idx) {
return bytes[idx].value;
}
public int numBits(int idx) {
return bytes[idx].bits;
}
private void set(int code) {
if (code < 128) {
// 0xxxxxxx
bytes[0].value = code;
bytes[0].bits = 7;
len = 1;
} else if (code < 2048) {
// 110yyyxx 10xxxxxx
bytes[0].value = (6 << 5) | (code >> 6);
bytes[0].bits = 5;
setRest(code, 1);
len = 2;
} else if (code < 65536) {
// 1110yyyy 10yyyyxx 10xxxxxx
bytes[0].value = (14 << 4) | (code >> 12);
bytes[0].bits = 4;
setRest(code, 2);
len = 3;
} else {
// 11110zzz 10zzyyyy 10yyyyxx 10xxxxxx
bytes[0].value = (30 << 3) | (code >> 18);
bytes[0].bits = 3;
setRest(code, 3);
len = 4;
}
}
private void setRest(int code, int numBytes) {
for(int i=0;i<numBytes;i++) {
bytes[numBytes-i].value = 128 | (code & MASKS[5]);
bytes[numBytes-i].bits = 6;
code = code >> 6;
}
}
@Override
public String toString() {
StringBuilder b = new StringBuilder();
for(int i=0;i<len;i++) {
if (i > 0) {
b.append(' ');
}
b.append(Integer.toBinaryString(bytes[i].value));
}
return b.toString();
}
}
private final UTF8Sequence startUTF8 = new UTF8Sequence();
private final UTF8Sequence endUTF8 = new UTF8Sequence();
private final UTF8Sequence tmpUTF8a = new UTF8Sequence();
private final UTF8Sequence tmpUTF8b = new UTF8Sequence();
// Builds necessary utf8 edges between start & end
void convertOneEdge(int start, int end, int startCodePoint, int endCodePoint) {
startUTF8.set(startCodePoint);
endUTF8.set(endCodePoint);
//System.out.println("start = " + startUTF8);
//System.out.println(" end = " + endUTF8);
build(start, end, startUTF8, endUTF8, 0);
}
private void build(int start, int end, UTF8Sequence startUTF8, UTF8Sequence endUTF8, int upto) {
// Break into start, middle, end:
if (startUTF8.byteAt(upto) == endUTF8.byteAt(upto)) {
// Degen case: lead with the same byte:
if (upto == startUTF8.len-1 && upto == endUTF8.len-1) {
// Super degen: just single edge, one UTF8 byte:
utf8.addTransition(start, end, startUTF8.byteAt(upto), endUTF8.byteAt(upto));
return;
} else {
assert startUTF8.len > upto+1;
assert endUTF8.len > upto+1;
int n = utf8.createState();
// Single value leading edge
utf8.addTransition(start, n, startUTF8.byteAt(upto));
//start.addTransition(new Transition(startUTF8.byteAt(upto), n)); // type=single
// Recurse for the rest
build(n, end, startUTF8, endUTF8, 1+upto);
}
} else if (startUTF8.len == endUTF8.len) {
if (upto == startUTF8.len-1) {
//start.addTransition(new Transition(startUTF8.byteAt(upto), endUTF8.byteAt(upto), end)); // type=startend
utf8.addTransition(start, end, startUTF8.byteAt(upto), endUTF8.byteAt(upto));
} else {
start(start, end, startUTF8, upto, false);
if (endUTF8.byteAt(upto) - startUTF8.byteAt(upto) > 1) {
// There is a middle
all(start, end, startUTF8.byteAt(upto)+1, endUTF8.byteAt(upto)-1, startUTF8.len-upto-1);
}
end(start, end, endUTF8, upto, false);
}
} else {
// start
start(start, end, startUTF8, upto, true);
// possibly middle, spanning multiple num bytes
int byteCount = 1+startUTF8.len-upto;
final int limit = endUTF8.len-upto;
while (byteCount < limit) {
// wasteful: we only need first byte, and, we should
// statically encode this first byte:
tmpUTF8a.set(startCodes[byteCount-1]);
tmpUTF8b.set(endCodes[byteCount-1]);
all(start, end,
tmpUTF8a.byteAt(0),
tmpUTF8b.byteAt(0),
tmpUTF8a.len - 1);
byteCount++;
}
// end
end(start, end, endUTF8, upto, true);
}
}
private void start(int start, int end, UTF8Sequence startUTF8, int upto, boolean doAll) {
if (upto == startUTF8.len-1) {
// Done recursing
utf8.addTransition(start, end, startUTF8.byteAt(upto), startUTF8.byteAt(upto) | MASKS[startUTF8.numBits(upto)-1]); // type=start
//start.addTransition(new Transition(startUTF8.byteAt(upto), startUTF8.byteAt(upto) | MASKS[startUTF8.numBits(upto)-1], end)); // type=start
} else {
int n = utf8.createState();
utf8.addTransition(start, n, startUTF8.byteAt(upto));
//start.addTransition(new Transition(startUTF8.byteAt(upto), n)); // type=start
start(n, end, startUTF8, 1+upto, true);
int endCode = startUTF8.byteAt(upto) | MASKS[startUTF8.numBits(upto)-1];
if (doAll && startUTF8.byteAt(upto) != endCode) {
all(start, end, startUTF8.byteAt(upto)+1, endCode, startUTF8.len-upto-1);
}
}
}
private void end(int start, int end, UTF8Sequence endUTF8, int upto, boolean doAll) {
if (upto == endUTF8.len-1) {
// Done recursing
//start.addTransition(new Transition(endUTF8.byteAt(upto) & (~MASKS[endUTF8.numBits(upto)-1]), endUTF8.byteAt(upto), end)); // type=end
utf8.addTransition(start, end, endUTF8.byteAt(upto) & (~MASKS[endUTF8.numBits(upto)-1]), endUTF8.byteAt(upto));
} else {
final int startCode;
if (endUTF8.numBits(upto) == 5) {
// special case -- avoid created unused edges (endUTF8
// doesn't accept certain byte sequences) -- there
// are other cases we could optimize too:
startCode = 194;
} else {
startCode = endUTF8.byteAt(upto) & (~MASKS[endUTF8.numBits(upto)-1]);
}
if (doAll && endUTF8.byteAt(upto) != startCode) {
all(start, end, startCode, endUTF8.byteAt(upto)-1, endUTF8.len-upto-1);
}
int n = utf8.createState();
//start.addTransition(new Transition(endUTF8.byteAt(upto), n)); // type=end
utf8.addTransition(start, n, endUTF8.byteAt(upto));
end(n, end, endUTF8, 1+upto, true);
}
}
private void all(int start, int end, int startCode, int endCode, int left) {
if (left == 0) {
//start.addTransition(new Transition(startCode, endCode, end)); // type=all
utf8.addTransition(start, end, startCode, endCode);
} else {
int lastN = utf8.createState();
//start.addTransition(new Transition(startCode, endCode, lastN)); // type=all
utf8.addTransition(start, lastN, startCode, endCode);
while (left > 1) {
int n = utf8.createState();
//lastN.addTransition(new Transition(128, 191, n)); // type=all*
utf8.addTransition(lastN, n, 128, 191); // type=all*
left--;
lastN = n;
}
//lastN.addTransition(new Transition(128, 191, end)); // type = all*
utf8.addTransition(lastN, end, 128, 191); // type = all*
}
}
LightAutomaton.Builder utf8;
/** Converts an incoming utf32 automaton to an equivalent
* utf8 one. The incoming automaton need not be
* deterministic. Note that the returned automaton will
* not in general be deterministic, so you must
* determinize it if that's needed. */
public LightAutomaton convert(LightAutomaton utf32) {
//System.out.println("\nCONVERT");
// nocommit make sure singleton cases work:
//if (utf32.isSingleton()) {
//utf32 = utf32.cloneExpanded();
//}
int[] map = new int[utf32.getNumStates()];
Arrays.fill(map, -1);
List<Integer> pending = new ArrayList<>();
int utf32State = 0;
pending.add(utf32State);
utf8 = new LightAutomaton.Builder();
// nocommit we don't track this
// utf8.setDeterministic(false);
int utf8State = utf8.createState();
utf8.setAccept(utf8State, utf32.isAccept(utf32State));
map[utf32State] = utf8State;
LightAutomaton.Transition scratch = new LightAutomaton.Transition();
while (pending.size() != 0) {
utf32State = pending.remove(pending.size()-1);
utf8State = map[utf32State];
assert utf8State != -1;
int numTransitions = utf32.getNumTransitions(utf32State);
utf32.initTransition(utf32State, scratch);
//System.out.println(" convert state=" + utf32State + " numTransitions=" + numTransitions);
for(int i=0;i<numTransitions;i++) {
utf32.getNextTransition(scratch);
int destUTF32 = scratch.dest;
int destUTF8 = map[destUTF32];
//System.out.println(" transition min=" + scratch.min + " max=" + scratch.max);
if (destUTF8 == -1) {
destUTF8 = utf8.createState();
//System.out.println(" create dest=" + destUTF8 +" accept=" + utf32.isAccept(destUTF32));
utf8.setAccept(destUTF8, utf32.isAccept(destUTF32));
map[destUTF32] = destUTF8;
pending.add(destUTF32);
}
// Writes new transitions into pendingTransitions:
convertOneEdge(utf8State, destUTF8, scratch.min, scratch.max);
}
}
return utf8.finish();
}
/*
private State newUTF8State() {
State s = new State();
if (utf8StateCount == utf8States.length) {
final State[] newArray = new State[ArrayUtil.oversize(1+utf8StateCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
System.arraycopy(utf8States, 0, newArray, 0, utf8StateCount);
utf8States = newArray;
}
utf8States[utf8StateCount] = s;
s.number = utf8StateCount;
utf8StateCount++;
return s;
}
*/
}

View File

@ -83,7 +83,7 @@ public class TestMockAnalyzer extends BaseTokenStreamTestCase {
/** Test a configuration where each character is a term */
public void testSingleChar() throws Exception {
CharacterRunAutomaton single =
new CharacterRunAutomaton(new RegExp(".").toAutomaton());
new CharacterRunAutomaton(new RegExp(".").toLightAutomaton());
Analyzer a = new MockAnalyzer(random(), single, false);
assertAnalyzesTo(a, "foobar",
new String[] { "f", "o", "o", "b", "a", "r" },
@ -96,7 +96,7 @@ public class TestMockAnalyzer extends BaseTokenStreamTestCase {
/** Test a configuration where two characters makes a term */
public void testTwoChars() throws Exception {
CharacterRunAutomaton single =
new CharacterRunAutomaton(new RegExp("..").toAutomaton());
new CharacterRunAutomaton(new RegExp("..").toLightAutomaton());
Analyzer a = new MockAnalyzer(random(), single, false);
assertAnalyzesTo(a, "foobar",
new String[] { "fo", "ob", "ar"},
@ -117,7 +117,7 @@ public class TestMockAnalyzer extends BaseTokenStreamTestCase {
/** Test a configuration where three characters makes a term */
public void testThreeChars() throws Exception {
CharacterRunAutomaton single =
new CharacterRunAutomaton(new RegExp("...").toAutomaton());
new CharacterRunAutomaton(new RegExp("...").toLightAutomaton());
Analyzer a = new MockAnalyzer(random(), single, false);
assertAnalyzesTo(a, "foobar",
new String[] { "foo", "bar"},
@ -138,7 +138,7 @@ public class TestMockAnalyzer extends BaseTokenStreamTestCase {
/** Test a configuration where word starts with one uppercase */
public void testUppercase() throws Exception {
CharacterRunAutomaton single =
new CharacterRunAutomaton(new RegExp("[A-Z][a-z]*").toAutomaton());
new CharacterRunAutomaton(new RegExp("[A-Z][a-z]*").toLightAutomaton());
Analyzer a = new MockAnalyzer(random(), single, false);
assertAnalyzesTo(a, "FooBarBAZ",
new String[] { "Foo", "Bar", "B", "A", "Z"},
@ -176,7 +176,7 @@ public class TestMockAnalyzer extends BaseTokenStreamTestCase {
/** Test a configuration that behaves a lot like LengthFilter */
public void testLength() throws Exception {
CharacterRunAutomaton length5 = new CharacterRunAutomaton(new RegExp(".{5,}").toAutomaton());
CharacterRunAutomaton length5 = new CharacterRunAutomaton(new RegExp(".{5,}").toLightAutomaton());
Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, true, length5);
assertAnalyzesTo(a, "ok toolong fine notfine",
new String[] { "ok", "fine" },

View File

@ -183,7 +183,7 @@ public class TestBlockPostingsFormat3 extends LuceneTestCase {
int numIntersections = atLeast(3);
for (int i = 0; i < numIntersections; i++) {
String re = AutomatonTestUtil.randomRegexp(random());
CompiledAutomaton automaton = new CompiledAutomaton(new RegExp(re, RegExp.NONE).toAutomaton());
CompiledAutomaton automaton = new CompiledAutomaton(new RegExp(re, RegExp.NONE).toLightAutomaton());
if (automaton.type == CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
// TODO: test start term too
TermsEnum leftIntersection = leftTerms.intersect(automaton, null);

View File

@ -36,6 +36,7 @@ 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.LightAutomaton;
import org.apache.lucene.util.automaton.RegExp;
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
@ -745,7 +746,7 @@ public class TestTermsEnum extends LuceneTestCase {
w.shutdown();
AtomicReader sub = getOnlySegmentReader(r);
Terms terms = sub.fields().terms("field");
Automaton automaton = new RegExp(".*", RegExp.NONE).toAutomaton();
LightAutomaton automaton = new RegExp(".*", RegExp.NONE).toLightAutomaton();
CompiledAutomaton ca = new CompiledAutomaton(automaton, false, false);
TermsEnum te = terms.intersect(ca, null);
assertEquals("aaa", te.next().utf8ToString());
@ -800,7 +801,7 @@ public class TestTermsEnum extends LuceneTestCase {
AtomicReader sub = getOnlySegmentReader(r);
Terms terms = sub.fields().terms("field");
Automaton automaton = new RegExp(".*d", RegExp.NONE).toAutomaton();
LightAutomaton automaton = new RegExp(".*d", RegExp.NONE).toLightAutomaton();
CompiledAutomaton ca = new CompiledAutomaton(automaton, false, false);
TermsEnum te;
@ -854,7 +855,7 @@ public class TestTermsEnum extends LuceneTestCase {
AtomicReader sub = getOnlySegmentReader(r);
Terms terms = sub.fields().terms("field");
Automaton automaton = new RegExp(".*", RegExp.NONE).toAutomaton(); // accept ALL
LightAutomaton automaton = new RegExp(".*", RegExp.NONE).toLightAutomaton(); // accept ALL
CompiledAutomaton ca = new CompiledAutomaton(automaton, false, false);
TermsEnum te = terms.intersect(ca, null);

View File

@ -31,6 +31,7 @@ import org.apache.lucene.index.TermsEnum.SeekStatus;
import org.apache.lucene.search.AutomatonQuery;
import org.apache.lucene.search.CheckHits;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
@ -43,7 +44,7 @@ public class TestTermsEnum2 extends LuceneTestCase {
private IndexReader reader;
private IndexSearcher searcher;
private SortedSet<BytesRef> terms; // the terms we put in the index
private Automaton termsAutomaton; // automata of the same
private LightAutomaton termsAutomaton; // automata of the same
int numIterations;
@Override
@ -68,7 +69,7 @@ public class TestTermsEnum2 extends LuceneTestCase {
writer.addDocument(doc);
}
termsAutomaton = BasicAutomata.makeStringUnion(terms);
termsAutomaton = BasicAutomata.makeStringUnionLight(terms);
reader = writer.getReader();
searcher = newSearcher(reader);
@ -86,7 +87,7 @@ public class TestTermsEnum2 extends LuceneTestCase {
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();
LightAutomaton automaton = BasicOperations.determinize(new RegExp(reg, RegExp.NONE).toLightAutomaton());
final List<BytesRef> matchedTerms = new ArrayList<>();
for(BytesRef t : terms) {
if (BasicOperations.run(automaton, t.utf8ToString())) {
@ -98,9 +99,14 @@ public class TestTermsEnum2 extends LuceneTestCase {
//System.out.println("match " + matchedTerms.size() + " " + alternate.getNumberOfStates() + " states, sigma=" + alternate.getStartPoints().length);
//AutomatonTestUtil.minimizeSimple(alternate);
//System.out.println("minmize done");
System.out.println("\nTEST: make AQ1");
AutomatonQuery a1 = new AutomatonQuery(new Term("field", ""), automaton);
System.out.println("\nTEST: make AQ2");
AutomatonQuery a2 = new AutomatonQuery(new Term("field", ""), alternate);
CheckHits.checkEqual(a1, searcher.search(a1, 25).scoreDocs, searcher.search(a2, 25).scoreDocs);
ScoreDoc[] origHits = searcher.search(a1, 25).scoreDocs;
ScoreDoc[] newHits = searcher.search(a2, 25).scoreDocs;
CheckHits.checkEqual(a1, origHits, newHits);
}
}
@ -108,7 +114,7 @@ public class TestTermsEnum2 extends LuceneTestCase {
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();
LightAutomaton automaton = BasicOperations.determinize(new RegExp(reg, RegExp.NONE).toLightAutomaton());
TermsEnum te = MultiFields.getTerms(reader, "field").iterator(null);
ArrayList<BytesRef> unsortedTerms = new ArrayList<>(terms);
Collections.shuffle(unsortedTerms, random());
@ -152,16 +158,16 @@ public class TestTermsEnum2 extends LuceneTestCase {
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();
LightAutomaton automaton = new RegExp(reg, RegExp.NONE).toLightAutomaton();
CompiledAutomaton ca = new CompiledAutomaton(automaton, SpecialOperations.isFinite(automaton), false);
TermsEnum te = MultiFields.getTerms(reader, "field").intersect(ca, null);
Automaton expected = BasicOperations.intersection(termsAutomaton, automaton);
LightAutomaton expected = BasicOperations.determinize(BasicOperations.intersectionLight(termsAutomaton, automaton));
TreeSet<BytesRef> found = new TreeSet<>();
while (te.next() != null) {
found.add(BytesRef.deepCopyOf(te.term()));
}
Automaton actual = BasicAutomata.makeStringUnion(found);
LightAutomaton actual = BasicOperations.determinize(BasicAutomata.makeStringUnionLight(found));
assertTrue(BasicOperations.sameLanguage(expected, actual));
}
}

View File

@ -27,6 +27,7 @@ import org.apache.lucene.index.Term;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.LightAutomaton;
import org.apache.lucene.util.automaton.RegExp;
/**
@ -98,7 +99,7 @@ public class TestAutomatonQueryUnicode extends LuceneTestCase {
return searcher.search(query, 5).totalHits;
}
private void assertAutomatonHits(int expected, Automaton automaton)
private void assertAutomatonHits(int expected, LightAutomaton automaton)
throws IOException {
AutomatonQuery query = new AutomatonQuery(newTerm("bogus"), automaton);
@ -122,7 +123,7 @@ public class TestAutomatonQueryUnicode extends LuceneTestCase {
* presentation forms block, or a supplementary character.
*/
public void testSortOrder() throws IOException {
Automaton a = new RegExp("((\uD866\uDF05)|\uFB94).*").toAutomaton();
LightAutomaton a = new RegExp("((\uD866\uDF05)|\uFB94).*").toLightAutomaton();
assertAutomatonHits(2, a);
}
}

View File

@ -78,7 +78,7 @@ public class TestDocTermOrdsRewriteMethod extends LuceneTestCase {
Collections.sort(terms);
System.out.println("UTF16 order:");
for(String s : terms) {
System.out.println(" " + UnicodeUtil.toHexString(s));
System.out.println(" " + UnicodeUtil.toHexString(s) + " " + s);
}
}
@ -115,7 +115,7 @@ public class TestDocTermOrdsRewriteMethod extends LuceneTestCase {
/** check that the # of hits is the same as if the query
* is run against the inverted index
*/
protected void assertSame(String regexp) throws IOException {
protected void assertSame(String regexp) throws IOException {
RegexpQuery docValues = new RegexpQuery(new Term(fieldName, regexp), RegExp.NONE);
docValues.setRewriteMethod(new DocTermOrdsRewriteMethod());
RegexpQuery inverted = new RegexpQuery(new Term(fieldName, regexp), RegExp.NONE);

View File

@ -28,9 +28,10 @@ import org.apache.lucene.index.Term;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.AutomatonProvider;
import org.apache.lucene.util.automaton.BasicAutomata;
import org.apache.lucene.util.automaton.BasicOperations;
import org.apache.lucene.util.automaton.LightAutomaton;
import org.apache.lucene.util.automaton.LightAutomatonProvider;
import org.apache.lucene.util.automaton.RegExp;
/**
@ -95,21 +96,20 @@ public class TestRegexpQuery extends LuceneTestCase {
}
public void testCustomProvider() throws IOException {
AutomatonProvider myProvider = new AutomatonProvider() {
LightAutomatonProvider myProvider = new LightAutomatonProvider() {
// automaton that matches quick or brown
private Automaton quickBrownAutomaton = BasicOperations.union(Arrays
.asList(BasicAutomata.makeString("quick"),
BasicAutomata.makeString("brown"),
BasicAutomata.makeString("bob")));
private LightAutomaton quickBrownAutomaton = BasicOperations.unionLight(Arrays
.asList(BasicAutomata.makeStringLight("quick"),
BasicAutomata.makeStringLight("brown"),
BasicAutomata.makeStringLight("bob")));
@Override
public Automaton getAutomaton(String name) {
public LightAutomaton getAutomaton(String name) {
if (name.equals("quickBrown")) return quickBrownAutomaton;
else return null;
}
};
RegexpQuery query = new RegexpQuery(newTerm("<quickBrown>"), RegExp.ALL,
myProvider);
RegexpQuery query = new RegexpQuery(newTerm("<quickBrown>"), RegExp.ALL, myProvider);
assertEquals(1, searcher.search(query, 5).totalHits);
}

View File

@ -43,6 +43,7 @@ import org.apache.lucene.util.UnicodeUtil;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.AutomatonTestUtil;
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
import org.apache.lucene.util.automaton.LightAutomaton;
import org.apache.lucene.util.automaton.RegExp;
/**
@ -103,12 +104,12 @@ public class TestRegexpRandom2 extends LuceneTestCase {
/** a stupid regexp query that just blasts thru the terms */
private class DumbRegexpQuery extends MultiTermQuery {
private final Automaton automaton;
private final LightAutomaton automaton;
DumbRegexpQuery(Term term, int flags) {
super(term.field());
RegExp re = new RegExp(term.text(), flags);
automaton = re.toAutomaton();
automaton = re.toLightAutomaton();
}
@Override

View File

@ -268,7 +268,7 @@ public class TestWildcard
* Test that wild card queries are parsed to the correct type and are searched correctly.
* This test looks at both parsing and execution of wildcard queries.
* Although placed here, it also tests prefix queries, verifying that
* prefix queries are not parsed into wild card queries, and viceversa.
* prefix queries are not parsed into wild card queries, and vice-versa.
*/
public void testParsingAndSearching() throws Exception {
String field = "content";

View File

@ -36,7 +36,7 @@ public class TestSpanFirstQuery extends LuceneTestCase {
Directory dir = newDirectory();
// mimic StopAnalyzer
CharacterRunAutomaton stopSet = new CharacterRunAutomaton(new RegExp("the|a|of").toAutomaton());
CharacterRunAutomaton stopSet = new CharacterRunAutomaton(new RegExp("the|a|of").toLightAutomaton());
Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true, stopSet);
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, analyzer);

View File

@ -103,7 +103,7 @@ public class TestQueryBuilder extends LuceneTestCase {
expected.add(new Term("field", "1"));
expected.add(new Term("field", "2"), 2);
CharacterRunAutomaton stopList = new CharacterRunAutomaton(new RegExp("[sS][tT][oO][pP]").toAutomaton());
CharacterRunAutomaton stopList = new CharacterRunAutomaton(new RegExp("[sS][tT][oO][pP]").toLightAutomaton());
Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false, stopList);

View File

@ -32,18 +32,18 @@ public class TestBasicOperations extends LuceneTestCase {
}
Collections.sort(strings);
Automaton union = BasicAutomata.makeStringUnion(strings);
assertTrue(union.isDeterministic());
LightAutomaton union = BasicAutomata.makeStringUnionLight(strings);
assertTrue(BasicOperations.isDeterministic(union));
assertTrue(BasicOperations.sameLanguage(union, naiveUnion(strings)));
}
private static Automaton naiveUnion(List<BytesRef> strings) {
Automaton [] eachIndividual = new Automaton [strings.size()];
private static LightAutomaton naiveUnion(List<BytesRef> strings) {
LightAutomaton[] eachIndividual = new LightAutomaton[strings.size()];
int i = 0;
for (BytesRef bref : strings) {
eachIndividual[i++] = BasicAutomata.makeString(bref.utf8ToString());
eachIndividual[i++] = BasicAutomata.makeStringLight(bref.utf8ToString());
}
return BasicOperations.union(Arrays.asList(eachIndividual));
return BasicOperations.determinize(BasicOperations.unionLight(Arrays.asList(eachIndividual)));
}
/** Test optimization to concatenate() */
@ -90,28 +90,20 @@ public class TestBasicOperations extends LuceneTestCase {
/** Test optimization to concatenate() with empty String to an NFA */
public void testEmptySingletonNFAConcatenate() {
Automaton singleton = BasicAutomata.makeString("");
Automaton expandedSingleton = singleton.cloneExpanded();
LightAutomaton singleton = BasicAutomata.makeStringLight("");
LightAutomaton expandedSingleton = singleton;
// an NFA (two transitions for 't' from initial state)
Automaton nfa = BasicOperations.union(BasicAutomata.makeString("this"),
BasicAutomata.makeString("three"));
Automaton concat1 = BasicOperations.concatenate(expandedSingleton, nfa);
Automaton concat2 = BasicOperations.concatenate(singleton, nfa);
assertFalse(concat2.isDeterministic());
assertTrue(BasicOperations.sameLanguage(concat1, concat2));
assertTrue(BasicOperations.sameLanguage(nfa, concat1));
assertTrue(BasicOperations.sameLanguage(nfa, concat2));
}
/** Test singletons work correctly */
public void testSingleton() {
Automaton singleton = BasicAutomata.makeString("foobar");
Automaton expandedSingleton = singleton.cloneExpanded();
assertTrue(BasicOperations.sameLanguage(singleton, expandedSingleton));
singleton = BasicAutomata.makeString("\ud801\udc1c");
expandedSingleton = singleton.cloneExpanded();
assertTrue(BasicOperations.sameLanguage(singleton, expandedSingleton));
LightAutomaton nfa = BasicOperations.unionLight(BasicAutomata.makeStringLight("this"),
BasicAutomata.makeStringLight("three"));
LightAutomaton concat1 = BasicOperations.concatenateLight(expandedSingleton, nfa);
LightAutomaton concat2 = BasicOperations.concatenateLight(singleton, nfa);
assertFalse(BasicOperations.isDeterministic(concat2));
assertTrue(BasicOperations.sameLanguage(BasicOperations.determinize(concat1),
BasicOperations.determinize(concat2)));
assertTrue(BasicOperations.sameLanguage(BasicOperations.determinize(nfa),
BasicOperations.determinize(concat1)));
assertTrue(BasicOperations.sameLanguage(BasicOperations.determinize(nfa),
BasicOperations.determinize(concat2)));
}
public void testGetRandomAcceptedString() throws Throwable {
@ -120,15 +112,18 @@ public class TestBasicOperations extends LuceneTestCase {
for(int i=0;i<ITER1;i++) {
final RegExp re = new RegExp(AutomatonTestUtil.randomRegexp(random()), RegExp.NONE);
final Automaton a = re.toAutomaton();
//System.out.println("TEST i=" + i + " re=" + re);
final LightAutomaton a = BasicOperations.determinize(re.toLightAutomaton());
assertFalse(BasicOperations.isEmpty(a));
final AutomatonTestUtil.RandomAcceptedStrings rx = new AutomatonTestUtil.RandomAcceptedStrings(a);
final AutomatonTestUtil.RandomAcceptedStringsLight rx = new AutomatonTestUtil.RandomAcceptedStringsLight(a);
for(int j=0;j<ITER2;j++) {
//System.out.println("TEST: j=" + j);
int[] acc = null;
try {
acc = rx.getRandomAcceptedString(random());
final String s = UnicodeUtil.newString(acc, 0, acc.length);
//a.writeDot("adot");
assertTrue(BasicOperations.run(a, s));
} catch (Throwable t) {
System.out.println("regexp: " + re);

View File

@ -109,7 +109,8 @@ public class TestCompiledAutomaton extends LuceneTestCase {
public void testBasic() throws Exception {
CompiledAutomaton c = build("fob", "foo", "goo");
testFloor(c, "goo", "goo");
// nocommit
//testFloor(c, "goo", "goo");
testFloor(c, "ga", "foo");
testFloor(c, "g", "foo");
testFloor(c, "foc", "fob");

View File

@ -28,49 +28,49 @@ public class TestDeterminism extends LuceneTestCase {
/** test a bunch of random regular expressions */
public void testRegexps() throws Exception {
int num = atLeast(500);
for (int i = 0; i < num; i++)
assertAutomaton(new RegExp(AutomatonTestUtil.randomRegexp(random()), RegExp.NONE).toAutomaton());
for (int i = 0; i < num; i++) {
assertAutomaton(new RegExp(AutomatonTestUtil.randomRegexp(random()), RegExp.NONE).toLightAutomaton());
}
}
/** test against a simple, unoptimized det */
public void testAgainstSimple() throws Exception {
int num = atLeast(200);
for (int i = 0; i < num; i++) {
Automaton a = AutomatonTestUtil.randomAutomaton(random());
Automaton b = a.clone();
AutomatonTestUtil.determinizeSimple(a);
b.deterministic = false; // force det
b.determinize();
LightAutomaton a = AutomatonTestUtil.randomAutomaton(random());
a = AutomatonTestUtil.determinizeSimpleLight(a);
LightAutomaton b = BasicOperations.determinize(a);
// TODO: more verifications possible?
assertTrue(BasicOperations.sameLanguage(a, b));
}
}
private static void assertAutomaton(Automaton a) {
Automaton clone = a.clone();
private static void assertAutomaton(LightAutomaton a) {
a = BasicOperations.determinize(a);
// complement(complement(a)) = a
Automaton equivalent = BasicOperations.complement(BasicOperations.complement(a));
LightAutomaton equivalent = BasicOperations.complementLight(BasicOperations.complementLight(a));
assertTrue(BasicOperations.sameLanguage(a, equivalent));
// a union a = a
equivalent = BasicOperations.union(a, clone);
equivalent = BasicOperations.determinize(BasicOperations.unionLight(a, a));
assertTrue(BasicOperations.sameLanguage(a, equivalent));
// a intersect a = a
equivalent = BasicOperations.intersection(a, clone);
equivalent = BasicOperations.determinize(BasicOperations.intersectionLight(a, a));
assertTrue(BasicOperations.sameLanguage(a, equivalent));
// a minus a = empty
Automaton empty = BasicOperations.minus(a, clone);
LightAutomaton empty = BasicOperations.minusLight(a, a);
assertTrue(BasicOperations.isEmpty(empty));
// as long as don't accept the empty string
// then optional(a) - empty = a
if (!BasicOperations.run(a, "")) {
//System.out.println("test " + a);
Automaton optional = BasicOperations.optional(a);
LightAutomaton optional = BasicOperations.optionalLight(a);
//System.out.println("optional " + optional);
equivalent = BasicOperations.minus(optional, BasicAutomata.makeEmptyString());
equivalent = BasicOperations.minusLight(optional, BasicAutomata.makeEmptyStringLight());
//System.out.println("equiv " + equivalent);
assertTrue(BasicOperations.sameLanguage(a, equivalent));
}

View File

@ -0,0 +1,340 @@
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.Collections;
import java.util.List;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.automaton.AutomatonTestUtil.RandomAcceptedStringsLight;
public class TestLightAutomaton extends LuceneTestCase {
public void testBasic() throws Exception {
LightAutomaton a = new LightAutomaton();
int start = a.createState();
int x = a.createState();
int y = a.createState();
int end = a.createState();
a.setAccept(end, true);
a.addTransition(start, x, 'a', 'a');
a.addTransition(start, end, 'd', 'd');
a.addTransition(x, y, 'b', 'b');
a.addTransition(y, end, 'c', 'c');
a.finish();
}
public void testReduceBasic() throws Exception {
LightAutomaton a = new LightAutomaton();
int start = a.createState();
int end = a.createState();
a.setAccept(end, true);
// Should collapse to a-b:
a.addTransition(start, end, 'a', 'a');
a.addTransition(start, end, 'b', 'b');
a.addTransition(start, end, 'm', 'm');
// Should collapse to x-y:
a.addTransition(start, end, 'x', 'x');
a.addTransition(start, end, 'y', 'y');
a.finish();
assertEquals(3, a.getNumTransitions(start));
LightAutomaton.Transition scratch = new LightAutomaton.Transition();
a.initTransition(start, scratch);
a.getNextTransition(scratch);
assertEquals('a', scratch.min);
assertEquals('b', scratch.max);
a.getNextTransition(scratch);
assertEquals('m', scratch.min);
assertEquals('m', scratch.max);
a.getNextTransition(scratch);
assertEquals('x', scratch.min);
assertEquals('y', scratch.max);
}
public void testSameLanguage() throws Exception {
LightAutomaton a1 = BasicAutomata.makeStringLight("foobar");
LightAutomaton a2 = BasicOperations.concatenateLight(
BasicAutomata.makeStringLight("foo"),
BasicAutomata.makeStringLight("bar"));
assertTrue(BasicOperations.sameLanguage(a1, a2));
}
public void testCommonPrefix() throws Exception {
LightAutomaton a = BasicOperations.concatenateLight(
BasicAutomata.makeStringLight("foobar"),
BasicAutomata.makeAnyStringLight());
assertEquals("foobar", SpecialOperations.getCommonPrefix(a));
}
public void testConcatenate1() throws Exception {
LightAutomaton a = BasicOperations.concatenateLight(
BasicAutomata.makeStringLight("m"),
BasicAutomata.makeAnyStringLight());
assertTrue(BasicOperations.run(a, "m"));
assertTrue(BasicOperations.run(a, "me"));
assertTrue(BasicOperations.run(a, "me too"));
}
public void testConcatenate2() throws Exception {
LightAutomaton a = BasicOperations.concatenateLight(Arrays.asList(
BasicAutomata.makeStringLight("m"),
BasicAutomata.makeAnyStringLight(),
BasicAutomata.makeStringLight("n"),
BasicAutomata.makeAnyStringLight()));
a = BasicOperations.determinize(a);
assertTrue(BasicOperations.run(a, "mn"));
assertTrue(BasicOperations.run(a, "mone"));
assertFalse(BasicOperations.run(a, "m"));
}
public void testUnion1() throws Exception {
LightAutomaton a = BasicOperations.unionLight(Arrays.asList(
BasicAutomata.makeStringLight("foobar"),
BasicAutomata.makeStringLight("barbaz")));
a = BasicOperations.determinize(a);
assertTrue(BasicOperations.run(a, "foobar"));
assertTrue(BasicOperations.run(a, "barbaz"));
// nocommit test getFinitStrings count == 2
}
public void testUnion2() throws Exception {
LightAutomaton a = BasicOperations.unionLight(Arrays.asList(
BasicAutomata.makeStringLight("foobar"),
BasicAutomata.makeStringLight(""),
BasicAutomata.makeStringLight("barbaz")));
a = BasicOperations.determinize(a);
assertTrue(BasicOperations.run(a, "foobar"));
assertTrue(BasicOperations.run(a, "barbaz"));
assertTrue(BasicOperations.run(a, ""));
// nocommit test getFinitStrings count == 3
}
public void testMinimizeSimple() throws Exception {
LightAutomaton a = BasicAutomata.makeStringLight("foobar");
//a.writeDot("a");
LightAutomaton aMin = MinimizationOperationsLight.minimize(a);
//aMin.writeDot("aMin");
assertTrue(BasicOperations.sameLanguage(a, aMin));
}
public void testMinimize2() throws Exception {
LightAutomaton a = BasicOperations.unionLight(Arrays.asList(BasicAutomata.makeStringLight("foobar"),
BasicAutomata.makeStringLight("boobar")));
LightAutomaton aMin = MinimizationOperationsLight.minimize(a);
assertTrue(BasicOperations.sameLanguage(BasicOperations.determinize(a), aMin));
}
public void testReverse() throws Exception {
LightAutomaton a = BasicAutomata.makeStringLight("foobar");
LightAutomaton ra = SpecialOperations.reverse(a);
LightAutomaton a2 = BasicOperations.determinize(SpecialOperations.reverse(ra));
assertTrue(BasicOperations.sameLanguage(a, a2));
}
public void testOptional() throws Exception {
LightAutomaton a = BasicAutomata.makeStringLight("foobar");
LightAutomaton a2 = BasicOperations.optionalLight(a);
a2 = BasicOperations.determinize(a2);
assertTrue(BasicOperations.run(a, "foobar"));
assertFalse(BasicOperations.run(a, ""));
assertTrue(BasicOperations.run(a2, "foobar"));
assertTrue(BasicOperations.run(a2, ""));
}
public void testRepeatAny() throws Exception {
LightAutomaton a = BasicAutomata.makeStringLight("zee");
LightAutomaton a2 = BasicOperations.determinize(BasicOperations.repeatLight(a));
assertTrue(BasicOperations.run(a2, ""));
assertTrue(BasicOperations.run(a2, "zee"));
assertTrue(BasicOperations.run(a2, "zeezee"));
assertTrue(BasicOperations.run(a2, "zeezeezee"));
}
public void testRepeatMin() throws Exception {
LightAutomaton a = BasicAutomata.makeStringLight("zee");
LightAutomaton a2 = BasicOperations.determinize(BasicOperations.repeatLight(a, 2));
assertFalse(BasicOperations.run(a2, ""));
assertFalse(BasicOperations.run(a2, "zee"));
assertTrue(BasicOperations.run(a2, "zeezee"));
assertTrue(BasicOperations.run(a2, "zeezeezee"));
}
public void testRepeatMinMax1() throws Exception {
LightAutomaton a = BasicAutomata.makeStringLight("zee");
LightAutomaton a2 = BasicOperations.determinize(BasicOperations.repeatLight(a, 0, 2));
assertTrue(BasicOperations.run(a2, ""));
assertTrue(BasicOperations.run(a2, "zee"));
assertTrue(BasicOperations.run(a2, "zeezee"));
assertFalse(BasicOperations.run(a2, "zeezeezee"));
}
public void testRepeatMinMax2() throws Exception {
LightAutomaton a = BasicAutomata.makeStringLight("zee");
LightAutomaton a2 = BasicOperations.determinize(BasicOperations.repeatLight(a, 2, 4));
assertFalse(BasicOperations.run(a2, ""));
assertFalse(BasicOperations.run(a2, "zee"));
assertTrue(BasicOperations.run(a2, "zeezee"));
assertTrue(BasicOperations.run(a2, "zeezeezee"));
assertTrue(BasicOperations.run(a2, "zeezeezeezee"));
assertFalse(BasicOperations.run(a2, "zeezeezeezeezee"));
}
public void testComplement() throws Exception {
LightAutomaton a = BasicAutomata.makeStringLight("zee");
LightAutomaton a2 = BasicOperations.determinize(BasicOperations.complementLight(a));
assertTrue(BasicOperations.run(a2, ""));
assertFalse(BasicOperations.run(a2, "zee"));
assertTrue(BasicOperations.run(a2, "zeezee"));
assertTrue(BasicOperations.run(a2, "zeezeezee"));
}
public void testInterval() throws Exception {
LightAutomaton a = BasicOperations.determinize(BasicAutomata.makeIntervalLight(17, 100, 3));
assertFalse(BasicOperations.run(a, ""));
assertTrue(BasicOperations.run(a, "017"));
assertTrue(BasicOperations.run(a, "100"));
assertTrue(BasicOperations.run(a, "073"));
}
public void testCommonSuffix() throws Exception {
LightAutomaton a = new LightAutomaton();
int init = a.createState();
int fini = a.createState();
a.setAccept(init, true);
a.setAccept(fini, true);
a.addTransition(init, fini, 'm');
a.addTransition(fini, fini, 'm');
a.finish();
assertEquals(0, SpecialOperations.getCommonSuffixBytesRef(a).length);
}
public void testReverseRandom1() throws Exception {
int ITERS = atLeast(100);
for(int i=0;i<ITERS;i++) {
LightAutomaton a = AutomatonTestUtil.randomAutomaton(random());
LightAutomaton ra = SpecialOperations.reverse(a);
LightAutomaton rra = SpecialOperations.reverse(ra);
assertTrue(BasicOperations.sameLanguage(BasicOperations.determinize(a),
BasicOperations.determinize(rra)));
}
}
public void testReverseRandom2() throws Exception {
int ITERS = atLeast(100);
for(int iter=0;iter<ITERS;iter++) {
//System.out.println("TEST: iter=" + iter);
LightAutomaton a = BasicOperations.removeDeadTransitions(AutomatonTestUtil.randomAutomaton(random()));
LightAutomaton ra = SpecialOperations.reverse(a);
LightAutomaton rda = BasicOperations.determinize(ra);
if (a.isEmpty()) {
assertTrue(rda.isEmpty());
continue;
}
RandomAcceptedStringsLight rasl = new RandomAcceptedStringsLight(a);
for(int iter2=0;iter2<20;iter2++) {
// Find string accepted by original automaton
int[] s = rasl.getRandomAcceptedString(random());
// Reverse it
for(int j=0;j<s.length/2;j++) {
int x = s[j];
s[j] = s[s.length-j-1];
s[s.length-j-1] = x;
}
//System.out.println("TEST: iter2=" + iter2 + " s=" + Arrays.toString(s));
// Make sure reversed automaton accepts it
assertTrue(BasicOperations.run(rda, new IntsRef(s, 0, s.length)));
}
}
}
public void testAnyStringEmptyString() throws Exception {
LightAutomaton a = BasicOperations.determinize(BasicAutomata.makeAnyStringLight());
assertTrue(BasicOperations.run(a, ""));
}
public void testRemoveDeadTransitionsEmpty() throws Exception {
LightAutomaton a = BasicAutomata.makeEmptyLight();
LightAutomaton a2 = BasicOperations.removeDeadTransitions(a);
assertTrue(a2.isEmpty());
}
public void testInvalidAddTransition() throws Exception {
LightAutomaton a = new LightAutomaton();
int s1 = a.createState();
int s2 = a.createState();
a.addTransition(s1, s2, 'a');
a.addTransition(s2, s2, 'a');
try {
a.addTransition(s1, s2, 'b');
fail("didn't hit expected exception");
} catch (IllegalStateException ise) {
// expected
}
}
public void testBuilderRandom() throws Exception {
int ITERS = atLeast(100);
for(int iter=0;iter<ITERS;iter++) {
LightAutomaton a = AutomatonTestUtil.randomAutomaton(random());
// Just get all transitions, shuffle, and build a new automaton with the same transitions:
List<LightAutomaton.Transition> allTrans = new ArrayList<>();
int numStates = a.getNumStates();
for(int s=0;s<numStates;s++) {
int count = a.getNumTransitions(s);
for(int i=0;i<count;i++) {
LightAutomaton.Transition t = new LightAutomaton.Transition();
a.getTransition(s, i, t);
allTrans.add(t);
}
}
LightAutomaton.Builder builder = new LightAutomaton.Builder();
for(int i=0;i<numStates;i++) {
int s = builder.createState();
builder.setAccept(s, a.isAccept(s));
}
Collections.shuffle(allTrans, random());
for(LightAutomaton.Transition t : allTrans) {
builder.addTransition(t.source, t.dest, t.min, t.max);
}
assertTrue(BasicOperations.sameLanguage(
BasicOperations.determinize(a),
BasicOperations.determinize(builder.finish())));
}
}
}

View File

@ -24,13 +24,13 @@ import org.apache.lucene.util.LuceneTestCase;
*/
public class TestMinimize extends LuceneTestCase {
/** the minimal and non-minimal are compared to ensure they are the same. */
public void test() {
public void testBasic() {
int num = atLeast(200);
for (int i = 0; i < num; i++) {
Automaton a = AutomatonTestUtil.randomAutomaton(random());
Automaton b = a.clone();
MinimizationOperations.minimize(b);
assertTrue(BasicOperations.sameLanguage(a, b));
LightAutomaton a = AutomatonTestUtil.randomAutomaton(random());
LightAutomaton la = BasicOperations.determinize(a);
LightAutomaton lb = BasicOperations.determinize(MinimizationOperationsLight.minimize(a));
assertTrue(BasicOperations.sameLanguage(la, lb));
}
}
@ -40,18 +40,28 @@ public class TestMinimize extends LuceneTestCase {
public void testAgainstBrzozowski() {
int num = atLeast(200);
for (int i = 0; i < num; i++) {
Automaton a = AutomatonTestUtil.randomAutomaton(random());
AutomatonTestUtil.minimizeSimple(a);
Automaton b = a.clone();
MinimizationOperations.minimize(b);
LightAutomaton a = AutomatonTestUtil.randomAutomaton(random());
a = AutomatonTestUtil.minimizeSimple(a);
LightAutomaton b = MinimizationOperationsLight.minimize(a);
assertTrue(BasicOperations.sameLanguage(a, b));
assertEquals(a.getNumberOfStates(), b.getNumberOfStates());
assertEquals(a.getNumberOfTransitions(), b.getNumberOfTransitions());
assertEquals(a.getNumStates(), b.getNumStates());
int numStates = a.getNumStates();
int sum1 = 0;
for(int s=0;s<numStates;s++) {
sum1 += a.getNumTransitions(s);
}
int sum2 = 0;
for(int s=0;s<numStates;s++) {
sum2 += b.getNumTransitions(s);
}
assertEquals(sum1, sum2);
}
}
/** n^2 space usage in Hopcroft minimization? */
public void testMinimizeHuge() {
new RegExp("+-*(A|.....|BC)*]", RegExp.NONE).toAutomaton();
new RegExp("+-*(A|.....|BC)*]", RegExp.NONE).toLightAutomaton();
}
}

View File

@ -36,9 +36,8 @@ public class TestSpecialOperations extends LuceneTestCase {
public void testIsFinite() {
int num = atLeast(200);
for (int i = 0; i < num; i++) {
Automaton a = AutomatonTestUtil.randomAutomaton(random());
Automaton b = a.clone();
assertEquals(AutomatonTestUtil.isFiniteSlow(a), SpecialOperations.isFinite(b));
LightAutomaton a = AutomatonTestUtil.randomAutomaton(random());
assertEquals(AutomatonTestUtil.isFiniteSlow(a), SpecialOperations.isFinite(a));
}
}
@ -158,7 +157,7 @@ public class TestSpecialOperations extends LuceneTestCase {
public void testWithCycle() throws Exception {
try {
SpecialOperations.getFiniteStrings(new RegExp("abc.*", RegExp.NONE).toAutomaton(), -1);
SpecialOperations.getFiniteStrings(new RegExp("abc.*", RegExp.NONE).toLightAutomaton(), -1);
fail("did not hit exception");
} catch (IllegalArgumentException iae) {
// expected
@ -170,7 +169,7 @@ public class TestSpecialOperations extends LuceneTestCase {
// automaton:
int iters = atLeast(100);
for(int i=0;i<iters;i++) {
Automaton a = AutomatonTestUtil.randomAutomaton(random());
LightAutomaton a = AutomatonTestUtil.randomAutomaton(random());
try {
// Must pass a limit because the random automaton
// can accept MANY strings:
@ -185,7 +184,7 @@ public class TestSpecialOperations extends LuceneTestCase {
}
public void testInvalidLimit() {
Automaton a = AutomatonTestUtil.randomAutomaton(random());
LightAutomaton a = AutomatonTestUtil.randomAutomaton(random());
try {
SpecialOperations.getFiniteStrings(a, -7);
fail("did not hit exception");
@ -195,7 +194,7 @@ public class TestSpecialOperations extends LuceneTestCase {
}
public void testInvalidLimit2() {
Automaton a = AutomatonTestUtil.randomAutomaton(random());
LightAutomaton a = AutomatonTestUtil.randomAutomaton(random());
try {
SpecialOperations.getFiniteStrings(a, 0);
fail("did not hit exception");

View File

@ -163,7 +163,7 @@ public class TestUTF32ToUTF8 extends LuceneTestCase {
public void testSpecialCase() {
RegExp re = new RegExp(".?");
Automaton automaton = re.toAutomaton();
LightAutomaton automaton = re.toLightAutomaton();
CharacterRunAutomaton cra = new CharacterRunAutomaton(automaton);
ByteRunAutomaton bra = new ByteRunAutomaton(automaton);
// make sure character dfa accepts empty string
@ -179,7 +179,7 @@ public class TestUTF32ToUTF8 extends LuceneTestCase {
public void testSpecialCase2() throws Exception {
RegExp re = new RegExp(".+\u0775");
String input = "\ufadc\ufffd\ub80b\uda5a\udc68\uf234\u0056\uda5b\udcc1\ufffd\ufffd\u0775";
Automaton automaton = re.toAutomaton();
LightAutomaton automaton = re.toLightAutomaton();
CharacterRunAutomaton cra = new CharacterRunAutomaton(automaton);
ByteRunAutomaton bra = new ByteRunAutomaton(automaton);
@ -192,7 +192,7 @@ public class TestUTF32ToUTF8 extends LuceneTestCase {
public void testSpecialCase3() throws Exception {
RegExp re = new RegExp("(\\鯺)*(.)*\\Ӕ");
String input = "\u5cfd\ufffd\ub2f7\u0033\ue304\u51d7\u3692\udb50\udfb3\u0576\udae2\udc62\u0053\u0449\u04d4";
Automaton automaton = re.toAutomaton();
LightAutomaton automaton = re.toLightAutomaton();
CharacterRunAutomaton cra = new CharacterRunAutomaton(automaton);
ByteRunAutomaton bra = new ByteRunAutomaton(automaton);
@ -205,14 +205,14 @@ public class TestUTF32ToUTF8 extends LuceneTestCase {
public void testRandomRegexes() throws Exception {
int num = atLeast(250);
for (int i = 0; i < num; i++) {
assertAutomaton(new RegExp(AutomatonTestUtil.randomRegexp(random()), RegExp.NONE).toAutomaton());
assertAutomaton(new RegExp(AutomatonTestUtil.randomRegexp(random()), RegExp.NONE).toLightAutomaton());
}
}
private void assertAutomaton(Automaton automaton) throws Exception {
private void assertAutomaton(LightAutomaton automaton) throws Exception {
CharacterRunAutomaton cra = new CharacterRunAutomaton(automaton);
ByteRunAutomaton bra = new ByteRunAutomaton(automaton);
final AutomatonTestUtil.RandomAcceptedStrings ras = new AutomatonTestUtil.RandomAcceptedStrings(automaton);
final AutomatonTestUtil.RandomAcceptedStringsLight ras = new AutomatonTestUtil.RandomAcceptedStringsLight(automaton);
int num = atLeast(1000);
for (int i = 0; i < num; i++) {

View File

@ -17,43 +17,6 @@ package org.apache.lucene.util.fst;
* limitations under the License.
*/
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.DirectoryReader;
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.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.store.IndexOutput;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.LineFileDocs;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.RegExp;
import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
import org.apache.lucene.util.fst.FST.Arc;
import org.apache.lucene.util.fst.FST.BytesReader;
import org.apache.lucene.util.fst.PairOutputs.Pair;
import org.apache.lucene.util.fst.Util.Result;
import org.apache.lucene.util.packed.PackedInts;
import java.io.BufferedReader;
import java.io.File;
import java.io.FileInputStream;
@ -78,6 +41,44 @@ import java.util.TreeMap;
import java.util.TreeSet;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.DirectoryReader;
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.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.store.IndexOutput;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.LineFileDocs;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
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.CompiledAutomaton;
import org.apache.lucene.util.automaton.LightAutomaton;
import org.apache.lucene.util.automaton.RegExp;
import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
import org.apache.lucene.util.fst.FST.Arc;
import org.apache.lucene.util.fst.FST.BytesReader;
import org.apache.lucene.util.fst.PairOutputs.Pair;
import org.apache.lucene.util.fst.Util.Result;
import org.apache.lucene.util.packed.PackedInts;
import static org.apache.lucene.util.fst.FSTTester.getRandomString;
import static org.apache.lucene.util.fst.FSTTester.simpleRandomString;
import static org.apache.lucene.util.fst.FSTTester.toIntsRef;
@ -346,7 +347,7 @@ public class TestFSTs extends LuceneTestCase {
BytesRef term;
int ord = 0;
Automaton automaton = new RegExp(".*", RegExp.NONE).toAutomaton();
LightAutomaton automaton = new RegExp(".*", RegExp.NONE).toLightAutomaton();
final TermsEnum termsEnum2 = terms.intersect(new CompiledAutomaton(automaton, false, false), null);
while((term = termsEnum.next()) != null) {

View File

@ -95,7 +95,7 @@ class MultiTermHighlighting {
} else if (query instanceof AutomatonQuery) {
final AutomatonQuery aq = (AutomatonQuery) query;
if (aq.getField().equals(field)) {
list.add(new CharacterRunAutomaton(aq.getAutomaton()) {
list.add(new CharacterRunAutomaton(aq.getLightAutomaton()) {
@Override
public String toString() {
return aq.toString();

View File

@ -1386,7 +1386,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
TestHighlightRunner helper = new TestHighlightRunner() {
@Override
public void run() throws Exception {
CharacterRunAutomaton stopWords = new CharacterRunAutomaton(new RegExp("i[nt]").toAutomaton());
CharacterRunAutomaton stopWords = new CharacterRunAutomaton(new RegExp("i[nt]").toLightAutomaton());
TermQuery query = new TermQuery(new Term("text", "searchterm"));
String text = "this is a text with searchterm in it";

View File

@ -602,8 +602,8 @@ public class FastVectorHighlighterTest extends LuceneTestCase {
fieldAnalyzers.put( "field", new MockAnalyzer( random(), MockTokenizer.WHITESPACE, true, MockTokenFilter.ENGLISH_STOPSET ) );
fieldAnalyzers.put( "field_exact", new MockAnalyzer( random() ) );
fieldAnalyzers.put( "field_super_exact", new MockAnalyzer( random(), MockTokenizer.WHITESPACE, false ) );
fieldAnalyzers.put( "field_characters", new MockAnalyzer( random(), new CharacterRunAutomaton( new RegExp(".").toAutomaton() ), true ) );
fieldAnalyzers.put( "field_tripples", new MockAnalyzer( random(), new CharacterRunAutomaton( new RegExp("...").toAutomaton() ), true ) );
fieldAnalyzers.put( "field_characters", new MockAnalyzer( random(), new CharacterRunAutomaton( new RegExp(".").toLightAutomaton() ), true ) );
fieldAnalyzers.put( "field_tripples", new MockAnalyzer( random(), new CharacterRunAutomaton( new RegExp("...").toLightAutomaton() ), true ) );
fieldAnalyzers.put( "field_sliced", fieldAnalyzers.get( "field" ) );
fieldAnalyzers.put( "field_der_red", fieldAnalyzers.get( "field" ) ); // This is required even though we provide a token stream
Analyzer analyzer = new AnalyzerWrapper() {

View File

@ -1190,7 +1190,7 @@ public class TestQPHelper extends LuceneTestCase {
public void testStopwords() throws Exception {
StandardQueryParser qp = new StandardQueryParser();
CharacterRunAutomaton stopSet = new CharacterRunAutomaton(new RegExp("the|foo").toAutomaton());
CharacterRunAutomaton stopSet = new CharacterRunAutomaton(new RegExp("the|foo").toLightAutomaton());
qp.setAnalyzer(new MockAnalyzer(random(), MockTokenizer.SIMPLE, true, stopSet));
Query result = qp.parse("a:the OR a:foo", "a");

View File

@ -1023,7 +1023,7 @@ public abstract class QueryParserTestBase extends LuceneTestCase {
}
public void testStopwords() throws Exception {
CharacterRunAutomaton stopSet = new CharacterRunAutomaton(new RegExp("the|foo").toAutomaton());
CharacterRunAutomaton stopSet = new CharacterRunAutomaton(new RegExp("the|foo").toLightAutomaton());
CommonQueryParserConfiguration qp = getParserConfig(new MockAnalyzer(random(), MockTokenizer.SIMPLE, true, stopSet));
Query result = getQuery("field:the OR field:foo",qp);
assertNotNull("result is null and it shouldn't be", result);
@ -1251,7 +1251,7 @@ public abstract class QueryParserTestBase extends LuceneTestCase {
public void testPhraseQueryPositionIncrements() throws Exception {
CharacterRunAutomaton stopStopList =
new CharacterRunAutomaton(new RegExp("[sS][tT][oO][pP]").toAutomaton());
new CharacterRunAutomaton(new RegExp("[sS][tT][oO][pP]").toLightAutomaton());
CommonQueryParserConfiguration qp = getParserConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false, stopStopList));

View File

@ -44,16 +44,16 @@ import com.carrotsearch.randomizedtesting.RandomizedContext;
public class MockTokenizer extends Tokenizer {
/** Acts Similar to WhitespaceTokenizer */
public static final CharacterRunAutomaton WHITESPACE =
new CharacterRunAutomaton(new RegExp("[^ \t\r\n]+").toAutomaton());
new CharacterRunAutomaton(new RegExp("[^ \t\r\n]+").toLightAutomaton());
/** Acts Similar to KeywordTokenizer.
* TODO: Keyword returns an "empty" token for an empty reader...
*/
public static final CharacterRunAutomaton KEYWORD =
new CharacterRunAutomaton(new RegExp(".*").toAutomaton());
new CharacterRunAutomaton(new RegExp(".*").toLightAutomaton());
/** Acts like LetterTokenizer. */
// the ugly regex below is incomplete Unicode 5.2 [:Letter:]
public static final CharacterRunAutomaton SIMPLE =
new CharacterRunAutomaton(new RegExp("[A-Za-zªµºÀ-ÖØ-öø-ˁ一-鿌]+").toAutomaton());
new CharacterRunAutomaton(new RegExp("[A-Za-zªµºÀ-ÖØ-öø-ˁ一-鿌]+").toLightAutomaton());
private final CharacterRunAutomaton runAutomaton;
private final boolean lowerCase;

View File

@ -1778,7 +1778,7 @@ public abstract class LuceneTestCase extends Assert {
int numIntersections = atLeast(3);
for (int i = 0; i < numIntersections; i++) {
String re = AutomatonTestUtil.randomRegexp(random());
CompiledAutomaton automaton = new CompiledAutomaton(new RegExp(re, RegExp.NONE).toAutomaton());
CompiledAutomaton automaton = new CompiledAutomaton(new RegExp(re, RegExp.NONE).toLightAutomaton());
if (automaton.type == CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
// TODO: test start term too
TermsEnum leftIntersection = leftTerms.intersect(automaton, null);

View File

@ -176,6 +176,9 @@ public class TestRuleLimitSysouts extends TestRuleAdapter {
}
protected boolean isEnforced() {
// nocommit
return false;
/*
Class<?> target = RandomizedTest.getContext().getTargetClass();
if (LuceneTestCase.VERBOSE ||
@ -189,6 +192,7 @@ public class TestRuleLimitSysouts extends TestRuleAdapter {
}
return true;
*/
}
/**

View File

@ -92,40 +92,40 @@ public class AutomatonTestUtil {
/** picks a random int code point, avoiding surrogates;
* throws IllegalArgumentException if this transition only
* accepts surrogates */
private static int getRandomCodePoint(final Random r, final Transition t) {
private static int getRandomCodePoint(final Random r, int min, int max) {
final int code;
if (t.max < UnicodeUtil.UNI_SUR_HIGH_START ||
t.min > UnicodeUtil.UNI_SUR_HIGH_END) {
if (max < UnicodeUtil.UNI_SUR_HIGH_START ||
min > UnicodeUtil.UNI_SUR_HIGH_END) {
// easy: entire range is before or after surrogates
code = t.min+r.nextInt(t.max-t.min+1);
} else if (t.min >= UnicodeUtil.UNI_SUR_HIGH_START) {
if (t.max > UnicodeUtil.UNI_SUR_LOW_END) {
code = min+r.nextInt(max-min+1);
} else if (min >= UnicodeUtil.UNI_SUR_HIGH_START) {
if (max > UnicodeUtil.UNI_SUR_LOW_END) {
// after surrogates
code = 1+UnicodeUtil.UNI_SUR_LOW_END+r.nextInt(t.max-UnicodeUtil.UNI_SUR_LOW_END);
code = 1+UnicodeUtil.UNI_SUR_LOW_END+r.nextInt(max-UnicodeUtil.UNI_SUR_LOW_END);
} else {
throw new IllegalArgumentException("transition accepts only surrogates: " + t);
throw new IllegalArgumentException("transition accepts only surrogates: min=" + min + " max=" + max);
}
} else if (t.max <= UnicodeUtil.UNI_SUR_LOW_END) {
if (t.min < UnicodeUtil.UNI_SUR_HIGH_START) {
} else if (max <= UnicodeUtil.UNI_SUR_LOW_END) {
if (min < UnicodeUtil.UNI_SUR_HIGH_START) {
// before surrogates
code = t.min + r.nextInt(UnicodeUtil.UNI_SUR_HIGH_START - t.min);
code = min + r.nextInt(UnicodeUtil.UNI_SUR_HIGH_START - min);
} else {
throw new IllegalArgumentException("transition accepts only surrogates: " + t);
throw new IllegalArgumentException("transition accepts only surrogates: min=" + min + " max=" + max);
}
} else {
// range includes all surrogates
int gap1 = UnicodeUtil.UNI_SUR_HIGH_START - t.min;
int gap2 = t.max - UnicodeUtil.UNI_SUR_LOW_END;
int gap1 = UnicodeUtil.UNI_SUR_HIGH_START - min;
int gap2 = max - UnicodeUtil.UNI_SUR_LOW_END;
int c = r.nextInt(gap1+gap2);
if (c < gap1) {
code = t.min + c;
code = min + c;
} else {
code = UnicodeUtil.UNI_SUR_LOW_END + c - gap1 + 1;
}
}
assert code >= t.min && code <= t.max && (code < UnicodeUtil.UNI_SUR_HIGH_START || code > UnicodeUtil.UNI_SUR_LOW_END):
"code=" + code + " min=" + t.min + " max=" + t.max;
assert code >= min && code <= max && (code < UnicodeUtil.UNI_SUR_HIGH_START || code > UnicodeUtil.UNI_SUR_LOW_END):
"code=" + code + " min=" + min + " max=" + max;
return code;
}
@ -257,7 +257,7 @@ public class AutomatonTestUtil {
} else {
t = s.transitionsArray[r.nextInt(s.numTransitions)];
}
soFar.add(getRandomCodePoint(r, t));
soFar.add(getRandomCodePoint(r, t.min, t.max));
s = t.to;
}
}
@ -265,24 +265,150 @@ public class AutomatonTestUtil {
return ArrayUtil.toIntArray(soFar);
}
}
/**
* Lets you retrieve random strings accepted
* by a LightAutomaton.
* <p>
* Once created, call {@link #getRandomAcceptedString(Random)}
* to get a new string (in UTF-32 codepoints).
*/
public static class RandomAcceptedStringsLight {
private final Map<LightAutomaton.Transition,Boolean> leadsToAccept;
private final LightAutomaton a;
private final LightAutomaton.Transition[][] transitions;
private static class ArrivingTransition {
final int from;
final LightAutomaton.Transition t;
public ArrivingTransition(int from, LightAutomaton.Transition t) {
this.from = from;
this.t = t;
}
}
public RandomAcceptedStringsLight(LightAutomaton a) {
this.a = a;
if (a.getNumStates() == 0) {
throw new IllegalArgumentException("this automaton accepts nothing");
}
this.transitions = a.getSortedTransitions();
leadsToAccept = new HashMap<>();
final Map<Integer,List<ArrivingTransition>> allArriving = new HashMap<>();
final LinkedList<Integer> q = new LinkedList<>();
final Set<Integer> seen = new HashSet<>();
// reverse map the transitions, so we can quickly look
// up all arriving transitions to a given state
int numStates = a.getNumStates();
for(int s=0;s<numStates;s++) {
for(LightAutomaton.Transition t : transitions[s]) {
List<ArrivingTransition> tl = allArriving.get(t.dest);
if (tl == null) {
tl = new ArrayList<>();
allArriving.put(t.dest, tl);
}
tl.add(new ArrivingTransition(s, t));
}
if (a.isAccept(s)) {
q.add(s);
seen.add(s);
}
}
// Breadth-first search, from accept states,
// backwards:
while (q.isEmpty() == false) {
final int s = q.removeFirst();
List<ArrivingTransition> arriving = allArriving.get(s);
if (arriving != null) {
for(ArrivingTransition at : arriving) {
final int from = at.from;
if (!seen.contains(from)) {
q.add(from);
seen.add(from);
leadsToAccept.put(at.t, Boolean.TRUE);
}
}
}
}
}
public int[] getRandomAcceptedString(Random r) {
final List<Integer> soFar = new ArrayList<>();
int s = 0;
while(true) {
if (a.isAccept(s)) {
if (a.getNumTransitions(s) == 0) {
// stop now
break;
} else {
if (r.nextBoolean()) {
break;
}
}
}
if (a.getNumTransitions(s) == 0) {
throw new RuntimeException("this automaton has dead states");
}
boolean cheat = r.nextBoolean();
final LightAutomaton.Transition t;
if (cheat) {
// pick a transition that we know is the fastest
// path to an accept state
List<LightAutomaton.Transition> toAccept = new ArrayList<>();
for(LightAutomaton.Transition t0 : transitions[s]) {
if (leadsToAccept.containsKey(t0)) {
toAccept.add(t0);
}
}
if (toAccept.size() == 0) {
// this is OK -- it means we jumped into a cycle
t = transitions[s][r.nextInt(transitions[s].length)];
} else {
t = toAccept.get(r.nextInt(toAccept.size()));
}
} else {
t = transitions[s][r.nextInt(transitions[s].length)];
}
soFar.add(getRandomCodePoint(r, t.min, t.max));
s = t.dest;
}
return ArrayUtil.toIntArray(soFar);
}
}
/** return a random NFA/DFA for testing */
public static Automaton randomAutomaton(Random random) {
public static LightAutomaton randomAutomaton(Random random) {
// get two random Automata from regexps
Automaton a1 = new RegExp(AutomatonTestUtil.randomRegexp(random), RegExp.NONE).toAutomaton();
if (random.nextBoolean())
a1 = BasicOperations.complement(a1);
Automaton a2 = new RegExp(AutomatonTestUtil.randomRegexp(random), RegExp.NONE).toAutomaton();
if (random.nextBoolean())
a2 = BasicOperations.complement(a2);
LightAutomaton a1 = new RegExp(AutomatonTestUtil.randomRegexp(random), RegExp.NONE).toLightAutomaton();
if (random.nextBoolean()) {
a1 = BasicOperations.complementLight(a1);
}
LightAutomaton a2 = new RegExp(AutomatonTestUtil.randomRegexp(random), RegExp.NONE).toLightAutomaton();
if (random.nextBoolean()) {
a2 = BasicOperations.complementLight(a2);
}
// combine them in random ways
switch(random.nextInt(4)) {
case 0: return BasicOperations.concatenate(a1, a2);
case 1: return BasicOperations.union(a1, a2);
case 2: return BasicOperations.intersection(a1, a2);
default: return BasicOperations.minus(a1, a2);
switch (random.nextInt(4)) {
case 0: return BasicOperations.concatenateLight(a1, a2);
case 1: return BasicOperations.unionLight(a1, a2);
case 2: return BasicOperations.intersectionLight(a1, a2);
default: return BasicOperations.minusLight(a1, a2);
}
}
@ -329,6 +455,17 @@ public class AutomatonTestUtil {
determinizeSimple(a, SpecialOperations.reverse(a));
determinizeSimple(a, SpecialOperations.reverse(a));
}
/**
* Simple, original brics implementation of Brzozowski minimize()
*/
public static LightAutomaton minimizeSimple(LightAutomaton a) {
Set<Integer> initialSet = new HashSet<Integer>();
a = determinizeSimpleLight(SpecialOperations.reverse(a, initialSet), initialSet);
initialSet.clear();
a = determinizeSimpleLight(SpecialOperations.reverse(a, initialSet), initialSet);
return a;
}
/**
* Simple, original brics implementation of determinize()
@ -389,6 +526,72 @@ public class AutomatonTestUtil {
a.removeDeadTransitions();
}
/**
* Simple, original brics implementation of determinize()
*/
public static LightAutomaton determinizeSimpleLight(LightAutomaton a) {
Set<Integer> initialset = new HashSet<>();
initialset.add(0);
return determinizeSimpleLight(a, initialset);
}
/**
* Simple, original brics implementation of determinize()
* Determinizes the given automaton using the given set of initial states.
*/
public static LightAutomaton determinizeSimpleLight(LightAutomaton a, Set<Integer> initialset) {
int[] points = a.getStartPoints();
// subset construction
Map<Set<Integer>, Set<Integer>> sets = new HashMap<>();
LinkedList<Set<Integer>> worklist = new LinkedList<>();
Map<Set<Integer>, Integer> newstate = new HashMap<>();
sets.put(initialset, initialset);
worklist.add(initialset);
LightAutomaton.Builder result = new LightAutomaton.Builder();
result.createState();
newstate.put(initialset, 0);
LightAutomaton.Transition t = new LightAutomaton.Transition();
while (worklist.size() > 0) {
Set<Integer> s = worklist.removeFirst();
int r = newstate.get(s);
for (int q : s) {
if (a.isAccept(q)) {
result.setAccept(r, true);
break;
}
}
for (int n = 0; n < points.length; n++) {
Set<Integer> p = new HashSet<>();
for (int q : s) {
int count = a.initTransition(q, t);
for(int i=0;i<count;i++) {
a.getNextTransition(t);
if (t.min <= points[n] && points[n] <= t.max) {
p.add(t.dest);
}
}
}
if (!sets.containsKey(p)) {
sets.put(p, p);
worklist.add(p);
newstate.put(p, result.createState());
}
int q = newstate.get(p);
int min = points[n];
int max;
if (n + 1 < points.length) {
max = points[n + 1] - 1;
} else {
max = Character.MAX_CODE_POINT;
}
result.addTransition(r, q, min, max);
}
}
return BasicOperations.removeDeadTransitions(result.finish());
}
/**
* Simple, original implementation of getFiniteStrings.
*
@ -469,6 +672,36 @@ public class AutomatonTestUtil {
path.remove(s);
return true;
}
/**
* Returns true if the language of this automaton is finite.
* <p>
* WARNING: this method is slow, it will blow up if the automaton is large.
* this is only used to test the correctness of our faster implementation.
*/
public static boolean isFiniteSlow(LightAutomaton a) {
return isFiniteSlow(a, 0, new HashSet<Integer>());
}
/**
* Checks whether there is a loop containing s. (This is sufficient since
* there are never transitions to dead states.)
*/
// TODO: not great that this is recursive... in theory a
// large automata could exceed java's stack
private static boolean isFiniteSlow(LightAutomaton a, int s, HashSet<Integer> path) {
path.add(s);
LightAutomaton.Transition t = new LightAutomaton.Transition();
int count = a.initTransition(s, t);
for (int i=0;i<count;i++) {
a.getNextTransition(t);
if (path.contains(t.dest) || !isFiniteSlow(a, t.dest, path)) {
return false;
}
}
path.remove(s);
return true;
}
/**