LUCENE-5752: finish cutover

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5752@1602228 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2014-06-12 16:57:33 +00:00
parent fbe4089673
commit fe6ba518c9
59 changed files with 183 additions and 4677 deletions

View File

@ -48,7 +48,6 @@ 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;
// TODO:
// - build depth-N prefix hash?

View File

@ -981,8 +981,9 @@
<!-- Restrict access to certain Java features and install security manager: -->
<sysproperty key="junit4.tempDir" file="@{workDir}/temp" />
<sysproperty key="clover.db.dir" file="${clover.db.dir}" />
<sysproperty key="java.security.manager" value="org.apache.lucene.util.TestSecurityManager" />
<sysproperty key="java.security.policy" file="${common.dir}/tools/junit4/tests.policy" />
<!-- nocommit -->
<!--<sysproperty key="java.security.manager" value="org.apache.lucene.util.TestSecurityManager" />
<sysproperty key="java.security.policy" file="${common.dir}/tools/junit4/tests.policy" />-->
<sysproperty key="lucene.version" value="${version}"/>

View File

@ -25,10 +25,7 @@ import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RollingBuffer;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.LightAutomaton;
import org.apache.lucene.util.automaton.State;
import org.apache.lucene.util.automaton.Transition;
// TODO: maybe also toFST? then we can translate atts into FST outputs/weights

View File

@ -50,7 +50,6 @@ import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.RunAutomaton;
import org.apache.lucene.util.automaton.Transition;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.Outputs;

View File

@ -25,7 +25,6 @@ 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;
// TODO: can we share this with the frame in STE?

View File

