/** * Creates a simple single state. For an OPTIONAL state it also consists * of a similar state without the PROCEED edge, so that for each PROCEED transition branches * in computation state graph can be created only once. * * @param sinkState state that the state being converted should point to * @return the created state */ @SuppressWarnings("unchecked") private State<T> createSingletonState(final State<T> sinkState) { return createSingletonState( sinkState, sinkState, getTakeCondition(currentPattern), getIgnoreCondition(currentPattern), isPatternOptional(currentPattern)); }
/** * Compiles the given pattern into a {@link NFAFactory}. The NFA factory can be used to create * multiple NFAs. * * @param pattern Definition of sequence pattern * @param timeoutHandling True if the NFA shall return timed out event patterns * @param <T> Type of the input events * @return Factory for NFAs corresponding to the given pattern */ @SuppressWarnings("unchecked") public static <T> NFAFactory<T> compileFactory( final Pattern<T, ?> pattern, boolean timeoutHandling) { if (pattern == null) { // return a factory for empty NFAs return new NFAFactoryImpl<>(0, Collections.<State<T>>emptyList(), timeoutHandling); } else { final NFAFactoryCompiler<T> nfaFactoryCompiler = new NFAFactoryCompiler<>(pattern); nfaFactoryCompiler.compileFactory(); return new NFAFactoryImpl<>(nfaFactoryCompiler.getWindowTime(), nfaFactoryCompiler.getStates(), timeoutHandling); } }
/** * Check if there are duplicate pattern names. If yes, it * throws a {@link MalformedPatternException}. */ private void checkPatternNameUniqueness() { // make sure there is no pattern with name "$endState$" stateNameHandler.checkNameUniqueness(ENDING_STATE_NAME); Pattern patternToCheck = currentPattern; while (patternToCheck != null) { checkPatternNameUniqueness(patternToCheck); patternToCheck = patternToCheck.getPrevious(); } stateNameHandler.clear(); }
/** * Compiles the given pattern into a {@link NFA}. * * @param pattern Definition of sequence pattern * @param timeoutHandling True if the NFA shall return timed out event patterns * @param <T> Type of the input events * @return Non-deterministic finite automaton representing the given pattern */ public static <T> NFA<T> compile(Pattern<T, ?> pattern, boolean timeoutHandling) { NFACompiler.NFAFactory<T> factory = compileFactory(pattern, timeoutHandling); return factory.createNFA(); }
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); } }
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; }
/** * Check if the given pattern's name is already used or not. If yes, it * throws a {@link MalformedPatternException}. * * @param pattern The pattern to be checked */ private void checkPatternNameUniqueness(final Pattern pattern) { if (pattern instanceof GroupPattern) { Pattern patternToCheck = ((GroupPattern) pattern).getRawPattern(); while (patternToCheck != null) { checkPatternNameUniqueness(patternToCheck); patternToCheck = patternToCheck.getPrevious(); } } else { stateNameHandler.checkNameUniqueness(pattern.getName()); } }
private void addStopStateToLooping(final State<T> loopingState) { if (followingPattern != null && followingPattern.getQuantifier().getConsumingStrategy() == Quantifier.ConsumingStrategy.NOT_FOLLOW) { final IterativeCondition<T> notCondition = getTakeCondition(followingPattern); final State<T> stopState = createStopState(notCondition, followingPattern.getName()); loopingState.addProceed(stopState, notCondition); } }
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; }
/** * Creates the Start {@link State} of the resulting NFA graph. * * @param sinkState the state that Start state should point to (always first state of middle states) * @return created state */ @SuppressWarnings("unchecked") private State<T> createStartState(State<T> sinkState) { final State<T> beginningState = convertPattern(sinkState); beginningState.makeStart(); return beginningState; }
/** * 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; }
public NodeId getNodeId(String prevState, long timestamp, int counter, V event) { return mappingContext.get(Tuple2.of(NFAStateNameHandler.getOriginalNameFromInternal(prevState), new ValueTimeWrapper<>(event, timestamp, counter))); }
/** * @return An true function extended with stop(until) condition if necessary. */ @SuppressWarnings("unchecked") private IterativeCondition<T> getTrueFunction() { IterativeCondition<T> trueCondition = BooleanConditions.trueFunction(); if (currentGroupPattern != null && currentGroupPattern.getUntilCondition() != null) { trueCondition = extendWithUntilCondition( trueCondition, (IterativeCondition<T>) currentGroupPattern.getUntilCondition(), true); } return trueCondition; }
@Override public NFA<Event> createNFA() { Pattern<Event, ?> pattern = Pattern.begin("1"); return NFACompiler.compileFactory(pattern, false).createNFA(); } }
/** * Creates the dummy Final {@link State} of the NFA graph. * @return dummy Final state */ private State<T> createEndingState() { State<T> endState = createState(ENDING_STATE_NAME, State.StateType.Final); windowTime = currentPattern.getWindowTime() != null ? currentPattern.getWindowTime().toMilliseconds() : 0L; return endState; }
private NFA<Event> getSingleElementAlwaysTrueNFA() { return NFACompiler.compileFactory(Pattern.<Event>begin("A"), false).createNFA(); }
@Override public NFA<Event> createNFA() { Pattern<Event, ?> pattern = Pattern.<Event>begin("1").next("2").within(Time.milliseconds(10)); return NFACompiler.compileFactory(pattern, true).createNFA(); } }
@Override public NFA<Event> createNFA() { Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new StartFilter()) .within(Time.milliseconds(10L)); return NFACompiler.compileFactory(pattern, handleTimeout).createNFA(); } }
@Override public NFA<Event> createNFA() { Pattern<Event, ?> pattern = Pattern.<Event>begin("start") .subtype(SubEvent.class) .where(new MiddleFilter()) .or(new SubEventEndFilter()) .times(2) .within(Time.milliseconds(10L)); return NFACompiler.compileFactory(pattern, handleTimeout).createNFA(); } }
@Override public NFA<Event> createNFA() { Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new StartFilter()) .followedByAny("middle") .subtype(SubEvent.class) .where(new MiddleFilter()) .followedByAny("end") .where(new EndFilter()) // add a window timeout to test whether timestamps of elements in the // priority queue in CEP operator are correctly checkpointed/restored .within(Time.milliseconds(10L)); return NFACompiler.compileFactory(pattern, handleTimeout).createNFA(); } }