private void checkIfPreviousPatternGreedy() { if (previous != null && previous.getQuantifier().hasProperty(Quantifier.QuantifierProperty.GREEDY)) { throw new MalformedPatternException("Optional pattern cannot be preceded by greedy pattern"); } }
private void checkIfNoNotPattern() { if (quantifier.getConsumingStrategy() == ConsumingStrategy.NOT_FOLLOW || quantifier.getConsumingStrategy() == ConsumingStrategy.NOT_NEXT) { throw new MalformedPatternException("Option not applicable to NOT pattern"); } }
private void checkIfQuantifierApplied() { if (!quantifier.hasProperty(Quantifier.QuantifierProperty.SINGLE)) { throw new MalformedPatternException("Already applied quantifier to this Pattern. " + "Current quantifier is: " + quantifier); } }
/** * Specifies that this pattern can occur the specified times at least. * This means at least the specified times and at most infinite number of events can * be matched to this pattern. * * @return The same pattern with a {@link Quantifier#looping(ConsumingStrategy)} quantifier applied. * @throws MalformedPatternException if the quantifier is not applicable to this pattern. */ public Pattern<T, F> timesOrMore(int times) { checkIfNoNotPattern(); checkIfQuantifierApplied(); this.quantifier = Quantifier.looping(quantifier.getConsumingStrategy()); this.times = Times.of(times); return this; }
/** * Checks if the given pattern is optional. If the given pattern is the head of a group pattern, * the optional status depends on the group pattern. */ private boolean isPatternOptional(Pattern<T, ?> pattern) { if (headOfGroup(pattern)) { return isCurrentGroupPatternFirstOfLoop() && currentGroupPattern.getQuantifier().hasProperty(Quantifier.QuantifierProperty.OPTIONAL); } else { return pattern.getQuantifier().hasProperty(Quantifier.QuantifierProperty.OPTIONAL); } }
/** * Specifies that this pattern is greedy. * This means as many events as possible will be matched to this pattern. * * @return The same pattern with {@link Quantifier#greedy} set to true. * @throws MalformedPatternException if the quantifier is not applicable to this pattern. */ public Pattern<T, F> greedy() { checkIfNoNotPattern(); checkIfNoGroupPattern(); this.quantifier.greedy(); return this; }
public void combinations() { checkPattern(!hasProperty(QuantifierProperty.SINGLE), "Combinations not applicable to " + this + "!"); checkPattern(innerConsumingStrategy != ConsumingStrategy.STRICT, "You can apply either combinations or consecutive, not both!"); checkPattern(innerConsumingStrategy != ConsumingStrategy.SKIP_TILL_ANY, "Combinations already applied!"); innerConsumingStrategy = ConsumingStrategy.SKIP_TILL_ANY; }
/** * Specifies that this pattern is optional for a final match of the pattern * sequence to happen. * * @return The same pattern as optional. * @throws MalformedPatternException if the quantifier is not applicable to this pattern. */ public Pattern<T, F> optional() { checkIfPreviousPatternGreedy(); quantifier.optional(); return this; }
/** * Appends a new pattern to the existing one. The new pattern enforces non-strict * temporal contiguity. This means that a matching event of this pattern and the * preceding matching event might be interleaved with other events which are ignored. * * @param name Name of the new pattern * @return A new pattern which is appended to this one */ public Pattern<T, T> followedBy(final String name) { return new Pattern<>(name, this, ConsumingStrategy.SKIP_TILL_NEXT, afterMatchSkipStrategy); }
/** * Checks if the given pattern is the head pattern of the current group pattern. * * @param pattern the pattern to be checked * @return {@code true} iff the given pattern is in a group pattern and it is the head pattern of the * group pattern, {@code false} otherwise */ private boolean headOfGroup(Pattern<T, ?> pattern) { return currentGroupPattern != null && pattern.getPrevious() == null; }
private void checkIfNoGroupPattern() { if (this instanceof GroupPattern) { throw new MalformedPatternException("Option not applicable to group pattern"); } }
/** * Starts a new pattern sequence. The provided pattern is the initial pattern * of the new sequence. * * * @param group the pattern to begin with * @param afterMatchSkipStrategy the {@link AfterMatchSkipStrategy.SkipStrategy} to use after each match. * @return The first pattern of a pattern sequence */ public static <T, F extends T> GroupPattern<T, F> begin(final Pattern<T, F> group, final AfterMatchSkipStrategy afterMatchSkipStrategy) { return new GroupPattern<>(null, group, ConsumingStrategy.STRICT, afterMatchSkipStrategy); }
protected Pattern( final String name, final Pattern<T, ? extends T> previous, final ConsumingStrategy consumingStrategy, final AfterMatchSkipStrategy afterMatchSkipStrategy) { this.name = name; this.previous = previous; this.quantifier = Quantifier.one(consumingStrategy); this.afterMatchSkipStrategy = afterMatchSkipStrategy; }
public void consecutive() { checkPattern(hasProperty(QuantifierProperty.LOOPING) || hasProperty(QuantifierProperty.TIMES), "Consecutive not applicable to " + this + "!"); checkPattern(innerConsumingStrategy != ConsumingStrategy.SKIP_TILL_ANY, "You can apply either combinations or consecutive, not both!"); checkPattern(innerConsumingStrategy != ConsumingStrategy.STRICT, "Consecutive already applied!"); innerConsumingStrategy = ConsumingStrategy.STRICT; }
/** * Starts a new pattern sequence. The provided name is the one of the initial pattern * of the new sequence. Furthermore, the base type of the event sequence is set. * * @param name The name of starting pattern of the new pattern sequence * @param afterMatchSkipStrategy the {@link AfterMatchSkipStrategy.SkipStrategy} to use after each match. * @param <X> Base type of the event pattern * @return The first pattern of a pattern sequence */ public static <X> Pattern<X, X> begin(final String name, final AfterMatchSkipStrategy afterMatchSkipStrategy) { return new Pattern<X, X>(name, null, ConsumingStrategy.STRICT, afterMatchSkipStrategy); }
/** * Appends a new group pattern to the existing one. The new pattern enforces non-strict * temporal contiguity. This means that a matching event of this pattern and the * preceding matching event might be interleaved with other events which are ignored. * * @param group the pattern to append * @return A new pattern which is appended to this one */ public GroupPattern<T, F> followedBy(Pattern<T, F> group) { return new GroupPattern<>(this, group, ConsumingStrategy.SKIP_TILL_NEXT, afterMatchSkipStrategy); }
/** * Appends a new pattern to the existing one. The new pattern enforces strict * temporal contiguity. This means that the whole pattern sequence matches only * if an event which matches this pattern directly follows the preceding matching * event. Thus, there cannot be any events in between two matching events. * * @param name Name of the new pattern * @return A new pattern which is appended to this one */ public Pattern<T, T> next(final String name) { return new Pattern<>(name, this, ConsumingStrategy.STRICT, afterMatchSkipStrategy); }
/** * Appends a new group pattern to the existing one. The new pattern enforces non-strict * temporal contiguity. This means that a matching event of this pattern and the * preceding matching event might be interleaved with other events which are ignored. * * @param group the pattern to append * @return A new pattern which is appended to this one */ public GroupPattern<T, F> followedByAny(Pattern<T, F> group) { return new GroupPattern<>(this, group, ConsumingStrategy.SKIP_TILL_ANY, afterMatchSkipStrategy); }
/** * Appends a new pattern to the existing one. The new pattern enforces non-strict * temporal contiguity. This means that a matching event of this pattern and the * preceding matching event might be interleaved with other events which are ignored. * * @param name Name of the new pattern * @return A new pattern which is appended to this one */ public Pattern<T, T> followedByAny(final String name) { return new Pattern<>(name, this, ConsumingStrategy.SKIP_TILL_ANY, afterMatchSkipStrategy); }
/** * Appends a new group pattern to the existing one. The new pattern enforces strict * temporal contiguity. This means that the whole pattern sequence matches only * if an event which matches this pattern directly follows the preceding matching * event. Thus, there cannot be any events in between two matching events. * * @param group the pattern to append * @return A new pattern which is appended to this one */ public GroupPattern<T, F> next(Pattern<T, F> group) { return new GroupPattern<>(this, group, ConsumingStrategy.STRICT, afterMatchSkipStrategy); }