private NFA<Event> createStartEndNFA() { State<Event> startState = new State<>("start", State.StateType.Start); State<Event> endState = new State<>("end", State.StateType.Normal); State<Event> endingState = new State<>("", State.StateType.Final); startState.addTake( endState, new SimpleCondition<Event>() { private static final long serialVersionUID = -4869589195918650396L; @Override public boolean filter(Event value) throws Exception { return value.getName().equals("start"); } }); endState.addTake( endingState, new SimpleCondition<Event>() { private static final long serialVersionUID = 2979804163709590673L; @Override public boolean filter(Event value) throws Exception { return value.getName().equals("end"); } }); endState.addIgnore(BooleanConditions.<Event>trueFunction()); List<State<Event>> states = new ArrayList<>(); states.add(startState); states.add(endState); states.add(endingState); return new NFA<>(states, 2L, false); }
private State<T> copy(final State<T> state) { final State<T> copyOfState = createState( NFAStateNameHandler.getOriginalNameFromInternal(state.getName()), state.getStateType()); for (StateTransition<T> tStateTransition : state.getStateTransitions()) { copyOfState.addStateTransition( tStateTransition.getAction(), tStateTransition.getTargetState().equals(tStateTransition.getSourceState()) ? copyOfState : tStateTransition.getTargetState(), tStateTransition.getCondition()); } return copyOfState; }
for (StateTransition<?> transition : currentState.getStateTransitions()) { if (transition.getAction() == StateTransitionAction.PROCEED) { if (transition.getTargetState().isFinal()) { return true; } else {
singletonState.addTake(sink, takeCondition); (IterativeCondition<T>) currentPattern.getUntilCondition(); if (untilCondition != null) { singletonState.addProceed( originalStateMap.get(proceedState.getName()), new RichAndCondition<>(proceedCondition, untilCondition)); singletonState.addProceed(proceedState, untilCondition != null ? new RichAndCondition<>(proceedCondition, new RichNotCondition<>(untilCondition)) : proceedCondition); } else { singletonState.addProceed(proceedState, proceedCondition); if (isOptional) { ignoreState = createState(currentPattern.getName(), State.StateType.Normal); ignoreState.addTake(sink, takeCondition); ignoreState.addIgnore(ignoreCondition); addStopStates(ignoreState); } else { ignoreState = singletonState; singletonState.addIgnore(ignoreState, ignoreCondition);
final State<T> stopState = createStopState(notCondition, currentPattern.getName()); if (lastSink.isFinal()) { notNext.addIgnore(lastSink, new RichNotCondition<>(notCondition)); } else { notNext.addProceed(lastSink, new RichNotCondition<>(notCondition)); notNext.addProceed(stopState, notCondition); lastSink = notNext; } else {
/** * Creates a pair of states that enables relaxed strictness before a zeroOrMore looping state. * * @param loopingState the first state of zeroOrMore complex state * @param lastSink the state that the looping one points to * @return the newly created state */ @SuppressWarnings("unchecked") private State<T> createInitOptionalStateOfZeroOrMore(final State<T> loopingState, final State<T> lastSink) { final IterativeCondition<T> currentCondition = (IterativeCondition<T>) currentPattern.getCondition(); final State<T> firstState = createState(currentPattern.getName(), State.StateType.Normal); firstState.addProceed(lastSink, BooleanConditions.<T>trueFunction()); firstState.addTake(loopingState, currentCondition); final IterativeCondition<T> ignoreFunction = getIgnoreCondition(currentPattern); if (ignoreFunction != null) { final State<T> firstStateWithoutProceed = createState(currentPattern.getName(), State.StateType.Normal); firstState.addIgnore(firstStateWithoutProceed, ignoreFunction); firstStateWithoutProceed.addIgnore(ignoreFunction); firstStateWithoutProceed.addTake(loopingState, currentCondition); addStopStates(firstStateWithoutProceed); } return firstState; }
public static <T> Map<String, State<T>> migrateGraph(State<T> oldStartState) { State<T> oldFirst = oldStartState; State<T> oldSecond = oldStartState.getStateTransitions().iterator().next().getTargetState(); oldFirst.getStateTransitions().iterator(), new Predicate<StateTransition<T>>() { @Override oldFirst.getStateTransitions().iterator(), new Predicate<StateTransition<T>>() { @Override oldSecond.getStateTransitions().iterator(), new Predicate<StateTransition<T>>() { @Override State<T> newFirst = new State<>(oldSecond.getName(), State.StateType.Start); convertedStates.put(newFirst.getName(), newFirst); while (oldSecondToThirdTake != null) { newSecond = new State<T>(oldSecondToThirdTake.getTargetState().getName(), State.StateType.Normal); convertedStates.put(newSecond.getName(), newSecond); newFirst.addTake(newSecond, oldFirstToSecondTake.getCondition()); newFirst.addIgnore(oldFirstIgnore.getCondition()); oldFirst.getStateTransitions().iterator(), new Predicate<StateTransition<T>>() { @Override
stateMap.put(state.getName(), state); assertTrue(startState.isStart()); final Set<Tuple2<String, StateTransitionAction>> startTransitions = unfoldTransitions(startState); assertEquals(Sets.newHashSet( assertTrue(endingState.isFinal()); assertEquals(0, endingState.getStateTransitions().size());
@Override public boolean equals(Object obj) { if (obj instanceof StateTransition) { @SuppressWarnings("unchecked") StateTransition<T> other = (StateTransition<T>) obj; return action == other.action && sourceState.getName().equals(other.sourceState.getName()) && targetState.getName().equals(other.targetState.getName()); } else { return false; } }
private boolean isStartState(ComputationState state) { State<T> stateObject = getState(state); if (stateObject == null) { throw new FlinkRuntimeException("State " + state.getCurrentStateName() + " does not exist in the NFA. NFA has states " + states.values()); } return stateObject.isStart(); }
private <T> Set<Tuple2<String, StateTransitionAction>> unfoldTransitions(final State<T> state) { final Set<Tuple2<String, StateTransitionAction>> transitions = new HashSet<>(); for (StateTransition<T> transition : state.getStateTransitions()) { transitions.add(Tuple2.of( transition.getTargetState().getName(), transition.getAction())); } return transitions; }
private void serializeStates(Set<State<T>> states, DataOutputView out) throws IOException { TypeSerializer<String> nameSerializer = StringSerializer.INSTANCE; TypeSerializer<State.StateType> stateTypeSerializer = new EnumSerializer<>(State.StateType.class); TypeSerializer<StateTransitionAction> actionSerializer = new EnumSerializer<>(StateTransitionAction.class); out.writeInt(states.size()); for (State<T> state: states) { nameSerializer.serialize(state.getName(), out); stateTypeSerializer.serialize(state.getStateType(), out); } for (State<T> state: states) { nameSerializer.serialize(state.getName(), out); out.writeInt(state.getStateTransitions().size()); for (StateTransition<T> transition : state.getStateTransitions()) { nameSerializer.serialize(transition.getSourceState().getName(), out); nameSerializer.serialize(transition.getTargetState().getName(), out); actionSerializer.serialize(transition.getAction(), out); serializeCondition(transition.getCondition(), out); } } }
public void addTake(final State<T> targetState, final IterativeCondition<T> condition) { addStateTransition(StateTransitionAction.TAKE, targetState, condition); }
private void addStopStates(final State<T> state) { for (Tuple2<IterativeCondition<T>, String> notCondition: getCurrentNotCondition()) { final State<T> stopState = createStopState(notCondition.f0, notCondition.f1); state.addProceed(stopState, notCondition.f0); } }
/** * Tear-down method for the NFA. */ public void close() throws Exception { for (State<T> state : getStates()) { for (StateTransition<T> transition : state.getStateTransitions()) { IterativeCondition condition = transition.getCondition(); FunctionUtils.closeFunction(condition); } } }
/** * Creates a state with {@link State.StateType#Normal} and adds it to the collection of created states. * Should be used instead of instantiating with new operator. * * @return the created state */ private State<T> createState(String name, State.StateType stateType) { String stateName = stateNameHandler.getUniqueInternalName(name); State<T> state = new State<>(stateName, stateType); states.add(state); return state; }
private State<T> createStopState(final IterativeCondition<T> notCondition, final String name) { // We should not duplicate the notStates. All states from which we can stop should point to the same one. State<T> stopState = stopStates.get(name); if (stopState == null) { stopState = createState(name, State.StateType.Stop); stopState.addTake(notCondition); stopStates.put(name, stopState); } return stopState; }
private boolean isFinalState(ComputationState state) { State<T> stateObject = getState(state); if (stateObject == null) { throw new FlinkRuntimeException("State " + state.getCurrentStateName() + " does not exist in the NFA. NFA has states " + states.values()); } return stateObject.isFinal(); }
/** * Patterns with quantifiers AT_LEAST_ONE_* are created as a pair of states: a singleton state and * looping state. This method creates the first of the two. * * @param sinkState the state the newly created state should point to, it should be a looping state * @return the newly created state */ @SuppressWarnings("unchecked") private State<T> createInitMandatoryStateOfOneOrMore(final State<T> sinkState) { final IterativeCondition<T> currentCondition = (IterativeCondition<T>) currentPattern.getCondition(); final State<T> firstState = createState(currentPattern.getName(), State.StateType.Normal); firstState.addTake(sinkState, currentCondition); final IterativeCondition<T> ignoreCondition = getIgnoreCondition(currentPattern); if (ignoreCondition != null) { firstState.addIgnore(ignoreCondition); } return firstState; }