@ -25,7 +25,6 @@ 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;
/**
* A FilteredTermsEnum that enumerates terms based upon what is accepted by a

View File

@ -24,7 +24,6 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
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;
@ -48,7 +47,6 @@ import org.apache.lucene.util.automaton.LightAutomaton;
*/
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 */
@ -62,18 +60,9 @@ public class AutomatonQuery extends MultiTermQuery {
* @param automaton Automaton to run, terms that are accepted are considered a
* match.
*/
public AutomatonQuery(final Term term, Automaton automaton) {
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);
}
@ -121,21 +110,12 @@ public class AutomatonQuery extends MultiTermQuery {
buffer.append(getClass().getSimpleName());
buffer.append(" {");
buffer.append('\n');
if (automaton == null) {
buffer.append(lightAutomaton.toString());
} else {
buffer.append(automaton.toString());
}
buffer.append(lightAutomaton.toString());
buffer.append("}");
buffer.append(ToStringUtils.boost(getBoost()));
return buffer.toString();
}
/** Returns the automaton used to create this query */
public Automaton getAutomaton() {
return automaton;
}
/** Returns the light automaton used to create this query */
public LightAutomaton getLightAutomaton() {
return lightAutomaton;

View File

@ -24,23 +24,23 @@ import java.util.List;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FilteredTermsEnum;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.FilteredTermsEnum;
import org.apache.lucene.util.Attribute;
import org.apache.lucene.util.AttributeImpl;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.UnicodeUtil;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.BasicAutomata;
import org.apache.lucene.util.automaton.BasicOperations;
import org.apache.lucene.util.automaton.ByteRunAutomaton;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.LevenshteinAutomata;
import org.apache.lucene.util.automaton.LightAutomaton;
/** Subclass of TermsEnum for enumerating all terms that are similar
* to the specified filter term.
@ -171,13 +171,13 @@ public class FuzzyTermsEnum extends TermsEnum {
new LevenshteinAutomata(UnicodeUtil.newString(termText, realPrefixLength, termText.length - realPrefixLength), transpositions);
for (int i = runAutomata.size(); i <= maxDistance; i++) {
Automaton a = builder.toAutomaton(i);
LightAutomaton a = builder.toLightAutomaton(i);
//System.out.println("compute automaton n=" + i);
// constant prefix
if (realPrefixLength > 0) {
Automaton prefix = BasicAutomata.makeString(
LightAutomaton prefix = BasicAutomata.makeStringLight(
UnicodeUtil.newString(termText, 0, realPrefixLength));
a = BasicOperations.concatenate(prefix, a);
a = BasicOperations.concatenateLight(prefix, a);
}
runAutomata.add(new CompiledAutomaton(a, true, false));
}

View File

@ -2,7 +2,6 @@ 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.LightAutomaton;
import org.apache.lucene.util.automaton.LightAutomatonProvider;
import org.apache.lucene.util.automaton.RegExp;

View File

@ -1,827 +0,0 @@
/*
* 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;
import java.io.PrintWriter;
import java.util.Arrays;
import java.util.BitSet;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Set;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
/**
* Finite-state automaton with regular expression operations.
* <p>
* Class invariants:
* <ul>
* <li>An automaton is either represented explicitly (with {@link State} and
* {@link Transition} objects) or with a singleton string (see
* {@link #getSingleton()} and {@link #expandSingleton()}) in case the automaton
* is known to accept exactly one string. (Implicitly, all states and
* transitions of an automaton are reachable from its initial state.)
* <li>Automata are always reduced (see {@link #reduce()}) and have no
* transitions to dead states (see {@link #removeDeadTransitions()}).
* <li>If an automaton is nondeterministic, then {@link #isDeterministic()}
* returns false (but the converse is not required).
* <li>Automata provided as input to operations are generally assumed to be
* disjoint.
* </ul>
* <p>
* If the states or transitions are manipulated manually, the
* {@link #restoreInvariant()} and {@link #setDeterministic(boolean)} methods
* should be used afterwards to restore representation invariants that are
* assumed by the built-in automata operations.
*
* <p>
* <p>
* Note: This class has internal mutable state and is not thread safe. It is
* the caller's responsibility to ensure any necessary synchronization if you
* wish to use the same Automaton from multiple threads. In general it is instead
* recommended to use a {@link RunAutomaton} for multithreaded matching: it is immutable,
* thread safe, and much faster.
* </p>
* @lucene.experimental
*/
public class Automaton implements Cloneable {
/**
* Minimize using Hopcroft's O(n log n) algorithm. This is regarded as one of
* the most generally efficient algorithms that exist.
*
* @see #setMinimization(int)
*/
public static final int MINIMIZE_HOPCROFT = 2;
/** Selects minimization algorithm (default: <code>MINIMIZE_HOPCROFT</code>). */
static int minimization = MINIMIZE_HOPCROFT;
/** Initial state of this automaton. */
State initial;
/**
* If true, then this automaton is definitely deterministic (i.e., there are
* no choices for any run, but a run may crash).
*/
boolean deterministic;
/** Extra data associated with this automaton. */
transient Object info;
/**
* Hash code. Recomputed by {@link MinimizationOperations#minimize(Automaton)}
*/
//int hash_code;
/** Singleton string. Null if not applicable. */
String singleton;
/** Minimize always flag. */
static boolean minimize_always = false;
/**
* Selects whether operations may modify the input automata (default:
* <code>false</code>).
*/
static boolean allow_mutation = false;
/**
* Constructs a new automaton that accepts the empty language. Using this
* constructor, automata can be constructed manually from {@link State} and
* {@link Transition} objects.
*
* @see State
* @see Transition
*/
public Automaton(State initial) {
this.initial = initial;
deterministic = true;
singleton = null;
}
public Automaton() {
this(new State());
}
/**
* Selects minimization algorithm (default: <code>MINIMIZE_HOPCROFT</code>).
*
* @param algorithm minimization algorithm
*/
static public void setMinimization(int algorithm) {
minimization = algorithm;
}
/**
* Sets or resets minimize always flag. If this flag is set, then
* {@link MinimizationOperations#minimize(Automaton)} will automatically be
* invoked after all operations that otherwise may produce non-minimal
* automata. By default, the flag is not set.
*
* @param flag if true, the flag is set
*/
static public void setMinimizeAlways(boolean flag) {
minimize_always = flag;
}
/**
* Sets or resets allow mutate flag. If this flag is set, then all automata
* operations may modify automata given as input; otherwise, operations will
* always leave input automata languages unmodified. By default, the flag is
* not set.
*
* @param flag if true, the flag is set
* @return previous value of the flag
*/
static public boolean setAllowMutate(boolean flag) {
boolean b = allow_mutation;
allow_mutation = flag;
return b;
}
/**
* Returns the state of the allow mutate flag. If this flag is set, then all
* automata operations may modify automata given as input; otherwise,
* operations will always leave input automata languages unmodified. By
* default, the flag is not set.
*
* @return current value of the flag
*/
static boolean getAllowMutate() {
return allow_mutation;
}
void checkMinimizeAlways() {
if (minimize_always) MinimizationOperations.minimize(this);
}
boolean isSingleton() {
return singleton != null;
}
/**
* Returns the singleton string for this automaton. An automaton that accepts
* exactly one string <i>may</i> be represented in singleton mode. In that
* case, this method may be used to obtain the string.
*
* @return string, null if this automaton is not in singleton mode.
*/
public String getSingleton() {
return singleton;
}
/**
* Sets initial state.
*
* @param s state
*/
/*
public void setInitialState(State s) {
initial = s;
singleton = null;
}
*/
/**
* Gets initial state.
*
* @return state
*/
public State getInitialState() {
expandSingleton();
return initial;
}
/**
* Returns deterministic flag for this automaton.
*
* @return true if the automaton is definitely deterministic, false if the
* automaton may be nondeterministic
*/
public boolean isDeterministic() {
return deterministic;
}
/**
* Sets deterministic flag for this automaton. This method should (only) be
* used if automata are constructed manually.
*
* @param deterministic true if the automaton is definitely deterministic,
* false if the automaton may be nondeterministic
*/
public void setDeterministic(boolean deterministic) {
this.deterministic = deterministic;
}
/**
* Associates extra information with this automaton.
*
* @param info extra information
*/
public void setInfo(Object info) {
this.info = info;
}
/**
* Returns extra information associated with this automaton.
*
* @return extra information
* @see #setInfo(Object)
*/
public Object getInfo() {
return info;
}
// cached
private State[] numberedStates;
public State[] getNumberedStates() {
if (numberedStates == null) {
expandSingleton();
final Set<State> visited = new HashSet<>();
final LinkedList<State> worklist = new LinkedList<>();
State states[] = new State[4];
int upto = 0;
worklist.add(initial);
visited.add(initial);
initial.number = upto;
states[upto] = initial;
upto++;
while (worklist.size() > 0) {
State s = worklist.removeFirst();
for (int i=0;i<s.numTransitions;i++) {
final Transition t = s.transitionsArray[i];
if (!visited.contains(t.to)) {
visited.add(t.to);
worklist.add(t.to);
t.to.number = upto;
if (upto == states.length) {
final State[] newArray = new State[ArrayUtil.oversize(1+upto, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
System.arraycopy(states, 0, newArray, 0, upto);
states = newArray;
}
states[upto] = t.to;
upto++;
}
}
}
if (states.length != upto) {
final State[] newArray = new State[upto];
System.arraycopy(states, 0, newArray, 0, upto);
states = newArray;
}
numberedStates = states;
}
return numberedStates;
}
public void setNumberedStates(State[] states) {
setNumberedStates(states, states.length);
}
public void setNumberedStates(State[] states, int count) {
assert count <= states.length;
// TODO: maybe we can eventually allow for oversizing here...
if (count < states.length) {
final State[] newArray = new State[count];
System.arraycopy(states, 0, newArray, 0, count);
numberedStates = newArray;
} else {
numberedStates = states;
}
}
public void clearNumberedStates() {
numberedStates = null;
}
/**
* Returns the set of reachable accept states.
*
* @return set of {@link State} objects
*/
public Set<State> getAcceptStates() {
expandSingleton();
HashSet<State> accepts = new HashSet<>();
HashSet<State> visited = new HashSet<>();
LinkedList<State> worklist = new LinkedList<>();
worklist.add(initial);
visited.add(initial);
while (worklist.size() > 0) {
State s = worklist.removeFirst();
if (s.accept) accepts.add(s);
for (Transition t : s.getTransitions())
if (!visited.contains(t.to)) {
visited.add(t.to);
worklist.add(t.to);
}
}
return accepts;
}
/**
* Adds transitions to explicit crash state to ensure that transition function
* is total.
*/
void totalize() {
State s = new State();
s.addTransition(new Transition(Character.MIN_CODE_POINT, Character.MAX_CODE_POINT,
s));
for (State p : getNumberedStates()) {
int maxi = Character.MIN_CODE_POINT;
p.sortTransitions(Transition.CompareByMinMaxThenDest);
for (Transition t : p.getTransitions()) {
if (t.min > maxi) p.addTransition(new Transition(maxi,
(t.min - 1), s));
if (t.max + 1 > maxi) maxi = t.max + 1;
}
if (maxi <= Character.MAX_CODE_POINT) p.addTransition(new Transition(
maxi, Character.MAX_CODE_POINT, s));
}
clearNumberedStates();
}
/**
* Restores representation invariant. This method must be invoked before any
* built-in automata operation is performed if automaton states or transitions
* are manipulated manually.
*
* @see #setDeterministic(boolean)
*/
public void restoreInvariant() {
removeDeadTransitions();
}
/**
* Reduces this automaton. An automaton is "reduced" by combining overlapping
* and adjacent edge intervals with same destination.
*/
public void reduce() {
final State[] states = getNumberedStates();
if (isSingleton()) return;
for (State s : states)
s.reduce();
}
/**
* Returns sorted array of all interval start points.
*/
int[] getStartPoints() {
final State[] states = getNumberedStates();
Set<Integer> pointset = new HashSet<>();
pointset.add(Character.MIN_CODE_POINT);
for (State s : states) {
for (Transition t : s.getTransitions()) {
pointset.add(t.min);
if (t.max < Character.MAX_CODE_POINT) pointset.add((t.max + 1));
}
}
int[] points = new int[pointset.size()];
int n = 0;
for (Integer m : pointset)
points[n++] = m;
Arrays.sort(points);
return points;
}
/**
* Returns the set of live states. A state is "live" if an accept state is
* reachable from it.
*
* @return set of {@link State} objects
*/
private State[] getLiveStates() {
final State[] states = getNumberedStates();
Set<State> live = new HashSet<>();
for (State q : states) {
if (q.isAccept()) {
live.add(q);
}
}
// map<state, set<state>>
@SuppressWarnings({"rawtypes","unchecked"}) Set<State> map[] = new Set[states.length];
for (int i = 0; i < map.length; i++)
map[i] = new HashSet<>();
for (State s : states) {
for(int i=0;i<s.numTransitions;i++) {
map[s.transitionsArray[i].to.number].add(s);
}
}
LinkedList<State> worklist = new LinkedList<>(live);
while (worklist.size() > 0) {
State s = worklist.removeFirst();
for (State p : map[s.number])
if (!live.contains(p)) {
live.add(p);
worklist.add(p);
}
}
return live.toArray(new State[live.size()]);
}
/**
* Removes transitions to dead states and calls {@link #reduce()}.
* (A state is "dead" if no accept state is
* reachable from it.)
*/
public void removeDeadTransitions() {
final State[] states = getNumberedStates();
//clearHashCode();
if (isSingleton()) return;
State[] live = getLiveStates();
BitSet liveSet = new BitSet(states.length);
for (State s : live)
liveSet.set(s.number);
for (State s : states) {
// filter out transitions to dead states:
int upto = 0;
for(int i=0;i<s.numTransitions;i++) {
final Transition t = s.transitionsArray[i];
if (liveSet.get(t.to.number)) {
s.transitionsArray[upto++] = s.transitionsArray[i];
}
}
s.numTransitions = upto;
}
for(int i=0;i<live.length;i++) {
live[i].number = i;
}
if (live.length > 0) {
setNumberedStates(live);
} else {
// sneaky corner case -- if machine accepts no strings
clearNumberedStates();
}
reduce();
}
/**
* Returns a sorted array of transitions for each state (and sets state
* numbers).
*/
public Transition[][] getSortedTransitions() {
final State[] states = getNumberedStates();
Transition[][] transitions = new Transition[states.length][];
for (State s : states) {
s.sortTransitions(Transition.CompareByMinMaxThenDest);
s.trimTransitionsArray();
transitions[s.number] = s.transitionsArray;
assert s.transitionsArray != null;
}
return transitions;
}
/**
* Expands singleton representation to normal representation. Does nothing if
* not in singleton representation.
*/
public void expandSingleton() {
if (isSingleton()) {
State p = new State();
initial = p;
for (int i = 0, cp = 0; i < singleton.length(); i += Character.charCount(cp)) {
State q = new State();
p.addTransition(new Transition(cp = singleton.codePointAt(i), q));
p = q;
}
p.accept = true;
deterministic = true;
singleton = null;
}
}
/**
* Returns the number of states in this automaton.
*/
public int getNumberOfStates() {
if (isSingleton()) return singleton.codePointCount(0, singleton.length()) + 1;
return getNumberedStates().length;
}
/**
* Returns the number of transitions in this automaton. This number is counted
* as the total number of edges, where one edge may be a character interval.
*/
public int getNumberOfTransitions() {
if (isSingleton()) return singleton.codePointCount(0, singleton.length());
int c = 0;
for (State s : getNumberedStates())
c += s.numTransitions();
return c;
}
@Override
public boolean equals(Object obj) {
throw new UnsupportedOperationException("use BasicOperations.sameLanguage instead");
}
@Override
public int hashCode() {
throw new UnsupportedOperationException();
}
/**
* Must be invoked when the stored hash code may no longer be valid.
*/
/*
void clearHashCode() {
hash_code = 0;
}
*/
/**
* Returns a string representation of this automaton.
*/
@Override
public String toString() {
StringBuilder b = new StringBuilder();
if (isSingleton()) {
b.append("singleton: ");
int length = singleton.codePointCount(0, singleton.length());
int codepoints[] = new int[length];
for (int i = 0, j = 0, cp = 0; i < singleton.length(); i += Character.charCount(cp))
codepoints[j++] = cp = singleton.codePointAt(i);
for (int c : codepoints)
Transition.appendCharString(c, b);
b.append("\n");
} else {
State[] states = getNumberedStates();
b.append("initial state: ").append(initial.number).append("\n");
for (State s : states)
b.append(s.toString());
}
return b.toString();
}
/**
* Returns <a href="http://www.research.att.com/sw/tools/graphviz/"
* target="_top">Graphviz Dot</a> representation of this automaton.
*/
public String toDot() {
StringBuilder b = new StringBuilder("digraph Automaton {\n");
b.append(" rankdir = LR;\n");
State[] states = getNumberedStates();
for (State s : states) {
b.append(" ").append(s.number);
if (s.accept) b.append(" [shape=doublecircle,label=\"" + s.number + "\"];\n");
else b.append(" [shape=circle,label=\" " + s.number + "\"];\n");
if (s == initial) {
b.append(" initial [shape=plaintext,label=\"\"];\n");
b.append(" initial -> ").append(s.number).append("\n");
}
for (Transition t : s.getTransitions()) {
b.append(" ").append(s.number);
t.appendDot(b);
}
}
return b.append("}\n").toString();
}
/**
* Returns a clone of this automaton, expands if singleton.
*/
Automaton cloneExpanded() {
Automaton a = clone();
a.expandSingleton();
return a;
}
/**
* Returns a clone of this automaton unless <code>allow_mutation</code> is
* set, expands if singleton.
*/
Automaton cloneExpandedIfRequired() {
if (allow_mutation) {
expandSingleton();
return this;
} else return cloneExpanded();
}
/**
* Returns a clone of this automaton.
*/
@Override
public Automaton clone() {
try {
Automaton a = (Automaton) super.clone();
if (!isSingleton()) {
HashMap<State,State> m = new HashMap<>();
State[] states = getNumberedStates();
for (State s : states)
m.put(s, new State());
for (State s : states) {
State p = m.get(s);
p.accept = s.accept;
if (s == initial) a.initial = p;
for (Transition t : s.getTransitions())
p.addTransition(new Transition(t.min, t.max, m.get(t.to)));
}
}
a.clearNumberedStates();
return a;
} catch (CloneNotSupportedException e) {
throw new RuntimeException(e);
}
}
/**
* Returns a clone of this automaton, or this automaton itself if
* <code>allow_mutation</code> flag is set.
*/
Automaton cloneIfRequired() {
if (allow_mutation) return this;
else return clone();
}
/**
* See {@link BasicOperations#concatenate(Automaton, Automaton)}.
*/
public Automaton concatenate(Automaton a) {
return BasicOperations.concatenate(this, a);
}
/**
* See {@link BasicOperations#concatenate(List)}.
*/
static public Automaton concatenate(List<Automaton> l) {
return BasicOperations.concatenate(l);
}
/**
* See {@link BasicOperations#optional(Automaton)}.
*/
public Automaton optional() {
return BasicOperations.optional(this);
}
/**
* See {@link BasicOperations#repeat(Automaton)}.
*/
public Automaton repeat() {
return BasicOperations.repeat(this);
}
/**
* See {@link BasicOperations#repeat(Automaton, int)}.
*/
public Automaton repeat(int min) {
return BasicOperations.repeat(this, min);
}
/**
* See {@link BasicOperations#repeat(Automaton, int, int)}.
*/
public Automaton repeat(int min, int max) {
return BasicOperations.repeat(this, min, max);
}
/**
* See {@link BasicOperations#complement(Automaton)}.
*/
public Automaton complement() {
return BasicOperations.complement(this);
}
/**
* See {@link BasicOperations#minus(Automaton, Automaton)}.
*/
public Automaton minus(Automaton a) {
return BasicOperations.minus(this, a);
}
/**
* See {@link BasicOperations#intersection(Automaton, Automaton)}.
*/
public Automaton intersection(Automaton a) {
return BasicOperations.intersection(this, a);
}
/**
* See {@link BasicOperations#subsetOf(Automaton, Automaton)}.
*/
public boolean subsetOf(Automaton a) {
return BasicOperations.subsetOf(this, a);
}
/**
* See {@link BasicOperations#union(Automaton, Automaton)}.
*/
public Automaton union(Automaton a) {
return BasicOperations.union(this, a);
}
/**
* See {@link BasicOperations#union(Collection)}.
*/
static public Automaton union(Collection<Automaton> l) {
return BasicOperations.union(l);
}
/**
* See {@link BasicOperations#determinize(Automaton)}.
*/
public void determinize() {
BasicOperations.determinize(this);
}
/**
* See {@link BasicOperations#isEmptyString(Automaton)}.
*/
public boolean isEmptyString() {
return BasicOperations.isEmptyString(this);
}
/**
* See {@link MinimizationOperations#minimize(Automaton)}. Returns the
* automaton being given as argument.
*/
public static Automaton minimize(Automaton a) {
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

@ -1,50 +0,0 @@
/*
* 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 AutomatonProvider {
/**
* Returns automaton of the given name.
*
* @param name automaton name
* @return automaton
* @throws IOException if errors occur
*/
public Automaton getAutomaton(String name) throws IOException;
}

View File

@ -45,30 +45,12 @@ final public class BasicAutomata {
/**
* Returns a new (deterministic) automaton with the empty language.
*/
public static Automaton makeEmpty() {
Automaton a = new Automaton();
State s = new State();
a.initial = s;
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.
*/
public static Automaton makeEmptyString() {
Automaton a = new Automaton();
a.singleton = "";
a.deterministic = true;
return a;
}
/**
* Returns a new (deterministic) automaton that accepts only the empty string.
*/
@ -79,20 +61,6 @@ final public class BasicAutomata {
return a;
}
/**
* Returns a new (deterministic) automaton that accepts all strings.
*/
public static Automaton makeAnyString() {
Automaton a = new Automaton();
State s = new State();
a.initial = s;
s.accept = true;
s.addTransition(new Transition(Character.MIN_CODE_POINT, Character.MAX_CODE_POINT,
s));
a.deterministic = true;
return a;
}
/**
* Returns a new (deterministic) automaton that accepts all strings.
*/
@ -105,13 +73,6 @@ final public class BasicAutomata {
return a;
}
/**
* Returns a new (deterministic) automaton that accepts any single codepoint.
*/
public static Automaton makeAnyChar() {
return makeCharRange(Character.MIN_CODE_POINT, Character.MAX_CODE_POINT);
}
/**
* Returns a new (deterministic) automaton that accepts any single codepoint.
*/
@ -119,17 +80,6 @@ final public class BasicAutomata {
return makeCharRangeLight(Character.MIN_CODE_POINT, Character.MAX_CODE_POINT);
}
/**
* Returns a new (deterministic) automaton that accepts a single codepoint of
* the given value.
*/
public static Automaton makeChar(int c) {
Automaton a = new Automaton();
a.singleton = new String(Character.toChars(c));
a.deterministic = true;
return a;
}
/**
* Returns a new (deterministic) automaton that accepts a single codepoint of
* the given value.
@ -138,49 +88,23 @@ final public class BasicAutomata {
return makeCharRangeLight(c, c);
}
/**
* Returns a new (deterministic) automaton that accepts a single codepoint whose
* value is in the given interval (including both end points).
*/
public static Automaton makeCharRange(int min, int max) {
if (min == max) return makeChar(min);
Automaton a = new Automaton();
State s1 = new State();
State s2 = new State();
a.initial = s1;
s2.accept = true;
if (min <= max) s1.addTransition(new Transition(min, max, s2));
a.deterministic = true;
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) {
if (min > max) {
return makeEmptyLight();
}
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.addTransition(s1, s2, min, max);
a.finish();
return a;
}
/**
* Constructs sub-automaton corresponding to decimal numbers of length
* x.substring(n).length().
*/
private static State anyOfRightLength(String x, int n) {
State s = new State();
if (x.length() == n) s.setAccept(true);
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().
@ -195,25 +119,6 @@ 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 State atLeast(String x, int n, Collection<State> initials,
boolean zeros) {
State s = new State();
if (x.length() == n) s.setAccept(true);
else {
if (zeros) initials.add(s);
char c = x.charAt(n);
s.addTransition(new Transition(c, atLeast(x, n + 1, initials, zeros
&& c == '0')));
if (c < '9') s.addTransition(new Transition((char) (c + 1), '9',
anyOfRightLength(x, n + 1)));
}
return s;
}
/**
* Constructs sub-automaton corresponding to decimal numbers of value at least
* x.substring(n) and length x.substring(n).length().
@ -236,22 +141,6 @@ 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 State atMost(String x, int n) {
State s = new State();
if (x.length() == n) s.setAccept(true);
else {
char c = x.charAt(n);
s.addTransition(new Transition(c, atMost(x, (char) n + 1)));
if (c > '0') s.addTransition(new Transition('0', (char) (c - 1),
anyOfRightLength(x, n + 1)));
}
return s;
}
/**
* Constructs sub-automaton corresponding to decimal numbers of value at most
* x.substring(n) and length x.substring(n).length().
@ -270,32 +159,6 @@ 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 State between(String x, String y, int n,
Collection<State> initials, boolean zeros) {
State s = new State();
if (x.length() == n) s.setAccept(true);
else {
if (zeros) initials.add(s);
char cx = x.charAt(n);
char cy = y.charAt(n);
if (cx == cy) s.addTransition(new Transition(cx, between(x, y, n + 1,
initials, zeros && cx == '0')));
else { // cx<cy
s.addTransition(new Transition(cx, atLeast(x, n + 1, initials, zeros
&& cx == '0')));
s.addTransition(new Transition(cy, atMost(y, n + 1)));
if (cx + 1 < cy) s.addTransition(new Transition((char) (cx + 1),
(char) (cy - 1), anyOfRightLength(x, n + 1)));
}
}
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()
@ -327,53 +190,6 @@ final public class BasicAutomata {
return s;
}
/**
* 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 Automaton makeInterval(int min, int max, int digits)
throws IllegalArgumentException {
Automaton a = new Automaton();
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();
Collection<State> initials = new ArrayList<>();
a.initial = between(x, y, 0, initials, digits <= 0);
if (digits <= 0) {
ArrayList<StatePair> pairs = new ArrayList<>();
for (State p : initials)
if (a.initial != p) pairs.add(new StatePair(a.initial, p));
BasicOperations.addEpsilons(a, pairs);
a.initial.addTransition(new Transition('0', a.initial));
a.deterministic = false;
} else a.deterministic = true;
a.checkMinimizeAlways();
return a;
}
/**
* Returns a new automaton that accepts strings representing decimal
* non-negative integers in the given interval.
@ -426,9 +242,7 @@ final public class BasicAutomata {
a2.copy(a1);
for (int p : initials) {
if (p != 0) {
a2.addEpsilon(0, p+1);
}
a2.addEpsilon(0, p+1);
}
a2.finish();
@ -438,17 +252,6 @@ final public class BasicAutomata {
}
}
/**
* Returns a new (deterministic) automaton that accepts the single given
* string.
*/
public static Automaton makeString(String s) {
Automaton a = new Automaton();
a.singleton = s;
a.deterministic = true;
return a;
}
/**
* Returns a new (deterministic) automaton that accepts the single given
* string.
@ -469,20 +272,10 @@ final public class BasicAutomata {
return a;
}
public static Automaton makeString(int[] word, int offset, int length) {
Automaton a = new Automaton();
a.setDeterministic(true);
State s = new State();
a.initial = s;
for (int i = offset; i < offset+length; i++) {
State s2 = new State();
s.addTransition(new Transition(word[i], s2));
s = s2;
}
s.accept = true;
return a;
}
/**
* Returns a new (deterministic) automaton that accepts the single given
* string from the specified unicode code points.
*/
public static LightAutomaton makeStringLight(int[] word, int offset, int length) {
LightAutomaton a = new LightAutomaton();
a.createState();
@ -498,27 +291,6 @@ final public class BasicAutomata {
return a;
}
/**
* 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 Automaton makeStringUnion(Collection<BytesRef> utf8Strings) {
if (utf8Strings.isEmpty()) {
return makeEmpty();
} else {
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

View File

@ -52,99 +52,24 @@ import org.apache.lucene.util.RamUsageEstimator;
final public class BasicOperations {
private BasicOperations() {}
/**
* Returns an automaton that accepts the concatenation of the languages of the
* given automata.
* <p>
* Complexity: linear in number of states.
*/
static public Automaton concatenate(Automaton a1, Automaton a2) {
if (a1.isSingleton() && a2.isSingleton()) return BasicAutomata
.makeString(a1.singleton + a2.singleton);
if (isEmpty(a1) || isEmpty(a2))
return BasicAutomata.makeEmpty();
// adding epsilon transitions with the NFA concatenation algorithm
// in this case always produces a resulting DFA, preventing expensive
// redundant determinize() calls for this common case.
boolean deterministic = a1.isSingleton() && a2.isDeterministic();
if (a1 == a2) {
a1 = a1.cloneExpanded();
a2 = a2.cloneExpanded();
} else {
a1 = a1.cloneExpandedIfRequired();
a2 = a2.cloneExpandedIfRequired();
}
for (State s : a1.getAcceptStates()) {
s.accept = false;
s.addEpsilon(a2.initial);
}
a1.deterministic = deterministic;
//a1.clearHashCode();
a1.clearNumberedStates();
a1.checkMinimizeAlways();
return a1;
}
/**
* Returns an automaton that accepts the concatenation of the languages of the
* given automata.
* <p>
* Complexity: linear in total number of states.
*/
// nocommit can we use varargs? rob was unhappy before?
static public Automaton concatenate(List<Automaton> l) {
if (l.isEmpty()) return BasicAutomata.makeEmptyString();
boolean all_singleton = true;
for (Automaton a : l)
if (!a.isSingleton()) {
all_singleton = false;
break;
}
if (all_singleton) {
StringBuilder b = new StringBuilder();
for (Automaton a : l)
b.append(a.singleton);
return BasicAutomata.makeString(b.toString());
} else {
for (Automaton a : l)
if (BasicOperations.isEmpty(a)) return BasicAutomata.makeEmpty();
Set<Integer> ids = new HashSet<>();
for (Automaton a : l)
ids.add(System.identityHashCode(a));
boolean has_aliases = ids.size() != l.size();
Automaton b = l.get(0);
if (has_aliases) b = b.cloneExpanded();
else b = b.cloneExpandedIfRequired();
Set<State> ac = b.getAcceptStates();
boolean first = true;
for (Automaton a : l)
if (first) first = false;
else {
if (a.isEmptyString()) continue;
Automaton aa = a;
if (has_aliases) aa = aa.cloneExpanded();
else aa = aa.cloneExpandedIfRequired();
Set<State> ns = aa.getAcceptStates();
for (State s : ac) {
s.accept = false;
s.addEpsilon(aa.initial);
if (s.accept) ns.add(s);
}
ac = ns;
}
b.deterministic = false;
//b.clearHashCode();
b.clearNumberedStates();
b.checkMinimizeAlways();
return b;
}
}
static public LightAutomaton concatenateLight(LightAutomaton a1, LightAutomaton a2) {
// nocommit we lost the two-arg optimization here (prepend tiny automaton in front of huge one)
return concatenateLight(Arrays.asList(a1, a2));
}
/**
* Returns an automaton that accepts the concatenation of the languages of the
* given automata.
* <p>
* Complexity: linear in total number of states.
*/
static public LightAutomaton concatenateLight(List<LightAutomaton> l) {
LightAutomaton result = new LightAutomaton();
LightAutomaton.Transition scratch = new LightAutomaton.Transition();
@ -213,26 +138,6 @@ final public class BasicOperations {
return result;
}
/**
* Returns an automaton that accepts the union of the empty string and the
* language of the given automaton.
* <p>
* Complexity: linear in number of states.
*/
static public Automaton optional(Automaton a) {
a = a.cloneExpandedIfRequired();
State s = new State();
s.addEpsilon(a.initial);
s.accept = true;
a.initial = s;
a.deterministic = false;
//a.clearHashCode();
a.clearNumberedStates();
a.checkMinimizeAlways();
return a;
}
/**
* Returns an automaton that accepts the union of the empty string and the
* language of the given automaton.
@ -268,28 +173,6 @@ final public class BasicOperations {
return result;
}
/**
* Returns an automaton that accepts the Kleene star (zero or more
* concatenated repetitions) of the language of the given automaton. Never
* modifies the input automaton language.
* <p>
* Complexity: linear in number of states.
*/
static public Automaton repeat(Automaton a) {
a = a.cloneExpanded();
State s = new State();
s.accept = true;
s.addEpsilon(a.initial);
for (State p : a.getAcceptStates())
p.addEpsilon(s);
a.initial = s;
a.deterministic = false;
//a.clearHashCode();
a.clearNumberedStates();
a.checkMinimizeAlways();
return a;
}
/**
* Returns an automaton that accepts the Kleene star (zero or more
* concatenated repetitions) of the language of the given automaton. Never
@ -324,6 +207,9 @@ final public class BasicOperations {
return builder.finish();
}
// nocommit make this privately computed in LA
/** Returns true if the automaton is deterministic. */
public static boolean isDeterministic(LightAutomaton a) {
BitSet done = new BitSet(a.getNumStates());
List<Integer> queue = new ArrayList<>();
@ -351,21 +237,6 @@ final public class BasicOperations {
return true;
}
/**
* Returns an automaton that accepts <code>min</code> or more concatenated
* repetitions of the language of the given automaton.
* <p>
* Complexity: linear in number of states and in <code>min</code>.
*/
static public Automaton repeat(Automaton a, int min) {
if (min == 0) return repeat(a);
List<Automaton> as = new ArrayList<>();
while (min-- > 0)
as.add(a);
as.add(repeat(a));
return concatenate(as);
}
/**
* Returns an automaton that accepts <code>min</code> or more concatenated
* repetitions of the language of the given automaton.
@ -384,45 +255,6 @@ final public class BasicOperations {
return concatenateLight(as);
}
/**
* Returns an automaton that accepts between <code>min</code> and
* <code>max</code> (including both) concatenated repetitions of the language
* of the given automaton.
* <p>
* Complexity: linear in number of states and in <code>min</code> and
* <code>max</code>.
*/
static public Automaton repeat(Automaton a, int min, int max) {
if (min > max) return BasicAutomata.makeEmpty();
max -= min;
a.expandSingleton();
Automaton b;
if (min == 0) b = BasicAutomata.makeEmptyString();
else if (min == 1) b = a.clone();
else {
List<Automaton> as = new ArrayList<>();
while (min-- > 0)
as.add(a);
b = concatenate(as);
}
if (max > 0) {
Automaton d = a.clone();
while (--max > 0) {
Automaton c = a.clone();
for (State p : c.getAcceptStates())
p.addEpsilon(d.initial);
d = c;
}
for (State p : b.getAcceptStates())
p.addEpsilon(d.initial);
b.deterministic = false;
//b.clearHashCode();
b.clearNumberedStates();
b.checkMinimizeAlways();
}
return b;
}
/**
* Returns an automaton that accepts between <code>min</code> and
* <code>max</code> (including both) concatenated repetitions of the language
@ -469,22 +301,6 @@ final public class BasicOperations {
return b;
}
/**
* Returns a (deterministic) automaton that accepts the complement of the
* language of the given automaton.
* <p>
* Complexity: linear in number of states (if already deterministic).
*/
static public Automaton complement(Automaton a) {
a = a.cloneExpandedIfRequired();
a.determinize();
a.totalize();
for (State p : a.getNumberedStates())
p.accept = !p.accept;
a.removeDeadTransitions();
return a;
}
/**
* Returns a (deterministic) automaton that accepts the complement of the
* language of the given automaton.
@ -500,25 +316,6 @@ final public class BasicOperations {
return removeDeadTransitions(a);
}
/**
* Returns a (deterministic) automaton that accepts the intersection of the
* language of <code>a1</code> and the complement of the language of
* <code>a2</code>. As a side-effect, the automata may be determinized, if not
* already deterministic.
* <p>
* Complexity: quadratic in number of states (if already deterministic).
*/
static public Automaton minus(Automaton a1, Automaton a2) {
if (BasicOperations.isEmpty(a1) || a1 == a2) return BasicAutomata
.makeEmpty();
if (BasicOperations.isEmpty(a2)) return a1.cloneIfRequired();
if (a1.isSingleton()) {
if (BasicOperations.run(a2, a1.singleton)) return BasicAutomata.makeEmpty();
else return a1.cloneIfRequired();
}
return intersection(a1, a2.complement());
}
/**
* Returns a (deterministic) automaton that accepts the intersection of the
* language of <code>a1</code> and the complement of the language of
@ -537,60 +334,6 @@ final public class BasicOperations {
return intersectionLight(a1, complementLight(a2));
}
/**
* Returns an automaton that accepts the intersection of the languages of the
* given automata. Never modifies the input automata languages.
* <p>
* Complexity: quadratic in number of states.
*/
static public Automaton intersection(Automaton a1, Automaton a2) {
if (a1.isSingleton()) {
if (BasicOperations.run(a2, a1.singleton)) return a1.cloneIfRequired();
else return BasicAutomata.makeEmpty();
}
if (a2.isSingleton()) {
if (BasicOperations.run(a1, a2.singleton)) return a2.cloneIfRequired();
else return BasicAutomata.makeEmpty();
}
if (a1 == a2) return a1.cloneIfRequired();
Transition[][] transitions1 = a1.getSortedTransitions();
Transition[][] transitions2 = a2.getSortedTransitions();
Automaton c = new Automaton();
LinkedList<StatePair> worklist = new LinkedList<>();
HashMap<StatePair,StatePair> newstates = new HashMap<>();
StatePair p = new StatePair(c.initial, a1.initial, a2.initial);
worklist.add(p);
newstates.put(p, p);
while (worklist.size() > 0) {
p = worklist.removeFirst();
p.s.accept = p.s1.accept && p.s2.accept;
Transition[] t1 = transitions1[p.s1.number];
Transition[] t2 = transitions2[p.s2.number];
for (int n1 = 0, b2 = 0; n1 < t1.length; n1++) {
while (b2 < t2.length && t2[b2].max < t1[n1].min)
b2++;
for (int n2 = b2; n2 < t2.length && t1[n1].max >= t2[n2].min; n2++)
if (t2[n2].max >= t1[n1].min) {
StatePair q = new StatePair(t1[n1].to, t2[n2].to);
StatePair r = newstates.get(q);
if (r == null) {
q.s = new State();
worklist.add(q);
newstates.put(q, q);
r = q;
}
int min = t1[n1].min > t2[n2].min ? t1[n1].min : t2[n2].min;
int max = t1[n1].max < t2[n2].max ? t1[n1].max : t2[n2].max;
p.s.addTransition(new Transition(min, max, r.s));
}
}
}
c.deterministic = a1.deterministic && a2.deterministic;
c.removeDeadTransitions();
c.checkMinimizeAlways();
return c;
}
/**
* Returns an automaton that accepts the intersection of the languages of the
* given automata. Never modifies the input automata languages.
@ -709,80 +452,6 @@ final public class BasicOperations {
}
*/
/** Returns true if these two automata accept exactly the
* same language. This is a costly computation! Note
* also that a1 and a2 will be determinized as a side
* effect. */
public static boolean sameLanguage(Automaton a1, Automaton a2) {
if (a1 == a2) {
return true;
}
if (a1.isSingleton() && a2.isSingleton()) {
return a1.singleton.equals(a2.singleton);
} else if (a1.isSingleton()) {
// subsetOf is faster if the first automaton is a singleton
return subsetOf(a1, a2) && subsetOf(a2, a1);
} else {
return subsetOf(a2, a1) && subsetOf(a1, a2);
}
}
/**
* Returns true if the language of <code>a1</code> is a subset of the language
* of <code>a2</code>. As a side-effect, <code>a2</code> is determinized if
* not already marked as deterministic.
* <p>
* Complexity: quadratic in number of states.
*/
public static boolean subsetOf(Automaton a1, Automaton a2) {
if (a1 == a2) return true;
if (a1.isSingleton()) {
if (a2.isSingleton()) return a1.singleton.equals(a2.singleton);
return BasicOperations.run(a2, a1.singleton);
}
a2.determinize();
Transition[][] transitions1 = a1.getSortedTransitions();
Transition[][] transitions2 = a2.getSortedTransitions();
LinkedList<StatePair> worklist = new LinkedList<>();
HashSet<StatePair> visited = new HashSet<>();
StatePair p = new StatePair(a1.initial, a2.initial);
worklist.add(p);
visited.add(p);
while (worklist.size() > 0) {
p = worklist.removeFirst();
if (p.s1.accept && !p.s2.accept) {
return false;
}
Transition[] t1 = transitions1[p.s1.number];
Transition[] t2 = transitions2[p.s2.number];
for (int n1 = 0, b2 = 0; n1 < t1.length; n1++) {
while (b2 < t2.length && t2[b2].max < t1[n1].min)
b2++;
int min1 = t1[n1].min, max1 = t1[n1].max;
for (int n2 = b2; n2 < t2.length && t1[n1].max >= t2[n2].min; n2++) {
if (t2[n2].min > min1) {
return false;
}
if (t2[n2].max < Character.MAX_CODE_POINT) min1 = t2[n2].max + 1;
else {
min1 = Character.MAX_CODE_POINT;
max1 = Character.MIN_CODE_POINT;
}
StatePair q = new StatePair(t1[n1].to, t2[n2].to);
if (!visited.contains(q)) {
worklist.add(q);
visited.add(q);
}
}
if (min1 <= max1) {
return false;
}
}
}
return true;
}
/** Returns true if these two automata accept exactly the
* same language. This is a costly computation! Note
* also that a1 and a2 will be determinized as a side
@ -921,62 +590,6 @@ final public class BasicOperations {
}
*/
/**
* Returns an automaton that accepts the union of the languages of the given
* automata.
* <p>
* Complexity: linear in number of states.
*/
public static Automaton union(Automaton a1, Automaton a2) {
if ((a1.isSingleton() && a2.isSingleton() && a1.singleton
.equals(a2.singleton))
|| a1 == a2) return a1.cloneIfRequired();
if (a1 == a2) {
a1 = a1.cloneExpanded();
a2 = a2.cloneExpanded();
} else {
a1 = a1.cloneExpandedIfRequired();
a2 = a2.cloneExpandedIfRequired();
}
State s = new State();
s.addEpsilon(a1.initial);
s.addEpsilon(a2.initial);
a1.initial = s;
a1.deterministic = false;
//a1.clearHashCode();
a1.clearNumberedStates();
a1.checkMinimizeAlways();
return a1;
}
/**
* Returns an automaton that accepts the union of the languages of the given
* automata.
* <p>
* Complexity: linear in number of states.
*/
public static Automaton union(Collection<Automaton> l) {
Set<Integer> ids = new HashSet<>();
for (Automaton a : l)
ids.add(System.identityHashCode(a));
boolean has_aliases = ids.size() != l.size();
State s = new State();
for (Automaton b : l) {
if (BasicOperations.isEmpty(b)) continue;
Automaton bb = b;
if (has_aliases) bb = bb.cloneExpanded();
else bb = bb.cloneExpandedIfRequired();
s.addEpsilon(bb.initial);
}
Automaton a = new Automaton();
a.initial = s;
a.deterministic = false;
//a.clearHashCode();
a.clearNumberedStates();
a.checkMinimizeAlways();
return a;
}
/**
* Returns an automaton that accepts the union of the languages of the given
* automata.
@ -1072,257 +685,6 @@ final public class BasicOperations {
return result;
}
// Simple custom ArrayList<Transition>
private final static class TransitionList {
Transition[] transitions = new Transition[2];
int count;
public void add(Transition t) {
if (transitions.length == count) {
Transition[] newArray = new Transition[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
System.arraycopy(transitions, 0, newArray, 0, count);
transitions = newArray;
}
transitions[count++] = t;
}
}
// Holds all transitions that start on this int point, or
// end at this point-1
private final static class PointTransitions implements Comparable<PointTransitions> {
int point;
final TransitionList ends = new TransitionList();
final TransitionList starts = new TransitionList();
@Override
public int compareTo(PointTransitions other) {
return point - other.point;
}
public void reset(int point) {
this.point = point;
ends.count = 0;
starts.count = 0;
}
@Override
public boolean equals(Object other) {
return ((PointTransitions) other).point == point;
}
@Override
public int hashCode() {
return point;
}
}
private final static class PointTransitionSet {
int count;
PointTransitions[] points = new PointTransitions[5];
private final static int HASHMAP_CUTOVER = 30;
private final HashMap<Integer,PointTransitions> map = new HashMap<>();
private boolean useHash = false;
private PointTransitions next(int point) {
// 1st time we are seeing this point
if (count == points.length) {
final PointTransitions[] newArray = new PointTransitions[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
System.arraycopy(points, 0, newArray, 0, count);
points = newArray;
}
PointTransitions points0 = points[count];
if (points0 == null) {
points0 = points[count] = new PointTransitions();
}
points0.reset(point);
count++;
return points0;
}
private PointTransitions find(int point) {
if (useHash) {
final Integer pi = point;
PointTransitions p = map.get(pi);
if (p == null) {
p = next(point);
map.put(pi, p);
}
return p;
} else {
for(int i=0;i<count;i++) {
if (points[i].point == point) {
return points[i];
}
}
final PointTransitions p = next(point);
if (count == HASHMAP_CUTOVER) {
// switch to HashMap on the fly
assert map.size() == 0;
for(int i=0;i<count;i++) {
map.put(points[i].point, points[i]);
}
useHash = true;
}
return p;
}
}
public void reset() {
if (useHash) {
map.clear();
useHash = false;
}
count = 0;
}
public void sort() {
// Tim sort performs well on already sorted arrays:
if (count > 1) ArrayUtil.timSort(points, 0, count);
}
public void add(Transition t) {
find(t.min).starts.add(t);
find(1+t.max).ends.add(t);
}
@Override
public String toString() {
StringBuilder s = new StringBuilder();
for(int i=0;i<count;i++) {
if (i > 0) {
s.append(' ');
}
s.append(points[i].point).append(':').append(points[i].starts.count).append(',').append(points[i].ends.count);
}
return s.toString();
}
}
/**
* Determinizes the given automaton.
* <p>
* Worst case complexity: exponential in number of states.
*/
public static void determinize(Automaton a) {
if (a.deterministic || a.isSingleton()) {
return;
}
final State[] allStates = a.getNumberedStates();
// subset construction
final boolean initAccept = a.initial.accept;
final int initNumber = a.initial.number;
a.initial = new State();
SortedIntSet.FrozenIntSet initialset = new SortedIntSet.FrozenIntSet(initNumber, a.initial);
LinkedList<SortedIntSet.FrozenIntSet> worklist = new LinkedList<>();
Map<SortedIntSet.FrozenIntSet,State> newstate = new HashMap<>();
worklist.add(initialset);
a.initial.accept = initAccept;
newstate.put(initialset, a.initial);
int newStateUpto = 0;
State[] newStatesArray = new State[5];
newStatesArray[newStateUpto] = a.initial;
a.initial.number = newStateUpto;
newStateUpto++;
// like Set<Integer,PointTransitions>
final PointTransitionSet points = new PointTransitionSet();
// like SortedMap<Integer,Integer>
final SortedIntSet statesSet = new SortedIntSet(5);
while (worklist.size() > 0) {
SortedIntSet.FrozenIntSet s = worklist.removeFirst();
// Collate all outgoing transitions by min/1+max:
for(int i=0;i<s.values.length;i++) {
final State s0 = allStates[s.values[i]];
for(int j=0;j<s0.numTransitions;j++) {
points.add(s0.transitionsArray[j]);
}
}
if (points.count == 0) {
// No outgoing transitions -- skip it
continue;
}
points.sort();
int lastPoint = -1;
int accCount = 0;
final State r = s.state;
for(int i=0;i<points.count;i++) {
final int point = points.points[i].point;
if (statesSet.upto > 0) {
assert lastPoint != -1;
statesSet.computeHash();
State q = newstate.get(statesSet);
if (q == null) {
q = new State();
final SortedIntSet.FrozenIntSet p = statesSet.freeze(q);
worklist.add(p);
if (newStateUpto == newStatesArray.length) {
final State[] newArray = new State[ArrayUtil.oversize(1+newStateUpto, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
System.arraycopy(newStatesArray, 0, newArray, 0, newStateUpto);
newStatesArray = newArray;
}
newStatesArray[newStateUpto] = q;
q.number = newStateUpto;
newStateUpto++;
q.accept = accCount > 0;
newstate.put(p, q);
} else {
assert (accCount > 0 ? true:false) == q.accept: "accCount=" + accCount + " vs existing accept=" + q.accept + " states=" + statesSet;
}
r.addTransition(new Transition(lastPoint, point-1, q));
}
// process transitions that end on this point
// (closes an overlapping interval)
Transition[] transitions = points.points[i].ends.transitions;
int limit = points.points[i].ends.count;
for(int j=0;j<limit;j++) {
final Transition t = transitions[j];
final Integer num = t.to.number;
statesSet.decr(num);
accCount -= t.to.accept ? 1:0;
}
points.points[i].ends.count = 0;
// process transitions that start on this point
// (opens a new interval)
transitions = points.points[i].starts.transitions;
limit = points.points[i].starts.count;
for(int j=0;j<limit;j++) {
final Transition t = transitions[j];
final Integer num = t.to.number;
statesSet.incr(num);
accCount += t.to.accept ? 1:0;
}
lastPoint = point;
points.points[i].starts.count = 0;
}
points.reset();
assert statesSet.upto == 0: "upto=" + statesSet.upto;
}
a.deterministic = true;
a.setNumberedStates(newStatesArray, newStateUpto);
}
// Simple custom ArrayList<Transition>
private final static class TransitionListLight {
// dest, min, max
@ -1579,89 +941,6 @@ final public class BasicOperations {
return b.finish();
}
/**
* Adds epsilon transitions to the given automaton. This method adds extra
* character interval transitions that are equivalent to the given set of
* epsilon transitions.
*
* @param pairs collection of {@link StatePair} objects representing pairs of
* source/destination states where epsilon transitions should be
* added
*/
public static void addEpsilons(Automaton a, Collection<StatePair> pairs) {
a.expandSingleton();
HashMap<State,HashSet<State>> forward = new HashMap<>();
HashMap<State,HashSet<State>> back = new HashMap<>();
for (StatePair p : pairs) {
HashSet<State> to = forward.get(p.s1);
if (to == null) {
to = new HashSet<>();
forward.put(p.s1, to);
}
to.add(p.s2);
HashSet<State> from = back.get(p.s2);
if (from == null) {
from = new HashSet<>();
back.put(p.s2, from);
}
from.add(p.s1);
}
// calculate epsilon closure
LinkedList<StatePair> worklist = new LinkedList<>(pairs);
HashSet<StatePair> workset = new HashSet<>(pairs);
while (!worklist.isEmpty()) {
StatePair p = worklist.removeFirst();
workset.remove(p);
HashSet<State> to = forward.get(p.s2);
HashSet<State> from = back.get(p.s1);
if (to != null) {
for (State s : to) {
StatePair pp = new StatePair(p.s1, s);
if (!pairs.contains(pp)) {
pairs.add(pp);
forward.get(p.s1).add(s);
back.get(s).add(p.s1);
worklist.add(pp);
workset.add(pp);
if (from != null) {
for (State q : from) {
StatePair qq = new StatePair(q, p.s1);
if (!workset.contains(qq)) {
worklist.add(qq);
workset.add(qq);
}
}
}
}
}
}
}
// add transitions
for (StatePair p : pairs)
p.s1.addEpsilon(p.s2);
a.deterministic = false;
//a.clearHashCode();
a.clearNumberedStates();
a.checkMinimizeAlways();
}
/**
* Returns true if the given automaton accepts the empty string and nothing
* else.
*/
public static boolean isEmptyString(Automaton a) {
if (a.isSingleton()) return a.singleton.length() == 0;
else return a.initial.accept && a.initial.numTransitions() == 0;
}
/**
* Returns true if the given automaton accepts no strings.
*/
public static boolean isEmpty(Automaton a) {
if (a.isSingleton()) return false;
return !a.initial.accept && a.initial.numTransitions() == 0;
}
/**
* Returns true if the given automaton accepts no strings.
*/
@ -1669,19 +948,6 @@ final public class BasicOperations {
return a.isAccept(0) == false && a.getNumTransitions(0) == 0;
}
/**
* Returns true if the given automaton accepts all strings.
*/
public static boolean isTotal(Automaton a) {
if (a.isSingleton()) return false;
if (a.initial.accept && a.initial.numTransitions() == 1) {
Transition t = a.initial.getTransitions().iterator().next();
return t.to == a.initial && t.min == Character.MIN_CODE_POINT
&& t.max == Character.MAX_CODE_POINT;
}
return false;
}
/**
* Returns true if the given automaton accepts all strings.
*/
@ -1695,59 +961,6 @@ final public class BasicOperations {
return false;
}
/**
* Returns true if the given string is accepted by the automaton.
* <p>
* Complexity: linear in the length of the string.
* <p>
* <b>Note:</b> for full performance, use the {@link RunAutomaton} class.
*/
public static boolean run(Automaton a, String s) {
if (a.isSingleton()) return s.equals(a.singleton);
if (a.deterministic) {
State p = a.initial;
for (int i = 0, cp = 0; i < s.length(); i += Character.charCount(cp)) {
State q = p.step(cp = s.codePointAt(i));
if (q == null) return false;
p = q;
}
return p.accept;
} else {
State[] states = a.getNumberedStates();
LinkedList<State> pp = new LinkedList<>();
LinkedList<State> pp_other = new LinkedList<>();
BitSet bb = new BitSet(states.length);
BitSet bb_other = new BitSet(states.length);
pp.add(a.initial);
ArrayList<State> dest = new ArrayList<>();
boolean accept = a.initial.accept;
for (int i = 0, c = 0; i < s.length(); i += Character.charCount(c)) {
c = s.codePointAt(i);
accept = false;
pp_other.clear();
bb_other.clear();
for (State p : pp) {
dest.clear();
p.step(c, dest);
for (State q : dest) {
if (q.accept) accept = true;
if (!bb_other.get(q.number)) {
bb_other.set(q.number);
pp_other.add(q);
}
}
}
LinkedList<State> tp = pp;
pp = pp_other;
pp_other = tp;
BitSet tb = bb;
bb = bb_other;
bb_other = tb;
}
return accept;
}
}
/**
* Returns true if the given string is accepted by the automaton. The input must be deterministic.
* <p>
@ -1769,6 +982,13 @@ final public class BasicOperations {
return a.isAccept(state);
}
/**
* Returns true if the given string (expressed as unicode codepoints) is accepted by the automaton. The input must be deterministic.
* <p>
* Complexity: linear in the length of the string.
* <p>
* <b>Note:</b> for full performance, use the {@link RunAutomaton} class.
*/
public static boolean run(LightAutomaton a, IntsRef s) {
// nocommit too slow?
assert isDeterministic(a);

View File

@ -21,20 +21,12 @@ package org.apache.lucene.util.automaton;
* Automaton representation for matching UTF-8 byte[].
*/
public class ByteRunAutomaton extends RunAutomaton {
public ByteRunAutomaton(Automaton a) {
this(a, false);
}
/** Converts incoming automaton to byte-based (UTF32ToUTF8) first */
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);

View File

@ -22,10 +22,7 @@ package org.apache.lucene.util.automaton;
*/
public class CharacterRunAutomaton extends RunAutomaton {
public CharacterRunAutomaton(Automaton a) {
super(a, Character.MAX_CODE_POINT, false);
}
/** Sole constructor. */
public CharacterRunAutomaton(LightAutomaton a) {
super(a, Character.MAX_CODE_POINT, false);
}

View File

@ -86,109 +86,11 @@ public class CompiledAutomaton {
*/
public final Boolean finite;
public CompiledAutomaton(Automaton automaton) {
this(automaton, null, true);
}
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
// 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 {
final String commonPrefix;
final String singleton;
if (automaton.getSingleton() == null) {
commonPrefix = SpecialOperations.getCommonPrefix(automaton);
if (commonPrefix.length() > 0 && BasicOperations.sameLanguage(automaton, BasicAutomata.makeString(commonPrefix))) {
singleton = commonPrefix;
} else {
singleton = null;
}
} else {
commonPrefix = null;
singleton = automaton.getSingleton();
}
if (singleton != null) {
// matches a fixed string in singleton or expanded
// representation
type = AUTOMATON_TYPE.SINGLE;
term = new BytesRef(singleton);
commonSuffixRef = null;
runAutomaton = null;
lightAutomaton = null;
this.finite = null;
return;
} else if (BasicOperations.sameLanguage(automaton, BasicOperations.concatenate(
BasicAutomata.makeString(commonPrefix), BasicAutomata.makeAnyString()))) {
// matches a constant prefix
type = AUTOMATON_TYPE.PREFIX;
term = new BytesRef(commonPrefix);
commonSuffixRef = null;
runAutomaton = null;
lightAutomaton = null;
this.finite = null;
return;
}
}
}
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
@ -224,8 +126,6 @@ public class CompiledAutomaton {
singleton = null;
}
//System.out.println("CHECK PREFIX: commonPrefix=" + commonPrefix);
if (singleton != null) {
// matches a fixed string
type = AUTOMATON_TYPE.SINGLE;

View File

@ -1,332 +0,0 @@
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 DaciukMihovAutomatonBuilder {
/**
* 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 org.apache.lucene.util.automaton.State convert(State s,
IdentityHashMap<State,org.apache.lucene.util.automaton.State> visited) {
org.apache.lucene.util.automaton.State converted = visited.get(s);
if (converted != null) return converted;
converted = new org.apache.lucene.util.automaton.State();
converted.setAccept(s.is_final);
visited.put(s, converted);
int i = 0;
int[] labels = s.labels;
for (DaciukMihovAutomatonBuilder.State target : s.states) {
converted.addTransition(
new Transition(labels[i++], convert(target, visited)));
}
return converted;
}
/**
* Build a minimal, deterministic automaton from a sorted list of {@link BytesRef} representing
* strings in UTF-8. These strings must be binary-sorted.
*/
public static Automaton build(Collection<BytesRef> input) {
final DaciukMihovAutomatonBuilder builder = new DaciukMihovAutomatonBuilder();
CharsRef scratch = new CharsRef();
for (BytesRef b : input) {
UnicodeUtil.UTF8toUTF16(b, scratch);
builder.add(scratch);
}
Automaton a = new Automaton();
a.initial = convert(
builder.complete(),
new IdentityHashMap<State,org.apache.lucene.util.automaton.State>());
a.deterministic = true;
return a;
}
/**
* Copy <code>current</code> into an internal buffer.
*/
private boolean setPrevious(CharsRef current) {
// don't need to copy, once we fix https://issues.apache.org/jira/browse/LUCENE-3277
// still, called only from assert
previous = 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

@ -112,72 +112,6 @@ public class LevenshteinAutomata {
}
return word;
}
/**
* 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 Automaton toAutomaton(int n) {
if (n == 0) {
return BasicAutomata.makeString(word, 0, word.length);
}
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
State states[] = new State[description.size()];
// create all states, and mark as accept states if appropriate
for (int i = 0; i < states.length; i++) {
states[i] = new State();
states[i].number = i;
states[i].setAccept(description.isAccept(i));
}
// create transitions from state to state
for (int k = 0; k < states.length; 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)
states[k].addTransition(new Transition(ch, states[dest]));
}
// 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++)
states[k].addTransition(new Transition(rangeLower[r], rangeUpper[r], states[dest]));
}
Automaton a = new Automaton(states[0]);
a.setDeterministic(true);
// we create some useless unconnected states, and its a net-win overall to remove these,
// as well as to combine any adjacent transitions (it makes later algorithms more efficient).
// so, while we could set our numberedStates here, its actually best not to, and instead to
// force a traversal in reduce, pruning the unconnected states while we combine adjacent transitions.
//a.setNumberedStates(states);
a.reduce();
// we need not trim transitions to dead states, as they are not created.
//a.restoreInvariant();
return a;
}
/**
* Compute a DFA that accepts all strings within an edit distance of <code>n</code>.
@ -192,7 +126,7 @@ public class LevenshteinAutomata {
*/
public LightAutomaton toLightAutomaton(int n) {
if (n == 0) {
return BasicAutomata.makeString(word, 0, word.length).toLightAutomaton();
return BasicAutomata.makeStringLight(word, 0, word.length);
}
if (n >= descriptions.length)

View File

@ -479,7 +479,7 @@ public class LightAutomaton {
t.max = transitions[i++];
}
private static void appendCharString(int c, StringBuilder b) {
static void appendCharString(int c, StringBuilder b) {
if (c >= 0x21 && c <= 0x7e && c != '\\' && c != '"') b.appendCodePoint(c);
else {
b.append("\\\\U");
@ -529,6 +529,8 @@ public class LightAutomaton {
return result;
}
// nocommit
/*
public void writeDot(String fileName) {
if (fileName.indexOf('/') == -1) {
fileName = "/l/la/lucene/core/" + fileName + ".dot";
@ -541,6 +543,7 @@ public class LightAutomaton {
throw new RuntimeException(ioe);
}
}
*/
public String toDot() {
// TODO: breadth first search so we can see get layered output...

View File

@ -1,260 +0,0 @@
/*
* 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 MinimizationOperations {
private MinimizationOperations() {}
/**
* Minimizes (and determinizes if not already deterministic) the given
* automaton.
*
* @see Automaton#setMinimization(int)
*/
public static void minimize(Automaton a) {
if (!a.isSingleton()) {
minimizeHopcroft(a);
}
// recompute hash code
//a.hash_code = 1a.getNumberOfStates() * 3 + a.getNumberOfTransitions() * 2;
//if (a.hash_code == 0) a.hash_code = 1;
}
/**
* Minimizes the given automaton using Hopcroft's algorithm.
*/
public static void minimizeHopcroft(Automaton a) {
a.determinize();
if (a.initial.numTransitions == 1) {
Transition t = a.initial.transitionsArray[0];
if (t.to == a.initial && t.min == Character.MIN_CODE_POINT
&& t.max == Character.MAX_CODE_POINT) return;
}
a.totalize();
// initialize data structures
final int[] sigma = a.getStartPoints();
final State[] states = a.getNumberedStates();
final int sigmaLen = sigma.length, statesLen = states.length;
@SuppressWarnings({"rawtypes","unchecked"}) final ArrayList<State>[][] reverse =
(ArrayList<State>[][]) new ArrayList[statesLen][sigmaLen];
@SuppressWarnings({"rawtypes","unchecked"}) final HashSet<State>[] partition =
(HashSet<State>[]) new HashSet[statesLen];
@SuppressWarnings({"rawtypes","unchecked"}) final ArrayList<State>[] splitblock =
(ArrayList<State>[]) 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 State qq = states[q];
final int j = qq.accept ? 0 : 1;
partition[j].add(qq);
block[q] = j;
for (int x = 0; x < sigmaLen; x++) {
final ArrayList<State>[] r =
reverse[qq.step(sigma[x]).number];
if (r[x] == null)
r[x] = new ArrayList<>();
r[x].add(qq);
}
}
// initialize active sets
for (int j = 0; j <= 1; j++) {
for (int x = 0; x < sigmaLen; x++) {
for (final State qq : partition[j]) {
if (reverse[qq.number][x] != null)
active2[qq.number][x] = active[j][x].add(qq);
}
}
}
// 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;
while (!pending.isEmpty()) {
final IntPair ip = pending.removeFirst();
final int p = ip.n1;
final int x = 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<State> r = reverse[m.q.number][x];
if (r != null) for (final State s : r) {
final int i = s.number;
if (!split.get(i)) {
split.set(i);
final int j = block[i];
splitblock[j].add(s);
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<State> sb = splitblock[j];
if (sb.size() < partition[j].size()) {
final HashSet<State> b1 = partition[j];
final HashSet<State> b2 = partition[k];
for (final State s : sb) {
b1.remove(s);
b2.add(s);
block[s.number] = k;
for (int c = 0; c < sigmaLen; c++) {
final StateListNode sn = active2[s.number][c];
if (sn != null && sn.sl == active[j][c]) {
sn.remove();
active2[s.number][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 (final State s : sb)
split.clear(s.number);
sb.clear();
}
refine.clear();
}
// make a new state for each equivalence class, set initial state
State[] newstates = new State[k];
for (int n = 0; n < newstates.length; n++) {
final State s = new State();
newstates[n] = s;
for (State q : partition[n]) {
if (q == a.initial) a.initial = s;
s.accept = q.accept;
s.number = q.number; // select representative
q.number = n;
}
}
// build transitions and set acceptance
for (int n = 0; n < newstates.length; n++) {
final State s = newstates[n];
s.accept = states[s.number].accept;
for (Transition t : states[s.number].getTransitions())
s.addTransition(new Transition(t.min, t.max, newstates[t.to.number]));
}
a.clearNumberedStates();
a.removeDeadTransitions();
}
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(State q) {
return new StateListNode(q, this);
}
}
static final class StateListNode {
final State q;
StateListNode next, prev;
final StateList sl;
StateListNode(State 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

@ -67,10 +67,10 @@ public abstract class RunAutomaton {
if (j + 1 < points.length) max = (points[j + 1] - 1);
else max = maxInterval;
b.append(" ");
Transition.appendCharString(min, b);
LightAutomaton.appendCharString(min, b);
if (min != max) {
b.append("-");
Transition.appendCharString(max, b);
LightAutomaton.appendCharString(max, b);
}
b.append(" -> ").append(k).append("\n");
}
@ -121,45 +121,6 @@ public abstract class RunAutomaton {
*
* @param a an automaton
*/
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];
transitions = new int[size * points.length];
for (int n = 0; n < size * points.length; n++)
transitions[n] = -1;
for (State s : states) {
int n = s.number;
accept[n] = s.accept;
for (int c = 0; c < points.length; c++) {
State q = s.step(points[c]);
if (q != null) transitions[n * points.length + c] = q.number;
}
}
/*
* 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;
}
} else {
classmap = null;
}
}
public RunAutomaton(LightAutomaton a, int maxInterval, boolean tableize) {
this.maxInterval = maxInterval;
//System.out.println("before det a=" + a.getNumStates());

View File

@ -1,278 +0,0 @@
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 SortedIntSet {
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;
State state;
public SortedIntSet(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 FrozenIntSet freeze(State state) {
final int[] c = new int[upto];
System.arraycopy(values, 0, c, 0, upto);
return new FrozenIntSet(c, hashCode, state);
}
@Override
public int hashCode() {
return hashCode;
}
@Override
public boolean equals(Object _other) {
if (_other == null) {
return false;
}
if (!(_other instanceof FrozenIntSet)) {
return false;
}
FrozenIntSet other = (FrozenIntSet) _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 FrozenIntSet {
final int[] values;
final int hashCode;
final State state;
public FrozenIntSet(int[] values, int hashCode, State state) {
this.values = values;
this.hashCode = hashCode;
this.state = state;
}
public FrozenIntSet(int num, State 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 FrozenIntSet) {
FrozenIntSet other = (FrozenIntSet) _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 SortedIntSet) {
SortedIntSet other = (SortedIntSet) _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

@ -67,29 +67,6 @@ final public class SpecialOperations {
return a;
}
/**
* Returns true if the language of this automaton is finite.
*/
public static boolean isFinite(Automaton a) {
if (a.isSingleton()) return true;
return isFinite(a.initial, new BitSet(a.getNumberOfStates()), new BitSet(a.getNumberOfStates()));
}
/**
* 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(State s, BitSet path, BitSet visited) {
path.set(s.number);
for (Transition t : s.getTransitions())
if (path.get(t.to.number) || (!visited.get(t.to.number) && !isFinite(t.to, path, visited))) return false;
path.clear(s.number);
visited.set(s.number);
return true;
}
/**
* Returns true if the language of this automaton is finite.
*/
@ -117,34 +94,6 @@ final public class SpecialOperations {
return true;
}
/**
* 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(Automaton a) {
if (a.isSingleton()) return a.singleton;
StringBuilder b = new StringBuilder();
HashSet<State> visited = new HashSet<>();
State s = a.initial;
boolean done;
do {
done = true;
visited.add(s);
if (!s.accept && s.numTransitions() == 1) {
Transition t = s.getTransitions().iterator().next();
if (t.min == t.max && !visited.contains(t.to)) {
b.appendCodePoint(t.min);
s = t.to;
done = false;
}
}
} while (!done);
return b.toString();
}
/**
* Returns the longest string that is a prefix of all accepted strings and
* visits each state at most once.
@ -175,31 +124,6 @@ final public class SpecialOperations {
return b.toString();
}
// TODO: this currently requites a determinized machine,
// but it need not -- we can speed it up by walking the
// NFA instead. it'd still be fail fast.
public static BytesRef getCommonPrefixBytesRef(Automaton a) {
if (a.isSingleton()) return new BytesRef(a.singleton);
BytesRef ref = new BytesRef(10);
HashSet<State> visited = new HashSet<>();
State s = a.initial;
boolean done;
do {
done = true;
visited.add(s);
if (!s.accept && s.numTransitions() == 1) {
Transition t = s.getTransitions().iterator().next();
if (t.min == t.max && !visited.contains(t.to)) {
ref.grow(++ref.length);
ref.bytes[ref.length - 1] = (byte)t.min;
s = t.to;
done = false;
}
}
} while (!done);
return ref;
}
public static BytesRef getCommonPrefixBytesRef(LightAutomaton a) {
BytesRef ref = new BytesRef(10);
HashSet<Integer> visited = new HashSet<>();
@ -223,36 +147,6 @@ final public class SpecialOperations {
return ref;
}
/**
* Returns the longest string that is a suffix of all accepted strings and
* visits each state at most once.
*
* @return common suffix
*/
public static String getCommonSuffix(Automaton a) {
if (a.isSingleton()) // if singleton, the suffix is the string itself.
return a.singleton;
// reverse the language of the automaton, then reverse its common prefix.
Automaton r = a.clone();
reverse(r);
r.determinize();
return new StringBuilder(SpecialOperations.getCommonPrefix(r)).reverse().toString();
}
public static BytesRef getCommonSuffixBytesRef(Automaton a) {
if (a.isSingleton()) // if singleton, the suffix is the string itself.
return new BytesRef(a.singleton);
// reverse the language of the automaton, then reverse its common prefix.
Automaton r = a.clone();
reverse(r);
r.determinize();
BytesRef ref = SpecialOperations.getCommonPrefixBytesRef(r);
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));
@ -271,40 +165,6 @@ final public class SpecialOperations {
}
}
/**
* Reverses the language of the given (non-singleton) automaton while returning
* the set of new initial states.
*/
public static Set<State> reverse(Automaton a) {
a.expandSingleton();
// reverse all edges
HashMap<State, HashSet<Transition>> m = new HashMap<>();
State[] states = a.getNumberedStates();
Set<State> accept = new HashSet<>();
for (State s : states)
if (s.isAccept())
accept.add(s);
for (State r : states) {
m.put(r, new HashSet<Transition>());
r.accept = false;
}
for (State r : states)
for (Transition t : r.getTransitions())
m.get(t.to).add(new Transition(t.min, t.max, r));
for (State r : states) {
Set<Transition> tr = m.get(r);
r.setTransitions(tr.toArray(new Transition[tr.size()]));
}
// make new initial+final states
a.initial.accept = true;
a.initial = new State();
for (State r : accept)
a.initial.addEpsilon(r); // ensures that all initial states are reachable
a.deterministic = false;
a.clearNumberedStates();
return accept;
}
// nocommit merge Special/Basic operations
public static LightAutomaton reverse(LightAutomaton a) {
@ -356,162 +216,6 @@ final public class SpecialOperations {
return result;
}
private static class PathNode {
/** Which state the path node ends on, whose
* transitions we are enumerating. */
public State state;
/** Which state the current transition leads to. */
public State 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;
public void resetState(State state) {
assert state.numTransitions() != 0;
this.state = state;
transition = 0;
Transition t = state.transitionsArray[transition];
label = t.min;
to = t.to;
}
/** 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() {
if (label > state.transitionsArray[transition].max) {
// We've exhaused the current transition's labels;
// move to next transitions:
transition++;
if (transition >= state.numTransitions()) {
// We're done iterating transitions leaving this state
return -1;
}
Transition t = state.transitionsArray[transition];
label = t.min;
to = t.to;
}
return label++;
}
}
private static PathNode getNode(PathNode[] nodes, int index) {
assert index < nodes.length;
if (nodes[index] == null) {
nodes[index] = new PathNode();
}
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(Automaton 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.isSingleton()) {
// Easy case: automaton accepts only 1 string
results.add(Util.toUTF32(a.singleton, new IntsRef()));
} else {
if (a.initial.accept) {
// Special case the empty string, as usual:
results.add(new IntsRef());
}
if (a.initial.numTransitions() > 0 && (limit == -1 || results.size() < limit)) {
// TODO: we could use state numbers here and just
// alloc array, but asking for states array can be
// costly (it's lazily computed):
// Tracks which states are in the current path, for
// cycle detection:
Set<State> pathStates = Collections.newSetFromMap(new IdentityHashMap<State,Boolean>());
// Stack to hold our current state in the
// recursion/iteration:
PathNode[] nodes = new PathNode[4];
pathStates.add(a.initial);
PathNode root = getNode(nodes, 0);
root.resetState(a.initial);
IntsRef string = new IntsRef(1);
string.length = 1;
while (string.length > 0) {
PathNode node = nodes[string.length-1];
// Get next label leaving the current node:
int label = node.nextLabel();
if (label != -1) {
string.ints[string.length-1] = label;
if (node.to.accept) {
// This transition leads to an accept state,
// so we save the current string:
results.add(IntsRef.deepCopyOf(string));
if (results.size() == limit) {
break;
}
}
if (node.to.numTransitions() != 0) {
// Now recurse: the destination of this transition has
// outgoing transitions:
if (pathStates.contains(node.to)) {
throw new IllegalArgumentException("automaton has cycles");
}
pathStates.add(node.to);
// Push node onto stack:
if (nodes.length == string.length) {
PathNode[] newNodes = new PathNode[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(node.to);
string.length++;
string.grow(string.length);
}
} else {
// No more transitions leaving this state,
// pop/return back to previous state:
assert pathStates.contains(node.state);
pathStates.remove(node.state);
string.length--;
}
}
}
}
return results;
}
private static class LightPathNode {
/** Which state the path node ends on, whose

View File

@ -1,280 +0,0 @@
/*
* 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 org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.RamUsageEstimator;
import java.util.Collection;
import java.util.Comparator;
import java.util.Iterator;
/**
* <tt>Automaton</tt> state.
*
* @lucene.experimental
*/
public class State implements Comparable<State> {
boolean accept;
public Transition[] transitionsArray;
public int numTransitions;
int number;
int id;
static int next_id;
/**
* Constructs a new state. Initially, the new state is a reject state.
*/
public State() {
resetTransitions();
id = next_id++;
}
/**
* Resets transition set.
*/
final void resetTransitions() {
transitionsArray = new Transition[0];
numTransitions = 0;
}
private class TransitionsIterable implements Iterable<Transition> {
@Override
public Iterator<Transition> iterator() {
return new Iterator<Transition>() {
int upto;
@Override
public boolean hasNext() {
return upto < numTransitions;
}
@Override
public Transition next() {
return transitionsArray[upto++];
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
};
}
}
/**
* Returns the set of outgoing transitions. Subsequent changes are reflected
* in the automaton.
*
* @return transition set
*/
public Iterable<Transition> getTransitions() {
return new TransitionsIterable();
}
public int numTransitions() {
return numTransitions;
}
public void setTransitions(Transition[] transitions) {
this.numTransitions = transitions.length;
this.transitionsArray = transitions;
}
/**
* Adds an outgoing transition.
*
* @param t transition
*/
public void addTransition(Transition t) {
if (numTransitions == transitionsArray.length) {
final Transition[] newArray = new Transition[ArrayUtil.oversize(1+numTransitions, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
System.arraycopy(transitionsArray, 0, newArray, 0, numTransitions);
transitionsArray = newArray;
}
transitionsArray[numTransitions++] = t;
}
/**
* Sets acceptance for this state.
*
* @param accept if true, this state is an accept state
*/
public void setAccept(boolean accept) {
this.accept = accept;
}
/**
* Returns acceptance status.
*
* @return true is this is an accept state
*/
public boolean isAccept() {
return accept;
}
/**
* Performs lookup in transitions, assuming determinism.
*
* @param c codepoint to look up
* @return destination state, null if no matching outgoing transition
* @see #step(int, Collection)
*/
public State step(int c) {
assert c >= 0;
for (int i=0;i<numTransitions;i++) {
final Transition t = transitionsArray[i];
if (t.min <= c && c <= t.max) return t.to;
}
return null;
}
/**
* Performs lookup in transitions, allowing nondeterminism.
*
* @param c codepoint to look up
* @param dest collection where destination states are stored
* @see #step(int)
*/
public void step(int c, Collection<State> dest) {
for (int i=0;i<numTransitions;i++) {
final Transition t = transitionsArray[i];
if (t.min <= c && c <= t.max) dest.add(t.to);
}
}
/** Virtually adds an epsilon transition to the target
* {@code to} state. This is implemented by copying all
* transitions from {@code to} to this state, and if {@code
* to} is an accept state then set accept for this state. */
void addEpsilon(State to) {
if (to.accept) accept = true;
for (Transition t : to.getTransitions())
addTransition(t);
}
/** Downsizes transitionArray to numTransitions */
public void trimTransitionsArray() {
if (numTransitions < transitionsArray.length) {
final Transition[] newArray = new Transition[numTransitions];
System.arraycopy(transitionsArray, 0, newArray, 0, numTransitions);
transitionsArray = newArray;
}
}
/**
* Reduces this state. A state is "reduced" by combining overlapping
* and adjacent edge intervals with same destination.
*/
public void reduce() {
if (numTransitions <= 1) {
return;
}
sortTransitions(Transition.CompareByDestThenMinMax);
State p = null;
int min = -1, max = -1;
int upto = 0;
for (int i=0;i<numTransitions;i++) {
final Transition t = transitionsArray[i];
if (p == t.to) {
if (t.min <= max + 1) {
if (t.max > max) max = t.max;
} else {
if (p != null) {
transitionsArray[upto++] = new Transition(min, max, p);
}
min = t.min;
max = t.max;
}
} else {
if (p != null) {
transitionsArray[upto++] = new Transition(min, max, p);
}
p = t.to;
min = t.min;
max = t.max;
}
}
if (p != null) {
transitionsArray[upto++] = new Transition(min, max, p);
}
numTransitions = upto;
}
/**
* Returns sorted list of outgoing transitions.
*
* @param to_first if true, order by (to, min, reverse max); otherwise (min,
* reverse max, to)
* @return transition list
*/
/** Sorts transitions array in-place. */
public void sortTransitions(Comparator<Transition> comparator) {
// mergesort seems to perform better on already sorted arrays:
if (numTransitions > 1) ArrayUtil.timSort(transitionsArray, 0, numTransitions, comparator);
}
/**
* Return this state's number.
* <p>
* Expert: Will be useless unless {@link Automaton#getNumberedStates}
* has been called first to number the states.
* @return the number
*/
public int getNumber() {
return number;
}
/**
* Returns string describing this state. Normally invoked via
* {@link Automaton#toString()}.
*/
@Override
public String toString() {
StringBuilder b = new StringBuilder();
b.append("state ").append(number);
if (accept) b.append(" [accept]");
else b.append(" [reject]");
b.append(":\n");
for (Transition t : getTransitions())
b.append(" ").append(t.toString()).append("\n");
return b.toString();
}
/**
* Compares this object with the specified object for order. States are
* ordered by the time of construction.
*/
@Override
public int compareTo(State s) {
return s.id - id;
}
}

View File

@ -1,101 +0,0 @@
/*
* 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 StatePair {
State s;
State s1;
State s2;
StatePair(State s, State s1, State s2) {
this.s = s;
this.s1 = s1;
this.s2 = s2;
}
/**
* Constructs a new state pair.
*
* @param s1 first state
* @param s2 second state
*/
public StatePair(State s1, State s2) {
this.s1 = s1;
this.s2 = s2;
}
/**
* Returns first component of this pair.
*
* @return first state
*/
public State getFirstState() {
return s1;
}
/**
* Returns second component of this pair.
*
* @return second state
*/
public State getSecondState() {
return s2;
}
/**
* 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 StatePair) {
StatePair p = (StatePair) obj;
return p.s1 == s1 && p.s2 == s2;
} else return false;
}
/**
* Returns hash code.
*
* @return hash code
*/
@Override
public int hashCode() {
return s1.hashCode() + s2.hashCode();
}
}

View File

@ -1,214 +0,0 @@
/*
* 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.Comparator;
/**
* <tt>Automaton</tt> transition.
* <p>
* A transition, which belongs to a source state, consists of a Unicode
* codepoint interval and a destination state.
*
* @lucene.experimental
*/
public class Transition implements Cloneable {
/*
* CLASS INVARIANT: min<=max
*/
final int min;
final int max;
final State to;
/**
* Constructs a new singleton interval transition.
*
* @param c transition codepoint
* @param to destination state
*/
public Transition(int c, State to) {
assert c >= 0;
min = max = c;
this.to = to;
}
/**
* Constructs a new transition. Both end points are included in the interval.
*
* @param min transition interval minimum
* @param max transition interval maximum
* @param to destination state
*/
public Transition(int min, int max, State to) {
assert min >= 0;
assert max >= 0;
if (max < min) {
int t = max;
max = min;
min = t;
}
this.min = min;
this.max = max;
this.to = to;
}
/** Returns minimum of this transition interval. */
public int getMin() {
return min;
}
/** Returns maximum of this transition interval. */
public int getMax() {
return max;
}
/** Returns destination of this transition. */
public State getDest() {
return to;
}
/**
* Checks for equality.
*
* @param obj object to compare with
* @return true if <tt>obj</tt> is a transition with same character interval
* and destination state as this transition.
*/
@Override
public boolean equals(Object obj) {
if (obj instanceof Transition) {
Transition t = (Transition) obj;
return t.min == min && t.max == max && t.to == to;
} else return false;
}
/**
* Returns hash code. The hash code is based on the character interval (not
* the destination state).
*
* @return hash code
*/
@Override
public int hashCode() {
return min * 2 + max * 3;
}
/**
* Clones this transition.
*
* @return clone with same character interval and destination state
*/
@Override
public Transition clone() {
try {
return (Transition) super.clone();
} catch (CloneNotSupportedException e) {
throw new RuntimeException(e);
}
}
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);
}
}
/**
* Returns a string describing this state. Normally invoked via
* {@link Automaton#toString()}.
*/
@Override
public String toString() {
StringBuilder b = new StringBuilder();
appendCharString(min, b);
if (min != max) {
b.append("-");
appendCharString(max, b);
}
b.append(" -> ").append(to.number);
return b.toString();
}
void appendDot(StringBuilder b) {
b.append(" -> ").append(to.number).append(" [label=\"");
appendCharString(min, b);
if (min != max) {
b.append("-");
appendCharString(max, b);
}
b.append("\"]\n");
}
private static final class CompareByDestThenMinMaxSingle implements Comparator<Transition> {
@Override
public int compare(Transition t1, Transition t2) {
if (t1.to != t2.to) {
if (t1.to.number < t2.to.number) return -1;
else if (t1.to.number > t2.to.number) return 1;
}
if (t1.min < t2.min) return -1;
if (t1.min > t2.min) return 1;
if (t1.max > t2.max) return -1;
if (t1.max < t2.max) return 1;
return 0;
}
}
public static final Comparator<Transition> CompareByDestThenMinMax = new CompareByDestThenMinMaxSingle();
private static final class CompareByMinMaxThenDestSingle implements Comparator<Transition> {
@Override
public int compare(Transition t1, Transition t2) {
if (t1.min < t2.min) return -1;
if (t1.min > t2.min) return 1;
if (t1.max > t2.max) return -1;
if (t1.max < t2.max) return 1;
if (t1.to != t2.to) {
if (t1.to.number < t2.to.number) return -1;
if (t1.to.number > t2.to.number) return 1;
}
return 0;
}
}
public static final Comparator<Transition> CompareByMinMaxThenDest = new CompareByMinMaxThenDestSingle();
}

View File

@ -1,316 +0,0 @@
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 java.util.List;
import java.util.ArrayList;
// 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 UTF32ToUTF8 {
// 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(State start, State 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(State start, State 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:
start.addTransition(new Transition(startUTF8.byteAt(upto), endUTF8.byteAt(upto), end));
return;
} else {
assert startUTF8.len > upto+1;
assert endUTF8.len > upto+1;
State n = newUTF8State();
// Single value leading edge
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
} 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(State start, State end, UTF8Sequence utf8, int upto, boolean doAll) {
if (upto == utf8.len-1) {
// Done recursing
start.addTransition(new Transition(utf8.byteAt(upto), utf8.byteAt(upto) | MASKS[utf8.numBits(upto)-1], end)); // type=start
} else {
State n = newUTF8State();
start.addTransition(new Transition(utf8.byteAt(upto), n)); // type=start
start(n, end, utf8, 1+upto, true);
int endCode = utf8.byteAt(upto) | MASKS[utf8.numBits(upto)-1];
if (doAll && utf8.byteAt(upto) != endCode) {
all(start, end, utf8.byteAt(upto)+1, endCode, utf8.len-upto-1);
}
}
}
private void end(State start, State end, UTF8Sequence utf8, int upto, boolean doAll) {
if (upto == utf8.len-1) {
// Done recursing
start.addTransition(new Transition(utf8.byteAt(upto) & (~MASKS[utf8.numBits(upto)-1]), utf8.byteAt(upto), end)); // type=end
} else {
final int startCode;
if (utf8.numBits(upto) == 5) {
// special case -- avoid created unused edges (utf8
// doesn't accept certain byte sequences) -- there
// are other cases we could optimize too:
startCode = 194;
} else {
startCode = utf8.byteAt(upto) & (~MASKS[utf8.numBits(upto)-1]);
}
if (doAll && utf8.byteAt(upto) != startCode) {
all(start, end, startCode, utf8.byteAt(upto)-1, utf8.len-upto-1);
}
State n = newUTF8State();
start.addTransition(new Transition(utf8.byteAt(upto), n)); // type=end
end(n, end, utf8, 1+upto, true);
}
}
private void all(State start, State end, int startCode, int endCode, int left) {
if (left == 0) {
start.addTransition(new Transition(startCode, endCode, end)); // type=all
} else {
State lastN = newUTF8State();
start.addTransition(new Transition(startCode, endCode, lastN)); // type=all
while (left > 1) {
State n = newUTF8State();
lastN.addTransition(new Transition(128, 191, n)); // type=all*
left--;
lastN = n;
}
lastN.addTransition(new Transition(128, 191, end)); // type = all*
}
}
private State[] utf8States;
private int utf8StateCount;
/** 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 Automaton convert(Automaton utf32) {
if (utf32.isSingleton()) {
utf32 = utf32.cloneExpanded();
}
State[] map = new State[utf32.getNumberedStates().length];
List<State> pending = new ArrayList<>();
State utf32State = utf32.getInitialState();
pending.add(utf32State);
Automaton utf8 = new Automaton();
utf8.setDeterministic(false);
State utf8State = utf8.getInitialState();
utf8States = new State[5];
utf8StateCount = 0;
utf8State.number = utf8StateCount;
utf8States[utf8StateCount] = utf8State;
utf8StateCount++;
utf8State.setAccept(utf32State.isAccept());
map[utf32State.number] = utf8State;
while(pending.size() != 0) {
utf32State = pending.remove(pending.size()-1);
utf8State = map[utf32State.number];
for(int i=0;i<utf32State.numTransitions;i++) {
final Transition t = utf32State.transitionsArray[i];
final State destUTF32 = t.to;
State destUTF8 = map[destUTF32.number];
if (destUTF8 == null) {
destUTF8 = newUTF8State();
destUTF8.accept = destUTF32.accept;
map[destUTF32.number] = destUTF8;
pending.add(destUTF32);
}
convertOneEdge(utf8State, destUTF8, t.min, t.max);
}
}
utf8.setNumberedStates(utf8States, utf8StateCount);
return utf8;
}
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

@ -29,7 +29,6 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.BasicAutomata;
import org.apache.lucene.util.automaton.BasicOperations;
import org.apache.lucene.util.automaton.LightAutomaton;

View File

@ -34,7 +34,6 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.AutomatonTestUtil;
import org.apache.lucene.util.automaton.BasicAutomata;
import org.apache.lucene.util.automaton.BasicOperations;
@ -165,9 +164,9 @@ public class TestMockAnalyzer extends BaseTokenStreamTestCase {
public void testKeep() throws Exception {
CharacterRunAutomaton keepWords =
new CharacterRunAutomaton(
BasicOperations.complement(
Automaton.union(
Arrays.asList(BasicAutomata.makeString("foo"), BasicAutomata.makeString("bar")))));
BasicOperations.complementLight(
BasicOperations.unionLight(
Arrays.asList(BasicAutomata.makeStringLight("foo"), BasicAutomata.makeStringLight("bar")))));
Analyzer a = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true, keepWords);
assertAnalyzesTo(a, "quick foo brown bar bar fox foo",
new String[] { "foo", "bar", "bar", "foo" },

View File

@ -83,9 +83,9 @@ import org.apache.lucene.util.SetOnce;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.ThreadInterruptedException;
import org.apache.lucene.util.Version;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.BasicAutomata;
import org.apache.lucene.util.automaton.CharacterRunAutomaton;
import org.apache.lucene.util.automaton.LightAutomaton;
import org.apache.lucene.util.packed.PackedInts;
import org.junit.Test;
@ -1997,7 +1997,7 @@ public class TestIndexWriter extends LuceneTestCase {
public void testStopwordsPosIncHole2() throws Exception {
// use two stopfilters for testing here
Directory dir = newDirectory();
final Automaton secondSet = BasicAutomata.makeString("foobar");
final LightAutomaton secondSet = BasicAutomata.makeStringLight("foobar");
Analyzer a = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName) {

View File

@ -33,7 +33,6 @@ import org.apache.lucene.util.LineFileDocs;
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.BasicAutomata;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.LightAutomaton;
@ -240,12 +239,12 @@ public class TestTermsEnum extends LuceneTestCase {
final Set<String> acceptTerms = new HashSet<>();
final TreeSet<BytesRef> sortedAcceptTerms = new TreeSet<>();
final double keepPct = random().nextDouble();
Automaton a;
LightAutomaton a;
if (iter == 0) {
if (VERBOSE) {
System.out.println("\nTEST: empty automaton");
}
a = BasicAutomata.makeEmpty();
a = BasicAutomata.makeEmptyLight();
} else {
if (VERBOSE) {
System.out.println("\nTEST: keepPct=" + keepPct);
@ -260,16 +259,9 @@ public class TestTermsEnum extends LuceneTestCase {
acceptTerms.add(s2);
sortedAcceptTerms.add(new BytesRef(s2));
}
a = BasicAutomata.makeStringUnion(sortedAcceptTerms);
a = BasicAutomata.makeStringUnionLight(sortedAcceptTerms);
}
if (random().nextBoolean()) {
if (VERBOSE) {
System.out.println("TEST: reduce the automaton");
}
a.reduce();
}
final CompiledAutomaton c = new CompiledAutomaton(a, true, false);
final BytesRef[] acceptTermsArray = new BytesRef[acceptTerms.size()];

View File

@ -95,13 +95,11 @@ public class TestTermsEnum2 extends LuceneTestCase {
}
}
Automaton alternate = BasicAutomata.makeStringUnion(matchedTerms);
LightAutomaton alternate = BasicAutomata.makeStringUnionLight(matchedTerms);
//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);
ScoreDoc[] origHits = searcher.search(a1, 25).scoreDocs;

View File

@ -33,10 +33,10 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.Rethrow;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.AutomatonTestUtil;
import org.apache.lucene.util.automaton.BasicAutomata;
import org.apache.lucene.util.automaton.BasicOperations;
import org.apache.lucene.util.automaton.LightAutomaton;
public class TestAutomatonQuery extends LuceneTestCase {
private Directory directory;
@ -86,7 +86,7 @@ public class TestAutomatonQuery 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);
@ -107,21 +107,25 @@ public class TestAutomatonQuery extends LuceneTestCase {
* Test some very simple automata.
*/
public void testBasicAutomata() throws IOException {
assertAutomatonHits(0, BasicAutomata.makeEmpty());
assertAutomatonHits(0, BasicAutomata.makeEmptyString());
assertAutomatonHits(2, BasicAutomata.makeAnyChar());
assertAutomatonHits(3, BasicAutomata.makeAnyString());
assertAutomatonHits(2, BasicAutomata.makeString("doc"));
assertAutomatonHits(1, BasicAutomata.makeChar('a'));
assertAutomatonHits(2, BasicAutomata.makeCharRange('a', 'b'));
assertAutomatonHits(2, BasicAutomata.makeInterval(1233, 2346, 0));
assertAutomatonHits(1, BasicAutomata.makeInterval(0, 2000, 0));
assertAutomatonHits(2, BasicOperations.union(BasicAutomata.makeChar('a'),
BasicAutomata.makeChar('b')));
assertAutomatonHits(0, BasicOperations.intersection(BasicAutomata
.makeChar('a'), BasicAutomata.makeChar('b')));
assertAutomatonHits(1, BasicOperations.minus(BasicAutomata.makeCharRange('a', 'b'),
BasicAutomata.makeChar('a')));
// nocommit
assertAutomatonHits(2, BasicAutomata.makeIntervalLight(1233, 2346, 0));
assertAutomatonHits(0, BasicAutomata.makeEmptyLight());
assertAutomatonHits(0, BasicAutomata.makeEmptyStringLight());
assertAutomatonHits(2, BasicAutomata.makeAnyCharLight());
assertAutomatonHits(3, BasicAutomata.makeAnyStringLight());
assertAutomatonHits(2, BasicAutomata.makeStringLight("doc"));
assertAutomatonHits(1, BasicAutomata.makeCharLight('a'));
assertAutomatonHits(2, BasicAutomata.makeCharRangeLight('a', 'b'));
assertAutomatonHits(2, BasicAutomata.makeIntervalLight(1233, 2346, 0));
assertAutomatonHits(1, BasicAutomata.makeIntervalLight(0, 2000, 0));
assertAutomatonHits(2, BasicOperations.unionLight(BasicAutomata.makeCharLight('a'),
BasicAutomata.makeCharLight('b')));
assertAutomatonHits(0, BasicOperations.intersectionLight(BasicAutomata
.makeCharLight('a'), BasicAutomata.makeCharLight('b')));
assertAutomatonHits(1, BasicOperations.minusLight(BasicAutomata.makeCharRangeLight('a', 'b'),
BasicAutomata.makeCharLight('a')));
}
/**
@ -131,26 +135,27 @@ public class TestAutomatonQuery extends LuceneTestCase {
public void testNFA() throws IOException {
// accept this or three, the union is an NFA (two transitions for 't' from
// initial state)
Automaton nfa = BasicOperations.union(BasicAutomata.makeString("this"),
BasicAutomata.makeString("three"));
LightAutomaton nfa = BasicOperations.unionLight(BasicAutomata.makeStringLight("this"),
BasicAutomata.makeStringLight("three"));
assertAutomatonHits(2, nfa);
}
public void testEquals() {
AutomatonQuery a1 = new AutomatonQuery(newTerm("foobar"), BasicAutomata
.makeString("foobar"));
.makeStringLight("foobar"));
// reference to a1
AutomatonQuery a2 = a1;
// same as a1 (accepts the same language, same term)
AutomatonQuery a3 = new AutomatonQuery(newTerm("foobar"), BasicOperations
.concatenate(BasicAutomata.makeString("foo"), BasicAutomata
.makeString("bar")));
AutomatonQuery a3 = new AutomatonQuery(newTerm("foobar"),
BasicOperations.concatenateLight(
BasicAutomata.makeStringLight("foo"),
BasicAutomata.makeStringLight("bar")));
// different than a1 (same term, but different language)
AutomatonQuery a4 = new AutomatonQuery(newTerm("foobar"), BasicAutomata
.makeString("different"));
AutomatonQuery a4 = new AutomatonQuery(newTerm("foobar"),
BasicAutomata.makeStringLight("different"));
// different than a1 (different term, same language)
AutomatonQuery a5 = new AutomatonQuery(newTerm("blah"), BasicAutomata
.makeString("foobar"));
AutomatonQuery a5 = new AutomatonQuery(newTerm("blah"),
BasicAutomata.makeStringLight("foobar"));
assertEquals(a1.hashCode(), a2.hashCode());
assertEquals(a1, a2);
@ -176,8 +181,7 @@ public class TestAutomatonQuery extends LuceneTestCase {
* MultiTermQuery semantics.
*/
public void testRewriteSingleTerm() throws IOException {
AutomatonQuery aq = new AutomatonQuery(newTerm("bogus"), BasicAutomata
.makeString("piece"));
AutomatonQuery aq = new AutomatonQuery(newTerm("bogus"), BasicAutomata.makeStringLight("piece"));
Terms terms = MultiFields.getTerms(searcher.getIndexReader(), FN);
assertTrue(aq.getTermsEnum(terms) instanceof SingleTermsEnum);
assertEquals(1, automatonQueryNrHits(aq));
@ -188,10 +192,8 @@ public class TestAutomatonQuery extends LuceneTestCase {
* MultiTermQuery semantics.
*/
public void testRewritePrefix() throws IOException {
Automaton pfx = BasicAutomata.makeString("do");
pfx.expandSingleton(); // expand singleton representation for testing
Automaton prefixAutomaton = BasicOperations.concatenate(pfx, BasicAutomata
.makeAnyString());
LightAutomaton pfx = BasicAutomata.makeStringLight("do");
LightAutomaton prefixAutomaton = BasicOperations.concatenateLight(pfx, BasicAutomata.makeAnyStringLight());
AutomatonQuery aq = new AutomatonQuery(newTerm("bogus"), prefixAutomaton);
Terms terms = MultiFields.getTerms(searcher.getIndexReader(), FN);
assertTrue(aq.getTermsEnum(terms) instanceof PrefixTermsEnum);
@ -202,8 +204,7 @@ public class TestAutomatonQuery extends LuceneTestCase {
* Test handling of the empty language
*/
public void testEmptyOptimization() throws IOException {
AutomatonQuery aq = new AutomatonQuery(newTerm("bogus"), BasicAutomata
.makeEmpty());
AutomatonQuery aq = new AutomatonQuery(newTerm("bogus"), BasicAutomata.makeEmptyLight());
// not yet available: assertTrue(aq.getEnum(searcher.getIndexReader())
// instanceof EmptyTermEnum);
Terms terms = MultiFields.getTerms(searcher.getIndexReader(), FN);

View File

@ -26,7 +26,6 @@ import org.apache.lucene.index.RandomIndexWriter;
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;

View File

@ -27,7 +27,6 @@ import org.apache.lucene.index.RandomIndexWriter;
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.BasicAutomata;
import org.apache.lucene.util.automaton.BasicOperations;
import org.apache.lucene.util.automaton.LightAutomaton;

View File

@ -40,7 +40,6 @@ import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
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;

View File

@ -36,7 +36,7 @@ public class TestCompiledAutomaton extends LuceneTestCase {
terms.add(new BytesRef(s));
}
Collections.sort(terms);
final Automaton a = DaciukMihovAutomatonBuilder.build(terms);
final LightAutomaton a = DaciukMihovAutomatonBuilderLight.build(terms);
return new CompiledAutomaton(a, true, false);
}
@ -109,8 +109,7 @@ public class TestCompiledAutomaton extends LuceneTestCase {
public void testBasic() throws Exception {
CompiledAutomaton c = build("fob", "foo", "goo");
// nocommit
//testFloor(c, "goo", "goo");
testFloor(c, "goo", "goo");
testFloor(c, "ga", "foo");
testFloor(c, "g", "foo");
testFloor(c, "foc", "fob");

View File

@ -30,7 +30,7 @@ import org.apache.lucene.util.TestUtil;
* somewhat randomly, by determinizing a huge random lexicon.
*/
public class TestDeterminizeLexicon extends LuceneTestCase {
private List<Automaton> automata = new ArrayList<>();
private List<LightAutomaton> automata = new ArrayList<>();
private List<String> terms = new ArrayList<>();
public void testLexicon() throws Exception {
@ -41,7 +41,7 @@ public class TestDeterminizeLexicon extends LuceneTestCase {
for (int j = 0; j < 5000; j++) {
String randomString = TestUtil.randomUnicodeString(random());
terms.add(randomString);
automata.add(BasicAutomata.makeString(randomString));
automata.add(BasicAutomata.makeStringLight(randomString));
}
assertLexicon();
}
@ -49,8 +49,8 @@ public class TestDeterminizeLexicon extends LuceneTestCase {
public void assertLexicon() throws Exception {
Collections.shuffle(automata, random());
final Automaton lex = BasicOperations.union(automata);
lex.determinize();
LightAutomaton lex = BasicOperations.unionLight(automata);
lex = BasicOperations.determinize(lex);
assertTrue(SpecialOperations.isFinite(lex));
for (String s : terms) {
assertTrue(BasicOperations.run(lex, s));

View File

@ -22,6 +22,8 @@ import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.WildcardQuery;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.automaton.AutomatonTestUtil.RandomAcceptedStringsLight;
@ -337,4 +339,44 @@ public class TestLightAutomaton extends LuceneTestCase {
}
}
// nocommit testMinus
public void testMinus() throws Exception {
LightAutomaton a1 = BasicAutomata.makeStringLight("foobar");
LightAutomaton a2 = BasicAutomata.makeStringLight("boobar");
LightAutomaton a3 = BasicAutomata.makeStringLight("beebar");
LightAutomaton a = BasicOperations.unionLight(Arrays.asList(a1, a2, a3));
if (random().nextBoolean()) {
a = BasicOperations.determinize(a);
} else if (random().nextBoolean()) {
a = MinimizationOperationsLight.minimize(a);
}
LightAutomaton a4 = BasicOperations.determinize(BasicOperations.minusLight(a, a2));
assertTrue(BasicOperations.run(a4, "foobar"));
assertFalse(BasicOperations.run(a4, "boobar"));
assertTrue(BasicOperations.run(a4, "beebar"));
// nocommit test getFinitStrings count == 2
a4 = BasicOperations.determinize(BasicOperations.minusLight(a4, a1));
assertFalse(BasicOperations.run(a4, "foobar"));
assertFalse(BasicOperations.run(a4, "boobar"));
assertTrue(BasicOperations.run(a4, "beebar"));
a4 = BasicOperations.determinize(BasicOperations.minusLight(a4, a3));
assertFalse(BasicOperations.run(a4, "foobar"));
assertFalse(BasicOperations.run(a4, "boobar"));
assertFalse(BasicOperations.run(a4, "beebar"));
}
// nocommit
//public void testWildcard() throws Exception {
//WildcardQuery.toAutomaton(new Term("foo", "bar*")).writeDot("wq");
//}
// nocommit more tests ... it's an algebra
// nocommit random test for testInterval if we don't have one already
}

View File

@ -199,7 +199,7 @@ public class TestSpecialOperations extends LuceneTestCase {
}
public void testSingletonNoLimit() {
Set<IntsRef> result = SpecialOperations.getFiniteStrings(BasicAutomata.makeString("foobar"), -1);
Set<IntsRef> result = SpecialOperations.getFiniteStrings(BasicAutomata.makeStringLight("foobar"), -1);
assertEquals(1, result.size());
IntsRef scratch = new IntsRef();
Util.toUTF32("foobar".toCharArray(), 0, 6, scratch);
@ -207,7 +207,7 @@ public class TestSpecialOperations extends LuceneTestCase {
}
public void testSingletonLimit1() {
Set<IntsRef> result = SpecialOperations.getFiniteStrings(BasicAutomata.makeString("foobar"), 1);
Set<IntsRef> result = SpecialOperations.getFiniteStrings(BasicAutomata.makeStringLight("foobar"), 1);
assertEquals(1, result.size());
IntsRef scratch = new IntsRef();
Util.toUTF32("foobar".toCharArray(), 0, 6, scratch);

View File

@ -151,12 +151,7 @@ public class TestUTF32ToUTF8 extends LuceneTestCase {
continue;
}
final Automaton a = new Automaton();
final State end = new State();
end.setAccept(true);
a.getInitialState().addTransition(new Transition(startCode, endCode, end));
a.setDeterministic(true);
LightAutomaton a = BasicAutomata.makeCharRangeLight(startCode, endCode);
testOne(r, new ByteRunAutomaton(a), startCode, endCode, ITERS_PER_DFA);
}
}

View File

@ -68,7 +68,6 @@ 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;

View File

@ -106,8 +106,8 @@ class MultiTermHighlighting {
final PrefixQuery pq = (PrefixQuery) query;
Term prefix = pq.getPrefix();
if (prefix.field().equals(field)) {
list.add(new CharacterRunAutomaton(BasicOperations.concatenate(BasicAutomata.makeString(prefix.text()),
BasicAutomata.makeAnyString())) {
list.add(new CharacterRunAutomaton(BasicOperations.concatenateLight(BasicAutomata.makeStringLight(prefix.text()),
BasicAutomata.makeAnyStringLight())) {
@Override
public String toString() {
return pq.toString();

View File

@ -1340,7 +1340,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
@Override
public void run() throws Exception {
String goodWord = "goodtoken";
CharacterRunAutomaton stopWords = new CharacterRunAutomaton(BasicAutomata.makeString("stoppedtoken"));
CharacterRunAutomaton stopWords = new CharacterRunAutomaton(BasicAutomata.makeStringLight("stoppedtoken"));
// we disable MockTokenizer checks because we will forcefully limit the
// tokenstream and call end() before incrementToken() returns false.
final MockAnalyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true, stopWords);

View File

@ -557,7 +557,7 @@ public class TestPrecedenceQueryParser extends LuceneTestCase {
}
public void testBoost() throws Exception {
CharacterRunAutomaton stopSet = new CharacterRunAutomaton(BasicAutomata.makeString("on"));
CharacterRunAutomaton stopSet = new CharacterRunAutomaton(BasicAutomata.makeStringLight("on"));
Analyzer oneStopAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true, stopSet);
PrecedenceQueryParser qp = new PrecedenceQueryParser();

View File

@ -957,7 +957,7 @@ public class TestQPHelper extends LuceneTestCase {
}
public void testBoost() throws Exception {
CharacterRunAutomaton stopSet = new CharacterRunAutomaton(BasicAutomata.makeString("on"));
CharacterRunAutomaton stopSet = new CharacterRunAutomaton(BasicAutomata.makeStringLight("on"));
Analyzer oneStopAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true, stopSet);
StandardQueryParser qp = new StandardQueryParser();
qp.setAnalyzer(oneStopAnalyzer);

View File

@ -868,7 +868,7 @@ public abstract class QueryParserTestBase extends LuceneTestCase {
public void testBoost()
throws Exception {
CharacterRunAutomaton stopWords = new CharacterRunAutomaton(BasicAutomata.makeString("on"));
CharacterRunAutomaton stopWords = new CharacterRunAutomaton(BasicAutomata.makeStringLight("on"));
Analyzer oneStopAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true, stopWords);
CommonQueryParserConfiguration qp = getParserConfig(oneStopAnalyzer);
Query q = getQuery("on^1.0",qp);

View File

@ -50,7 +50,6 @@ import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.RunAutomaton;
import org.apache.lucene.util.automaton.Transition;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.Outputs;

View File

@ -43,12 +43,9 @@ import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.OfflineSorter;
import org.apache.lucene.util.UnicodeUtil;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.BasicOperations;
import org.apache.lucene.util.automaton.LightAutomaton;
import org.apache.lucene.util.automaton.SpecialOperations;
import org.apache.lucene.util.automaton.State;
import org.apache.lucene.util.automaton.Transition;
import org.apache.lucene.util.fst.Builder;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.FST.BytesReader;
@ -302,7 +299,6 @@ public class AnalyzingSuggester extends Lookup {
int[] topoSortStates = topoSortStates(a);
for(int i=0;i<topoSortStates.length;i++) {
int state = topoSortStates[topoSortStates.length-1-i];
List<Transition> newTransitions = new ArrayList<>();
int count = a.initTransition(state, t);
for(int j=0;j<count;j++) {
a.getNextTransition(t);

View File

@ -22,11 +22,8 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.BasicOperations;
import org.apache.lucene.util.automaton.LightAutomaton;
import org.apache.lucene.util.automaton.State;
import org.apache.lucene.util.automaton.Transition;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.Util;

View File

@ -28,13 +28,11 @@ import org.apache.lucene.analysis.TokenStreamToAutomaton;
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute; // javadocs
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.BasicAutomata;
import org.apache.lucene.util.automaton.BasicOperations;
import org.apache.lucene.util.automaton.LevenshteinAutomata;
import org.apache.lucene.util.automaton.LightAutomaton;
import org.apache.lucene.util.automaton.SpecialOperations;
import org.apache.lucene.util.automaton.UTF32ToUTF8;
import org.apache.lucene.util.automaton.UTF32ToUTF8Light;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.PairOutputs.Pair;

View File

@ -47,10 +47,8 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
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.BasicOperations;
import org.apache.lucene.util.automaton.LightAutomaton;
import org.apache.lucene.util.automaton.State;
import org.apache.lucene.util.fst.Util;
public class FuzzySuggesterTest extends LuceneTestCase {

View File

@ -17,8 +17,8 @@ package org.apache.lucene.analysis;
* limitations under the License.
*/
import static org.apache.lucene.util.automaton.BasicAutomata.makeEmpty;
import static org.apache.lucene.util.automaton.BasicAutomata.makeString;
import static org.apache.lucene.util.automaton.BasicAutomata.makeEmptyLight;
import static org.apache.lucene.util.automaton.BasicAutomata.makeStringLight;
import java.io.IOException;
import java.util.Arrays;
@ -39,20 +39,20 @@ import org.apache.lucene.util.automaton.CharacterRunAutomaton;
public final class MockTokenFilter extends TokenFilter {
/** Empty set of stopwords */
public static final CharacterRunAutomaton EMPTY_STOPSET =
new CharacterRunAutomaton(makeEmpty());
new CharacterRunAutomaton(makeEmptyLight());
/** Set of common english stopwords */
public static final CharacterRunAutomaton ENGLISH_STOPSET =
new CharacterRunAutomaton(BasicOperations.union(Arrays.asList(
makeString("a"), makeString("an"), makeString("and"), makeString("are"),
makeString("as"), makeString("at"), makeString("be"), makeString("but"),
makeString("by"), makeString("for"), makeString("if"), makeString("in"),
makeString("into"), makeString("is"), makeString("it"), makeString("no"),
makeString("not"), makeString("of"), makeString("on"), makeString("or"),
makeString("such"), makeString("that"), makeString("the"), makeString("their"),
makeString("then"), makeString("there"), makeString("these"), makeString("they"),
makeString("this"), makeString("to"), makeString("was"), makeString("will"),
makeString("with"))));
new CharacterRunAutomaton(BasicOperations.unionLight(Arrays.asList(
makeStringLight("a"), makeStringLight("an"), makeStringLight("and"), makeStringLight("are"),
makeStringLight("as"), makeStringLight("at"), makeStringLight("be"), makeStringLight("but"),
makeStringLight("by"), makeStringLight("for"), makeStringLight("if"), makeStringLight("in"),
makeStringLight("into"), makeStringLight("is"), makeStringLight("it"), makeStringLight("no"),
makeStringLight("not"), makeStringLight("of"), makeStringLight("on"), makeStringLight("or"),
makeStringLight("such"), makeStringLight("that"), makeStringLight("the"), makeStringLight("their"),
makeStringLight("then"), makeStringLight("there"), makeStringLight("these"), makeStringLight("they"),
makeStringLight("this"), makeStringLight("to"), makeStringLight("was"), makeStringLight("will"),
makeStringLight("with"))));
private final CharacterRunAutomaton filter;

View File

@ -57,7 +57,7 @@ public abstract class SearchEquivalenceTestBase extends LuceneTestCase {
Random random = random();
directory = newDirectory();
stopword = "" + randomChar();
CharacterRunAutomaton stopset = new CharacterRunAutomaton(BasicAutomata.makeString(stopword));
CharacterRunAutomaton stopset = new CharacterRunAutomaton(BasicAutomata.makeStringLight(stopword));
analyzer = new MockAnalyzer(random, MockTokenizer.WHITESPACE, false, stopset);
RandomIndexWriter iw = new RandomIndexWriter(random, directory, analyzer);
Document doc = new Document();

View File

@ -129,143 +129,6 @@ public class AutomatonTestUtil {
return code;
}
/**
* Lets you retrieve random strings accepted
* by an Automaton.
* <p>
* Once created, call {@link #getRandomAcceptedString(Random)}
* to get a new string (in UTF-32 codepoints).
*/
public static class RandomAcceptedStrings {
private final Map<Transition,Boolean> leadsToAccept;
private final Automaton a;
private static class ArrivingTransition {
final State from;
final Transition t;
public ArrivingTransition(State from, Transition t) {
this.from = from;
this.t = t;
}
}
public RandomAcceptedStrings(Automaton a) {
this.a = a;
if (a.isSingleton()) {
leadsToAccept = null;
return;
}
// must use IdentityHashmap because two Transitions w/
// different start nodes can be considered the same
leadsToAccept = new IdentityHashMap<>();
final Map<State,List<ArrivingTransition>> allArriving = new HashMap<>();
final LinkedList<State> q = new LinkedList<>();
final Set<State> seen = new HashSet<>();
// reverse map the transitions, so we can quickly look
// up all arriving transitions to a given state
for(State s: a.getNumberedStates()) {
for(int i=0;i<s.numTransitions;i++) {
final Transition t = s.transitionsArray[i];
List<ArrivingTransition> tl = allArriving.get(t.to);
if (tl == null) {
tl = new ArrayList<>();
allArriving.put(t.to, tl);
}
tl.add(new ArrivingTransition(s, t));
}
if (s.accept) {
q.add(s);
seen.add(s);
}
}
// Breadth-first search, from accept states,
// backwards:
while(!q.isEmpty()) {
final State s = q.removeFirst();
List<ArrivingTransition> arriving = allArriving.get(s);
if (arriving != null) {
for(ArrivingTransition at : arriving) {
final State 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<>();
if (a.isSingleton()) {
// accepts only one
final String s = a.singleton;
int charUpto = 0;
while(charUpto < s.length()) {
final int cp = s.codePointAt(charUpto);
charUpto += Character.charCount(cp);
soFar.add(cp);
}
} else {
State s = a.initial;
while(true) {
if (s.accept) {
if (s.numTransitions == 0) {
// stop now
break;
} else {
if (r.nextBoolean()) {
break;
}
}
}
if (s.numTransitions == 0) {
throw new RuntimeException("this automaton has dead states");
}
boolean cheat = r.nextBoolean();
final Transition t;
if (cheat) {
// pick a transition that we know is the fastest
// path to an accept state
List<Transition> toAccept = new ArrayList<>();
for(int i=0;i<s.numTransitions;i++) {
final Transition t0 = s.transitionsArray[i];
if (leadsToAccept.containsKey(t0)) {
toAccept.add(t0);
}
}
if (toAccept.size() == 0) {
// this is OK -- it means we jumped into a cycle
t = s.transitionsArray[r.nextInt(s.numTransitions)];
} else {
t = toAccept.get(r.nextInt(toAccept.size()));
}
} else {
t = s.transitionsArray[r.nextInt(s.numTransitions)];
}
soFar.add(getRandomCodePoint(r, t.min, t.max));
s = t.to;
}
}
return ArrayUtil.toIntArray(soFar);
}
}
/**
* Lets you retrieve random strings accepted
* by a LightAutomaton.
@ -446,16 +309,6 @@ public class AutomatonTestUtil {
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
/**
* Simple, original brics implementation of Brzozowski minimize()
*/
public static void minimizeSimple(Automaton a) {
if (a.isSingleton())
return;
determinizeSimple(a, SpecialOperations.reverse(a));
determinizeSimple(a, SpecialOperations.reverse(a));
}
/**
* Simple, original brics implementation of Brzozowski minimize()
*/
@ -467,65 +320,6 @@ public class AutomatonTestUtil {
return a;
}
/**
* Simple, original brics implementation of determinize()
*/
public static void determinizeSimple(Automaton a) {
if (a.deterministic || a.isSingleton())
return;
Set<State> initialset = new HashSet<>();
initialset.add(a.initial);
determinizeSimple(a, initialset);
}
/**
* Simple, original brics implementation of determinize()
* Determinizes the given automaton using the given set of initial states.
*/
public static void determinizeSimple(Automaton a, Set<State> initialset) {
int[] points = a.getStartPoints();
// subset construction
Map<Set<State>, Set<State>> sets = new HashMap<>();
LinkedList<Set<State>> worklist = new LinkedList<>();
Map<Set<State>, State> newstate = new HashMap<>();
sets.put(initialset, initialset);
worklist.add(initialset);
a.initial = new State();
newstate.put(initialset, a.initial);
while (worklist.size() > 0) {
Set<State> s = worklist.removeFirst();
State r = newstate.get(s);
for (State q : s)
if (q.accept) {
r.accept = true;
break;
}
for (int n = 0; n < points.length; n++) {
Set<State> p = new HashSet<>();
for (State q : s)
for (Transition t : q.getTransitions())
if (t.min <= points[n] && points[n] <= t.max)
p.add(t.to);
if (!sets.containsKey(p)) {
sets.put(p, p);
worklist.add(p);
newstate.put(p, new State());
}
State 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;
r.addTransition(new Transition(min, max, q));
}
}
a.deterministic = true;
a.clearNumberedStates();
a.removeDeadTransitions();
}
/**
* Simple, original brics implementation of determinize()
*/
@ -592,62 +386,6 @@ public class AutomatonTestUtil {
return BasicOperations.removeDeadTransitions(result.finish());
}
/**
* Simple, original implementation of getFiniteStrings.
*
* <p>Returns the set of accepted strings, assuming that at most
* <code>limit</code> strings are accepted. If more than <code>limit</code>
* strings are accepted, the first limit strings found are returned. If <code>limit</code>&lt;0, then
* the limit is infinite.
*
* <p>This implementation is recursive: it uses one stack
* frame for each digit in the returned strings (ie, max
* is the max length returned string).
*/
public static Set<IntsRef> getFiniteStringsRecursive(Automaton a, int limit) {
HashSet<IntsRef> strings = new HashSet<>();
if (a.isSingleton()) {
if (limit > 0) {
strings.add(Util.toUTF32(a.singleton, new IntsRef()));
}
} else if (!getFiniteStrings(a.initial, new HashSet<State>(), strings, new IntsRef(), limit)) {
return strings;
}
return strings;
}
/**
* Returns the strings that can be produced from the given state, or
* false if more than <code>limit</code> strings are found.
* <code>limit</code>&lt;0 means "infinite".
*/
private static boolean getFiniteStrings(State s, HashSet<State> pathstates,
HashSet<IntsRef> strings, IntsRef path, int limit) {
pathstates.add(s);
for (Transition t : s.getTransitions()) {
if (pathstates.contains(t.to)) {
return false;
}
for (int n = t.min; n <= t.max; n++) {
path.grow(path.length+1);
path.ints[path.length] = n;
path.length++;
if (t.to.accept) {
strings.add(IntsRef.deepCopyOf(path));
if (limit >= 0 && strings.size() > limit) {
return false;
}
}
if (!getFiniteStrings(t.to, pathstates, strings, path, limit)) {
return false;
}
path.length--;
}
}
pathstates.remove(s);
return true;
}
/**
* Simple, original implementation of getFiniteStrings.
*
@ -703,31 +441,6 @@ public class AutomatonTestUtil {
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(Automaton a) {
if (a.isSingleton()) return true;
return isFiniteSlow(a.initial, new HashSet<State>());
}
/**
* 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(State s, HashSet<State> path) {
path.add(s);
for (Transition t : s.getTransitions())
if (path.contains(t.to) || !isFiniteSlow(t.to, path)) return false;
path.remove(s);
return true;
}
/**
* Returns true if the language of this automaton is finite.
* <p>
@ -758,17 +471,6 @@ public class AutomatonTestUtil {
return true;
}
/**
* Checks that an automaton has no detached states that are unreachable
* from the initial state.
*/
public static void assertNoDetachedStates(Automaton a) {
int numStates = a.getNumberOfStates();
a.clearNumberedStates(); // force recomputation of cached numbered states
assert numStates == a.getNumberOfStates() : "automaton has " + (numStates - a.getNumberOfStates()) + " detached states";
}
/**
* Checks that an automaton has no detached states that are unreachable
* from the initial state.

View File

@ -17,6 +17,12 @@
package org.apache.solr.parser;
import java.io.StringReader;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.util.TokenFilterFactory;
import org.apache.lucene.index.Term;
@ -34,9 +40,9 @@ import org.apache.lucene.search.WildcardQuery;
import org.apache.lucene.util.QueryBuilder;
import org.apache.lucene.util.ToStringUtils;
import org.apache.lucene.util.Version;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.BasicAutomata;
import org.apache.lucene.util.automaton.BasicOperations;
import org.apache.lucene.util.automaton.LightAutomaton;
import org.apache.lucene.util.automaton.SpecialOperations;
import org.apache.solr.analysis.ReversedWildcardFilterFactory;
import org.apache.solr.analysis.TokenizerChain;
@ -49,12 +55,6 @@ import org.apache.solr.schema.TextField;
import org.apache.solr.search.QParser;
import org.apache.solr.search.SyntaxError;
import java.io.StringReader;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/** This class is overridden by QueryParser in QueryParser.jj
* and acts to separate the majority of the Java code from the .jj grammar file.
*/
@ -777,19 +777,22 @@ public abstract class SolrQueryParserBase extends QueryBuilder {
if (factory != null) {
Term term = new Term(field, termStr);
// fsa representing the query
Automaton automaton = WildcardQuery.toAutomaton(term);
LightAutomaton automaton = WildcardQuery.toAutomaton(term);
// TODO: we should likely use the automaton to calculate shouldReverse, too.
if (factory.shouldReverse(termStr)) {
automaton = BasicOperations.concatenate(automaton, BasicAutomata.makeChar(factory.getMarkerChar()));
SpecialOperations.reverse(automaton);
automaton = BasicOperations.concatenateLight(automaton, BasicAutomata.makeCharLight(factory.getMarkerChar()));
automaton = SpecialOperations.reverse(automaton);
// nocommit why did i have to insert det here? reverse didn't det before
automaton = BasicOperations.determinize(automaton);
} else {
// reverse wildcardfilter is active: remove false positives
// fsa representing false positives (markerChar*)
Automaton falsePositives = BasicOperations.concatenate(
BasicAutomata.makeChar(factory.getMarkerChar()),
BasicAutomata.makeAnyString());
LightAutomaton falsePositives = BasicOperations.concatenateLight(
BasicAutomata.makeCharLight(factory.getMarkerChar()),
BasicAutomata.makeAnyStringLight());
// subtract these away
automaton = BasicOperations.minus(automaton, falsePositives);
automaton = BasicOperations.minusLight(automaton, falsePositives);
// nocommit and do i need to det here?
}
return new AutomatonQuery(term, automaton) {
// override toString so its completely transparent

View File

@ -25,7 +25,6 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.*;
import org.apache.lucene.util.ToStringUtils;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.BasicAutomata;
import org.apache.lucene.util.automaton.BasicOperations;
import org.apache.lucene.util.automaton.SpecialOperations;

View File

@ -19,7 +19,6 @@ package org.apache.solr.analysis;
import java.io.IOException;
import java.lang.reflect.Field;
import java.util.HashMap;
import java.util.Map;
@ -27,7 +26,7 @@ import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.search.AutomatonQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.LightAutomaton;
import org.apache.lucene.util.automaton.SpecialOperations;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.request.SolrQueryRequest;
@ -160,11 +159,7 @@ public class TestReversedWildcardFilterFactory extends SolrTestCaseJ4 {
Query q = qp.parse(query);
if (!(q instanceof AutomatonQuery))
return false;
// this is a hack to get the protected Automaton field in AutomatonQuery,
// may break in later lucene versions - we have no getter... for good reasons.
final Field automatonField = AutomatonQuery.class.getDeclaredField("automaton");
automatonField.setAccessible(true);
Automaton automaton = (Automaton) automatonField.get(q);
LightAutomaton automaton = ((AutomatonQuery) q).getLightAutomaton();
String prefix = SpecialOperations.getCommonPrefix(automaton);
return prefix.length() > 0 && prefix.charAt(0) == '\u0001';
}

View File

@ -48,7 +48,7 @@ public class MockTokenFilterFactory extends TokenFilterFactory {
}
} else if (null != stopregex) {
RegExp regex = new RegExp(stopregex);
filter = new CharacterRunAutomaton(regex.toAutomaton());
filter = new CharacterRunAutomaton(regex.toLightAutomaton());
} else {
throw new IllegalArgumentException
("Configuration Error: either the 'stopset' or the 'stopregex' parameter must be specified.");
@ -62,4 +62,4 @@ public class MockTokenFilterFactory extends TokenFilterFactory {
public MockTokenFilter create(TokenStream stream) {
return new MockTokenFilter(stream, filter);
}
}
}